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 2016/03/15 18:44:34 UTC

[01/30] storm git commit: port backtype.storm.stats to java

Repository: storm
Updated Branches:
  refs/heads/master afcb2a065 -> fa25f3d7f


http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index 4b96620..bcf6e4f 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -21,13 +21,14 @@
         ring.middleware.multipart-params)
   (:use [ring.middleware.json :only [wrap-json-params]])
   (:use [hiccup core page-helpers])
-  (:use [org.apache.storm config util log stats zookeeper converter])
+  (:use [org.apache.storm config util log zookeeper converter])
   (:use [org.apache.storm.ui helpers])
   (:use [org.apache.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
                                               ACKER-FAIL-STREAM-ID mk-authorization-handler
                                               start-metrics-reporters]]])
   (:import [org.apache.storm.utils Time]
-           [org.apache.storm.generated NimbusSummary])
+           [org.apache.storm.generated NimbusSummary]
+           [org.apache.storm.stats StatsUtil])
   (:use [clojure.string :only [blank? lower-case trim split]])
   (:import [org.apache.storm.generated ExecutorSpecificStats
             ExecutorStats ExecutorSummary ExecutorInfo TopologyInfo SpoutStats BoltStats
@@ -109,7 +110,7 @@
 
 (defn executor-summary-type
   [topology ^ExecutorSummary s]
-  (component-type topology (.get_component_id s)))
+  (StatsUtil/componentType topology (.get_component_id s)))
 
 (defn is-ack-stream
   [stream]
@@ -119,6 +120,12 @@
          ACKER-FAIL-STREAM-ID]]
     (every? #(not= %1 stream) acker-streams)))
 
+(defn mk-include-sys-fn
+  [include-sys?]
+  (if include-sys?
+    (fn [_] true)
+    (fn [stream] (and (string? stream) (not (Utils/isSystemId stream))))))
+
 (defn spout-summary?
   [topology s]
   (= :spout (executor-summary-type topology s)))
@@ -167,7 +174,7 @@
 (defn get-error-data
   [error]
   (if error
-    (error-subset (.get_error ^ErrorInfo error))
+    (StatsUtil/errorSubset (.get_error ^ErrorInfo error))
     ""))
 
 (defn get-error-port
@@ -234,23 +241,23 @@
                    bolt-summs (get bolt-comp-summs id)
                    spout-summs (get spout-comp-summs id)
                    bolt-cap (if bolt-summs
-                              (compute-bolt-capacity bolt-summs)
+                              (StatsUtil/computeBoltCapacity bolt-summs)
                               0)]
                {:type (if bolt-summs "bolt" "spout")
                 :capacity bolt-cap
                 :latency (if bolt-summs
                            (get-in
-                             (bolt-streams-stats bolt-summs true)
+                             (clojurify-structure (StatsUtil/boltStreamsStats bolt-summs true))
                              [:process-latencies window])
                            (get-in
-                             (spout-streams-stats spout-summs true)
+                             (clojurify-structure (StatsUtil/spoutStreamsStats spout-summs true))
                              [:complete-latencies window]))
                 :transferred (or
                                (get-in
-                                 (spout-streams-stats spout-summs true)
+                                 (clojurify-structure (StatsUtil/spoutStreamsStats spout-summs true))
                                  [:transferred window])
                                (get-in
-                                 (bolt-streams-stats bolt-summs true)
+                                 (clojurify-structure (StatsUtil/boltStreamsStats bolt-summs true))
                                  [:transferred window]))
                 :stats (let [mapfn (fn [dat]
                                      (map (fn [^ExecutorSummary summ]
@@ -492,7 +499,7 @@
        "window" w
        "emitted" (get-in stats [:emitted w])
        "transferred" (get-in stats [:transferred w])
-       "completeLatency" (float-str (get-in stats [:complete-latencies w]))
+       "completeLatency" (StatsUtil/floatStr (get-in stats [:complete-latencies w]))
        "acked" (get-in stats [:acked w])
        "failed" (get-in stats [:failed w])})))
 
@@ -555,7 +562,7 @@
       (get-error-json topo-id (.get_last_error s) secure?)
       {"spoutId" id
        "encodedSpoutId" (URLEncoder/encode id)
-       "completeLatency" (float-str (.get_complete_latency_ms ss))})))
+       "completeLatency" (StatsUtil/floatStr (.get_complete_latency_ms ss))})))
 
 (defmethod comp-agg-stats-json ComponentType/BOLT
   [topo-id secure? [id ^ComponentAggregateStats s]]
@@ -566,10 +573,10 @@
       (get-error-json topo-id (.get_last_error s) secure?)
       {"boltId" id
        "encodedBoltId" (URLEncoder/encode id)
-       "capacity" (float-str (.get_capacity ss))
-       "executeLatency" (float-str (.get_execute_latency_ms ss))
+       "capacity" (StatsUtil/floatStr (.get_capacity ss))
+       "executeLatency" (StatsUtil/floatStr (.get_execute_latency_ms ss))
        "executed" (.get_executed ss)
-       "processLatency" (float-str (.get_process_latency_ms ss))})))
+       "processLatency" (StatsUtil/floatStr (.get_process_latency_ms ss))})))
 
 (defn- unpack-topology-page-info
   "Unpacks the serialized object to data structures"
@@ -679,10 +686,10 @@
      "transferred" (.get_transferred comm-s)
      "acked" (.get_acked comm-s)
      "failed" (.get_failed comm-s)
-     "executeLatency" (float-str (.get_execute_latency_ms bolt-s))
-     "processLatency"  (float-str (.get_process_latency_ms bolt-s))
+     "executeLatency" (StatsUtil/floatStr (.get_execute_latency_ms bolt-s))
+     "processLatency"  (StatsUtil/floatStr (.get_process_latency_ms bolt-s))
      "executed" (.get_executed bolt-s)
-     "capacity" (float-str (.get_capacity bolt-s))}))
+     "capacity" (StatsUtil/floatStr (.get_capacity bolt-s))}))
 
 (defmethod unpack-comp-agg-stat ComponentType/SPOUT
   [[window ^ComponentAggregateStats s]]
@@ -695,7 +702,7 @@
      "transferred" (.get_transferred comm-s)
      "acked" (.get_acked comm-s)
      "failed" (.get_failed comm-s)
-     "completeLatency" (float-str (.get_complete_latency_ms spout-s))}))
+     "completeLatency" (StatsUtil/floatStr (.get_complete_latency_ms spout-s))}))
 
 (defn- unpack-bolt-input-stat
   [[^GlobalStreamId s ^ComponentAggregateStats stats]]
@@ -706,8 +713,8 @@
     {"component" comp-id
      "encodedComponentId" (URLEncoder/encode comp-id)
      "stream" (.get_streamId s)
-     "executeLatency" (float-str (.get_execute_latency_ms bas))
-     "processLatency" (float-str (.get_process_latency_ms bas))
+     "executeLatency" (StatsUtil/floatStr (.get_execute_latency_ms bas))
+     "processLatency" (StatsUtil/floatStr (.get_process_latency_ms bas))
      "executed" (Utils/nullToZero (.get_executed bas))
      "acked" (Utils/nullToZero (.get_acked cas))
      "failed" (Utils/nullToZero (.get_failed cas))}))
@@ -730,7 +737,7 @@
     {"stream" stream-id
      "emitted" (Utils/nullToZero (.get_emitted cas))
      "transferred" (Utils/nullToZero (.get_transferred cas))
-     "completeLatency" (float-str (.get_complete_latency_ms spout-s))
+     "completeLatency" (StatsUtil/floatStr (.get_complete_latency_ms spout-s))
      "acked" (Utils/nullToZero (.get_acked cas))
      "failed" (Utils/nullToZero (.get_failed cas))}))
 
@@ -757,10 +764,10 @@
      "port" port
      "emitted" (Utils/nullToZero (.get_emitted cas))
      "transferred" (Utils/nullToZero (.get_transferred cas))
-     "capacity" (float-str (Utils/nullToZero (.get_capacity bas)))
-     "executeLatency" (float-str (.get_execute_latency_ms bas))
+     "capacity" (StatsUtil/floatStr (Utils/nullToZero (.get_capacity bas)))
+     "executeLatency" (StatsUtil/floatStr (.get_execute_latency_ms bas))
      "executed" (Utils/nullToZero (.get_executed bas))
-     "processLatency" (float-str (.get_process_latency_ms bas))
+     "processLatency" (StatsUtil/floatStr (.get_process_latency_ms bas))
      "acked" (Utils/nullToZero (.get_acked cas))
      "failed" (Utils/nullToZero (.get_failed cas))
      "workerLogLink" (worker-log-link host port topology-id secure?)}))
@@ -785,7 +792,7 @@
      "port" port
      "emitted" (Utils/nullToZero (.get_emitted cas))
      "transferred" (Utils/nullToZero (.get_transferred cas))
-     "completeLatency" (float-str (.get_complete_latency_ms sas))
+     "completeLatency" (StatsUtil/floatStr (.get_complete_latency_ms sas))
      "acked" (Utils/nullToZero (.get_acked cas))
      "failed" (Utils/nullToZero (.get_failed cas))
      "workerLogLink" (worker-log-link host port topology-id secure?)}))

http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/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 ce58f42..a76db54 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -15,14 +15,15 @@
 ;; limitations under the License.
 (ns org.apache.storm.nimbus-test
   (:use [clojure test])
-  (:require [org.apache.storm [util :as util] [stats :as stats]])
+  (:require [org.apache.storm [util :as util]])
   (:require [org.apache.storm.daemon [nimbus :as nimbus]])
   (:require [org.apache.storm [converter :as converter]])
   (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
             TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
            [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
            [org.apache.storm.generated GlobalStreamId]
-           [org.apache.storm Thrift])
+           [org.apache.storm Thrift]
+           [org.apache.storm.stats StatsUtil])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
   (:import [org.apache.storm.scheduler INimbus])
   (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
@@ -139,7 +140,8 @@
         curr-beat (.get-worker-heartbeat state storm-id node port)
         stats (:executor-stats curr-beat)]
     (.worker-heartbeat! state storm-id node port
-      {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
+      {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10
+       :executor-stats (merge stats {executor (clojurify-structure (StatsUtil/renderStats (StatsUtil/mkBoltStats 20)))})}
       )))
 
 (defn slot-assignments [cluster storm-id]


[15/30] storm git commit: upmerge storm master

Posted by bo...@apache.org.
upmerge storm master


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

Branch: refs/heads/master
Commit: 69f2906c0748826101511b1dd6005f910ebfd5fc
Parents: faaacae 9aa8bf0
Author: 卫乐 <we...@taobao.com>
Authored: Tue Mar 1 11:49:59 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Tue Mar 1 11:49:59 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   2 +
 README.markdown                                 |   4 +
 .../src/clj/org/apache/storm/daemon/acker.clj   | 108 ----------------
 .../src/clj/org/apache/storm/daemon/common.clj  |  17 ++-
 storm-core/src/clj/org/apache/storm/testing.clj |   7 +-
 .../storm/blobstore/LocalFsBlobStore.java       |   2 +-
 .../src/jvm/org/apache/storm/daemon/Acker.java  | 128 +++++++++++++++++++
 .../apache/storm/topology/TopologyBuilder.java  |  13 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   |   4 +
 .../storm/topology/TopologyBuilderTest.java     |  65 ++++++++++
 10 files changed, 226 insertions(+), 124 deletions(-)
----------------------------------------------------------------------



[08/30] storm git commit: added last-error to stats

Posted by bo...@apache.org.
added last-error to stats


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

Branch: refs/heads/master
Commit: e5564c0f888e40af2726a645d24cfad0aaeed26a
Parents: 8801348
Author: 卫乐 <we...@taobao.com>
Authored: Thu Feb 25 15:06:59 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Thu Feb 25 15:06:59 2016 +0800

----------------------------------------------------------------------
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |  8 ++----
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 26 ++++++++++++--------
 2 files changed, 18 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e5564c0f/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index a0e652b..f58353a 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -2109,19 +2109,15 @@
         [this ^String topo-id ^String window ^boolean include-sys?]
         (mark! nimbus:num-getTopologyPageInfo-calls)
         (let [info (get-common-topo-info topo-id "getTopologyPageInfo")
-
               exec->node+port (:executor->node+port (:assignment info))
-              last-err-fn (partial get-last-error
-                                   (:storm-cluster-state info)
-                                   topo-id)
-              ;;TODO: add last-error-fn to aggTopoExecsStats method
               topo-page-info (StatsUtil/aggTopoExecsStats topo-id
                                                          exec->node+port
                                                          (:task->component info)
                                                          (:beats info)
                                                          (:topology info)
                                                          window
-                                                         include-sys?)]
+                                                         include-sys?
+                                                         (:storm-cluster-state info))]
           (when-let [owner (:owner (:base info))]
             (.set_owner topo-page-info owner))
           (when-let [sched-status (.get @(:id->sched-status nimbus) topo-id)]

http://git-wip-us.apache.org/repos/asf/storm/blob/e5564c0f/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index 22ececf..c06d7db 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -18,7 +18,6 @@
 package org.apache.storm.stats;
 
 import clojure.lang.Keyword;
-import clojure.lang.PersistentVector;
 import clojure.lang.RT;
 import com.google.common.collect.Lists;
 import java.util.ArrayList;
@@ -28,6 +27,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.storm.cluster.IStormClusterState;
 import org.apache.storm.generated.Bolt;
 import org.apache.storm.generated.BoltAggregateStats;
 import org.apache.storm.generated.BoltStats;
@@ -543,13 +543,12 @@ public class StatsUtil {
         return ret;
     }
 
-    // TODO: add last-error-fn arg to get last error
     public static TopologyPageInfo aggTopoExecsStats(
             String topologyId, Map exec2nodePort, Map task2component,
-            Map beats, StormTopology topology, String window, boolean includeSys) {
+            Map beats, StormTopology topology, String window, boolean includeSys, IStormClusterState clusterState) {
         List beatList = extractDataFromHb(exec2nodePort, task2component, beats, includeSys, topology);
         Map topoStats = aggregateTopoStats(window, includeSys, beatList);
-        topoStats = postAggregateTopoStats(task2component, exec2nodePort, topoStats);
+        topoStats = postAggregateTopoStats(task2component, exec2nodePort, topoStats, topologyId, clusterState);
 
         return thriftifyTopoPageData(topologyId, topoStats);
     }
@@ -574,7 +573,8 @@ public class StatsUtil {
         return initVal;
     }
 
-    public static Map postAggregateTopoStats(Map task2comp, Map exec2nodePort, Map accData) {
+    public static Map postAggregateTopoStats(
+            Map task2comp, Map exec2nodePort, Map accData, String topologyId, IStormClusterState clusterState) {
         Map ret = new HashMap();
         putRawKV(ret, NUM_TASKS, task2comp.size());
         putRawKV(ret, NUM_WORKERS, ((Set) getByKeyword(accData, WORKERS_SET)).size());
@@ -596,8 +596,7 @@ public class StatsUtil {
             }
             removeByKeyword(m, EXEC_LAT_TOTAL);
             removeByKeyword(m, PROC_LAT_TOTAL);
-            //TODO: get last error depends on cluster.clj
-            putRawKV(m, "last-error", null);
+            putRawKV(m, "last-error", getLastError(clusterState, topologyId, id));
 
             aggBolt2stats.put(id, m);
         }
@@ -615,8 +614,7 @@ public class StatsUtil {
                 putRawKV(m, COMP_LATENCY, compLatencyTotal / acked);
             }
             removeByKeyword(m, COMP_LAT_TOTAL);
-            //TODO: get last error depends on cluster.clj
-            putRawKV(m, "last-error", null);
+            putRawKV(m, "last-error", getLastError(clusterState, topologyId, id));
 
             spoutBolt2stats.put(id, m);
         }
@@ -1493,6 +1491,7 @@ public class StatsUtil {
     }
 
     private static ComponentAggregateStats thriftifySpoutAggStats(Map m) {
+        logger.warn("spout agg stats:{}", m);
         ComponentAggregateStats stats = new ComponentAggregateStats();
         stats.set_type(ComponentType.SPOUT);
         stats.set_last_error((ErrorInfo) getByKeyword(m, LAST_ERROR));
@@ -1958,7 +1957,10 @@ public class StatsUtil {
         return t / c;
     }
 
-    public static String floatStr(double n) {
+    public static String floatStr(Double n) {
+        if (n == null) {
+            return "0";
+        }
         return String.format("%.3f", n);
     }
 
@@ -1970,6 +1972,10 @@ public class StatsUtil {
         return RT.keyword(null, key);
     }
 
+    private static ErrorInfo getLastError(IStormClusterState stormClusterState, String stormId, String compId) {
+        return stormClusterState.lastError(stormId, compId);
+    }
+
     interface KeyTransformer<T> {
         T transform(Object key);
     }


[19/30] storm git commit: upmerge from master

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/39ea23cd/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index 0ed2af9,0000000..351e830
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@@ -1,2062 -1,0 +1,2065 @@@
 +/**
 + * 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 clojure.lang.Keyword;
 +import clojure.lang.RT;
 +import com.google.common.collect.Lists;
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +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.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.TopologyPageInfo;
 +import org.apache.storm.generated.TopologyStats;
 +import org.apache.storm.utils.Utils;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +@SuppressWarnings("unchecked, unused")
 +public class StatsUtil {
 +    private static final Logger logger = LoggerFactory.getLogger(StatsUtil.class);
 +
 +    public static final String TYPE = "type";
 +    private static final String SPOUT = "spout";
 +    private static final String BOLT = "bolt";
 +    public static final Keyword KW_SPOUT = keyword(SPOUT);
 +    public static final Keyword KW_BOLT = keyword(BOLT);
 +
 +    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 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";
 +
 +    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();
 +    private static final ToStringTransformer TO_STRING = new ToStringTransformer();
 +    private static final FromGlobalStreamIdTransformer FROM_GSID = new FromGlobalStreamIdTransformer();
 +    public static final ToGlobalStreamIdTransformer TO_GSID = new ToGlobalStreamIdTransformer();
 +
 +
 +    // =====================================================================================
 +    // 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 aggBoltLatAndCount(Map id2execAvg, Map id2procAvg, Map id2numExec) {
 +        Map ret = new HashMap();
-         putRawKV(ret, EXEC_LAT_TOTAL, weightAvgAndSum(id2execAvg, id2numExec));
-         putRawKV(ret, PROC_LAT_TOTAL, weightAvgAndSum(id2procAvg, id2numExec));
-         putRawKV(ret, EXECUTED, sumValues(id2numExec));
++        putKV(ret, EXEC_LAT_TOTAL, weightAvgAndSum(id2execAvg, id2numExec));
++        putKV(ret, PROC_LAT_TOTAL, weightAvgAndSum(id2procAvg, id2numExec));
++        putKV(ret, EXECUTED, sumValues(id2numExec));
 +
 +        return ret;
 +    }
 +
 +    /**
 +     * Aggregates number acked and complete latencies across all streams.
 +     */
 +    public static Map aggSpoutLatAndCount(Map id2compAvg, Map id2numAcked) {
 +        Map ret = new HashMap();
-         putRawKV(ret, COMP_LAT_TOTAL, weightAvgAndSum(id2compAvg, id2numAcked));
-         putRawKV(ret, ACKED, sumValues(id2numAcked));
++        putKV(ret, COMP_LAT_TOTAL, weightAvgAndSum(id2compAvg, id2numAcked));
++        putKV(ret, ACKED, sumValues(id2numAcked));
 +
 +        return ret;
 +    }
 +
 +    /**
 +     * Aggregates number executed and process & execute latencies.
 +     */
 +    public static Map aggBoltStreamsLatAndCount(Map id2execAvg, Map id2procAvg, Map id2numExec) {
 +        Map ret = new HashMap();
 +        if (id2execAvg == null || id2procAvg == null || id2numExec == null) {
 +            return ret;
 +        }
 +        for (Object k : id2execAvg.keySet()) {
 +            Map subMap = new HashMap();
-             putRawKV(subMap, EXEC_LAT_TOTAL, weightAvg(id2execAvg, id2numExec, k));
-             putRawKV(subMap, PROC_LAT_TOTAL, weightAvg(id2procAvg, id2numExec, k));
-             putRawKV(subMap, EXECUTED, id2numExec.get(k));
++            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 Map aggSpoutStreamsLatAndCount(Map id2compAvg, Map id2acked) {
 +        Map ret = new HashMap();
 +        if (id2compAvg == null || id2acked == null) {
 +            return ret;
 +        }
 +        for (Object k : id2compAvg.keySet()) {
 +            Map subMap = new HashMap();
-             putRawKV(subMap, COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
-             putRawKV(subMap, ACKED, id2acked.get(k));
++            putKV(subMap, COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
++            putKV(subMap, ACKED, id2acked.get(k));
 +            ret.put(k, subMap);
 +        }
 +        return ret;
 +    }
 +
 +    public static Map aggPreMergeCompPageBolt(Map m, String window, boolean includeSys) {
 +        Map ret = new HashMap();
-         putRawKV(ret, EXECUTOR_ID, getByKeyword(m, "exec-id"));
-         putRawKV(ret, HOST, getByKeyword(m, HOST));
-         putRawKV(ret, PORT, getByKeyword(m, PORT));
-         putRawKV(ret, UPTIME, getByKeyword(m, UPTIME));
-         putRawKV(ret, NUM_EXECUTORS, 1);
-         putRawKV(ret, NUM_TASKS, getByKeyword(m, NUM_TASKS));
++        putKV(ret, EXECUTOR_ID, getByKey(m, "exec-id"));
++        putKV(ret, HOST, getByKey(m, HOST));
++        putKV(ret, PORT, getByKey(m, PORT));
++        putKV(ret, UPTIME, getByKey(m, UPTIME));
++        putKV(ret, NUM_EXECUTORS, 1);
++        putKV(ret, NUM_TASKS, getByKey(m, NUM_TASKS));
 +
-         Map stat2win2sid2num = getMapByKeyword(m, STATS);
-         putRawKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
++        Map stat2win2sid2num = getMapByKey(m, STATS);
++        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
 +
 +        // calc cid+sid->input_stats
 +        Map inputStats = new HashMap();
-         Map sid2acked = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, ACKED), TO_STRING).get(window);
-         Map sid2failed = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, FAILED), TO_STRING).get(window);
-         putRawKV(inputStats, ACKED, sid2acked != null ? sid2acked : new HashMap());
-         putRawKV(inputStats, FAILED, sid2failed != null ? sid2failed : 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(getMapByKeyword(stat2win2sid2num, EXEC_LATENCIES), TO_STRING).get(window);
-         Map sid2procLat = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, PROC_LATENCIES), TO_STRING).get(window);
-         Map sid2exec = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, EXECUTED), TO_STRING).get(window);
++        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));
-         putRawKV(ret, CID_SID_TO_IN_STATS, inputStats);
++        putKV(ret, CID_SID_TO_IN_STATS, inputStats);
 +
 +        // calc sid->output_stats
 +        Map outputStats = new HashMap();
-         Map sid2emitted = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, EMITTED), TO_STRING).get(window);
-         Map sid2transferred = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, TRANSFERRED), TO_STRING).get(window);
++        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) {
-             putRawKV(outputStats, EMITTED, filterSysStreams(sid2emitted, includeSys));
++            putKV(outputStats, EMITTED, filterSysStreams(sid2emitted, includeSys));
 +        } else {
-             putRawKV(outputStats, EMITTED, new HashMap());
++            putKV(outputStats, EMITTED, new HashMap());
 +        }
 +        if (sid2transferred != null) {
-             putRawKV(outputStats, TRANSFERRED, filterSysStreams(sid2transferred, includeSys));
++            putKV(outputStats, TRANSFERRED, filterSysStreams(sid2transferred, includeSys));
 +        } else {
-             putRawKV(outputStats, TRANSFERRED, new HashMap());
++            putKV(outputStats, TRANSFERRED, new HashMap());
 +        }
 +        outputStats = swapMapOrder(outputStats);
-         putRawKV(ret, SID_TO_OUT_STATS, outputStats);
++        putKV(ret, SID_TO_OUT_STATS, outputStats);
 +
 +        return ret;
 +    }
 +
 +    public static Map aggPreMergeCompPageSpout(Map m, String window, boolean includeSys) {
 +        Map ret = new HashMap();
-         putRawKV(ret, EXECUTOR_ID, getByKeyword(m, "exec-id"));
-         putRawKV(ret, HOST, getByKeyword(m, HOST));
-         putRawKV(ret, PORT, getByKeyword(m, PORT));
-         putRawKV(ret, UPTIME, getByKeyword(m, UPTIME));
-         putRawKV(ret, NUM_EXECUTORS, 1);
-         putRawKV(ret, NUM_TASKS, getByKeyword(m, NUM_TASKS));
++        putKV(ret, EXECUTOR_ID, getByKey(m, "exec-id"));
++        putKV(ret, HOST, getByKey(m, HOST));
++        putKV(ret, PORT, getByKey(m, PORT));
++        putKV(ret, UPTIME, getByKey(m, UPTIME));
++        putKV(ret, NUM_EXECUTORS, 1);
++        putKV(ret, NUM_TASKS, getByKey(m, NUM_TASKS));
 +
-         Map stat2win2sid2num = getMapByKeyword(m, STATS);
++        Map stat2win2sid2num = getMapByKey(m, STATS);
 +
 +        // calc sid->output-stats
 +        Map outputStats = new HashMap();
-         Map win2sid2acked = windowSetConverter(getMapByKeyword(stat2win2sid2num, ACKED), TO_STRING);
-         Map win2sid2failed = windowSetConverter(getMapByKeyword(stat2win2sid2num, FAILED), TO_STRING);
-         Map win2sid2emitted = windowSetConverter(getMapByKeyword(stat2win2sid2num, EMITTED), TO_STRING);
-         Map win2sid2transferred = windowSetConverter(getMapByKeyword(stat2win2sid2num, TRANSFERRED), TO_STRING);
-         Map win2sid2compLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
- 
-         putRawKV(outputStats, ACKED, win2sid2acked.get(window));
-         putRawKV(outputStats, FAILED, win2sid2failed.get(window));
-         putRawKV(outputStats, EMITTED, filterSysStreams((Map) win2sid2emitted.get(window), includeSys));
-         putRawKV(outputStats, TRANSFERRED, filterSysStreams((Map) win2sid2transferred.get(window), includeSys));
++        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));
++        putKV(outputStats, EMITTED, filterSysStreams((Map) win2sid2emitted.get(window), includeSys));
++        putKV(outputStats, TRANSFERRED, filterSysStreams((Map) win2sid2transferred.get(window), includeSys));
 +        outputStats = swapMapOrder(outputStats);
 +
 +        Map sid2compLat = (Map) win2sid2compLat.get(window);
 +        Map sid2acked = (Map) win2sid2acked.get(window);
 +        mergeMaps(outputStats, aggSpoutStreamsLatAndCount(sid2compLat, sid2acked));
-         putRawKV(ret, SID_TO_OUT_STATS, outputStats);
++        putKV(ret, SID_TO_OUT_STATS, outputStats);
 +
 +        return ret;
 +    }
 +
 +    public static Map aggPreMergeTopoPageBolt(Map m, String window, boolean includeSys) {
 +        Map ret = new HashMap();
 +
 +        Map subRet = new HashMap();
-         putRawKV(subRet, NUM_EXECUTORS, 1);
-         putRawKV(subRet, NUM_TASKS, getByKeyword(m, NUM_TASKS));
++        putKV(subRet, NUM_EXECUTORS, 1);
++        putKV(subRet, NUM_TASKS, getByKey(m, NUM_TASKS));
 +
-         Map stat2win2sid2num = getMapByKeyword(m, STATS);
-         putRawKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
++        Map stat2win2sid2num = getMapByKey(m, STATS);
++        putKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
 +
 +        for (String key : new String[]{EMITTED, TRANSFERRED, ACKED, FAILED}) {
-             Map stat = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, key), TO_STRING).get(window);
++            Map stat = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, key), TO_STRING).get(window);
 +            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
 +                stat = filterSysStreams(stat, includeSys);
 +            }
 +            long sum = 0;
 +            if (stat != null) {
 +                for (Object o : stat.values()) {
 +                    sum += ((Number) o).longValue();
 +                }
 +            }
-             putRawKV(subRet, key, sum);
++            putKV(subRet, key, sum);
 +        }
 +
-         Map win2sid2execLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, EXEC_LATENCIES), TO_STRING);
-         Map win2sid2procLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, PROC_LATENCIES), TO_STRING);
-         Map win2sid2exec = windowSetConverter(getMapByKeyword(stat2win2sid2num, EXECUTED), TO_STRING);
++        Map win2sid2execLat = windowSetConverter(getMapByKey(stat2win2sid2num, EXEC_LATENCIES), TO_STRING);
++        Map win2sid2procLat = windowSetConverter(getMapByKey(stat2win2sid2num, PROC_LATENCIES), TO_STRING);
++        Map win2sid2exec = windowSetConverter(getMapByKey(stat2win2sid2num, EXECUTED), TO_STRING);
 +        subRet.putAll(aggBoltLatAndCount(
 +                (Map) win2sid2execLat.get(window), (Map) win2sid2procLat.get(window), (Map) win2sid2exec.get(window)));
 +
-         ret.put(getByKeyword(m, "comp-id"), subRet);
++        ret.put(getByKey(m, "comp-id"), subRet);
 +        return ret;
 +    }
 +
 +    public static Map aggPreMergeTopoPageSpout(Map m, String window, boolean includeSys) {
 +        Map ret = new HashMap();
 +
 +        Map subRet = new HashMap();
-         putRawKV(subRet, NUM_EXECUTORS, 1);
-         putRawKV(subRet, NUM_TASKS, getByKeyword(m, NUM_TASKS));
++        putKV(subRet, NUM_EXECUTORS, 1);
++        putKV(subRet, NUM_TASKS, getByKey(m, NUM_TASKS));
 +
 +        // no capacity for spout
-         Map stat2win2sid2num = getMapByKeyword(m, STATS);
++        Map stat2win2sid2num = getMapByKey(m, STATS);
 +        for (String key : new String[]{EMITTED, TRANSFERRED, FAILED}) {
-             Map stat = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, key), TO_STRING).get(window);
++            Map stat = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, key), TO_STRING).get(window);
 +            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
 +                stat = filterSysStreams(stat, includeSys);
 +            }
 +            long sum = 0;
 +            if (stat != null) {
 +                for (Object o : stat.values()) {
 +                    sum += ((Number) o).longValue();
 +                }
 +            }
-             putRawKV(subRet, key, sum);
++            putKV(subRet, key, sum);
 +        }
 +
-         Map win2sid2compLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
-         Map win2sid2acked = windowSetConverter(getMapByKeyword(stat2win2sid2num, ACKED), TO_STRING);
++        Map win2sid2compLat = windowSetConverter(getMapByKey(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
++        Map win2sid2acked = windowSetConverter(getMapByKey(stat2win2sid2num, ACKED), TO_STRING);
 +        subRet.putAll(aggSpoutLatAndCount((Map) win2sid2compLat.get(window), (Map) win2sid2acked.get(window)));
 +
-         ret.put(getByKeyword(m, "comp-id"), subRet);
++        ret.put(getByKey(m, "comp-id"), subRet);
 +        return ret;
 +    }
 +
 +    public static Map mergeAggCompStatsCompPageBolt(Map accBoltStats, Map boltStats) {
 +        Map ret = new HashMap();
 +
-         Map accIn = getMapByKeyword(accBoltStats, CID_SID_TO_IN_STATS);
-         Map accOut = getMapByKeyword(accBoltStats, SID_TO_OUT_STATS);
-         Map boltIn = getMapByKeyword(boltStats, CID_SID_TO_IN_STATS);
-         Map boltOut = getMapByKeyword(boltStats, SID_TO_OUT_STATS);
++        Map accIn = getMapByKey(accBoltStats, CID_SID_TO_IN_STATS);
++        Map accOut = getMapByKey(accBoltStats, SID_TO_OUT_STATS);
++        Map boltIn = getMapByKey(boltStats, CID_SID_TO_IN_STATS);
++        Map boltOut = getMapByKey(boltStats, SID_TO_OUT_STATS);
 +
 +        int numExecutors = getByKeywordOr0(accBoltStats, NUM_EXECUTORS).intValue();
-         putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
-         putRawKV(ret, NUM_TASKS, sumOr0(
++        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
++        putKV(ret, NUM_TASKS, sumOr0(
 +                getByKeywordOr0(accBoltStats, NUM_TASKS), getByKeywordOr0(boltStats, NUM_TASKS)));
 +
 +        // (merge-with (partial merge-with sum-or-0) acc-out spout-out)
-         putRawKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, boltOut));
-         putRawKV(ret, CID_SID_TO_IN_STATS, fullMergeWithSum(accIn, boltIn));
++        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, boltOut));
++        putKV(ret, CID_SID_TO_IN_STATS, fullMergeWithSum(accIn, boltIn));
 +
 +        long executed = sumStreamsLong(boltIn, EXECUTED);
-         putRawKV(ret, EXECUTED, executed);
++        putKV(ret, EXECUTED, executed);
 +
 +        Map executorStats = new HashMap();
-         putRawKV(executorStats, EXECUTOR_ID, getByKeyword(boltStats, EXECUTOR_ID));
-         putRawKV(executorStats, UPTIME, getByKeyword(boltStats, UPTIME));
-         putRawKV(executorStats, HOST, getByKeyword(boltStats, HOST));
-         putRawKV(executorStats, PORT, getByKeyword(boltStats, PORT));
-         putRawKV(executorStats, CAPACITY, getByKeyword(boltStats, CAPACITY));
- 
-         putRawKV(executorStats, EMITTED, sumStreamsLong(boltOut, EMITTED));
-         putRawKV(executorStats, TRANSFERRED, sumStreamsLong(boltOut, TRANSFERRED));
-         putRawKV(executorStats, ACKED, sumStreamsLong(boltIn, ACKED));
-         putRawKV(executorStats, FAILED, sumStreamsLong(boltIn, FAILED));
-         putRawKV(executorStats, EXECUTED, executed);
++        putKV(executorStats, EXECUTOR_ID, getByKey(boltStats, EXECUTOR_ID));
++        putKV(executorStats, UPTIME, getByKey(boltStats, UPTIME));
++        putKV(executorStats, HOST, getByKey(boltStats, HOST));
++        putKV(executorStats, PORT, getByKey(boltStats, PORT));
++        putKV(executorStats, CAPACITY, getByKey(boltStats, 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) {
-             putRawKV(executorStats, EXEC_LATENCY, sumStreamsDouble(boltIn, EXEC_LAT_TOTAL) / executed);
-             putRawKV(executorStats, PROC_LATENCY, sumStreamsDouble(boltIn, PROC_LAT_TOTAL) / executed);
++            putKV(executorStats, EXEC_LATENCY, sumStreamsDouble(boltIn, EXEC_LAT_TOTAL) / executed);
++            putKV(executorStats, PROC_LATENCY, sumStreamsDouble(boltIn, PROC_LAT_TOTAL) / executed);
 +        } else {
-             putRawKV(executorStats, EXEC_LATENCY, null);
-             putRawKV(executorStats, PROC_LATENCY, null);
++            putKV(executorStats, EXEC_LATENCY, null);
++            putKV(executorStats, PROC_LATENCY, null);
 +        }
-         List executorStatsList = ((List) getByKeyword(accBoltStats, EXECUTOR_STATS));
++        List executorStatsList = ((List) getByKey(accBoltStats, EXECUTOR_STATS));
 +        executorStatsList.add(executorStats);
-         putRawKV(ret, EXECUTOR_STATS, executorStatsList);
++        putKV(ret, EXECUTOR_STATS, executorStatsList);
 +
 +        return ret;
 +    }
 +
 +    public static Map mergeAggCompStatsCompPageSpout(Map accSpoutStats, Map spoutStats) {
 +        Map ret = new HashMap();
 +
-         Map accOut = getMapByKeyword(accSpoutStats, SID_TO_OUT_STATS);
-         Map spoutOut = getMapByKeyword(spoutStats, SID_TO_OUT_STATS);
++        Map accOut = getMapByKey(accSpoutStats, SID_TO_OUT_STATS);
++        Map spoutOut = getMapByKey(spoutStats, SID_TO_OUT_STATS);
 +
 +        int numExecutors = getByKeywordOr0(accSpoutStats, NUM_EXECUTORS).intValue();
-         putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
-         putRawKV(ret, NUM_TASKS, sumOr0(
++        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
++        putKV(ret, NUM_TASKS, sumOr0(
 +                getByKeywordOr0(accSpoutStats, NUM_TASKS), getByKeywordOr0(spoutStats, NUM_TASKS)));
-         putRawKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
++        putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
 +
 +        Map executorStats = new HashMap();
-         putRawKV(executorStats, EXECUTOR_ID, getByKeyword(spoutStats, EXECUTOR_ID));
-         putRawKV(executorStats, UPTIME, getByKeyword(spoutStats, UPTIME));
-         putRawKV(executorStats, HOST, getByKeyword(spoutStats, HOST));
-         putRawKV(executorStats, PORT, getByKeyword(spoutStats, PORT));
- 
-         putRawKV(executorStats, EMITTED, sumStreamsLong(spoutOut, EMITTED));
-         putRawKV(executorStats, TRANSFERRED, sumStreamsLong(spoutOut, TRANSFERRED));
-         putRawKV(executorStats, FAILED, sumStreamsLong(spoutOut, FAILED));
++        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);
-         putRawKV(executorStats, ACKED, acked);
++        putKV(executorStats, ACKED, acked);
 +        if (acked > 0) {
-             putRawKV(executorStats, COMP_LATENCY, sumStreamsDouble(spoutOut, COMP_LAT_TOTAL) / acked);
++            putKV(executorStats, COMP_LATENCY, sumStreamsDouble(spoutOut, COMP_LAT_TOTAL) / acked);
 +        } else {
-             putRawKV(executorStats, COMP_LATENCY, null);
++            putKV(executorStats, COMP_LATENCY, null);
 +        }
-         List executorStatsList = ((List) getByKeyword(accSpoutStats, EXECUTOR_STATS));
++        List executorStatsList = ((List) getByKey(accSpoutStats, EXECUTOR_STATS));
 +        executorStatsList.add(executorStats);
-         putRawKV(ret, EXECUTOR_STATS, executorStatsList);
++        putKV(ret, EXECUTOR_STATS, executorStatsList);
 +
 +        return ret;
 +    }
 +
 +    public static Map mergeAggCompStatsTopoPageBolt(Map accBoltStats, Map boltStats) {
 +        Map ret = new HashMap();
 +        Integer numExecutors = getByKeywordOr0(accBoltStats, NUM_EXECUTORS).intValue();
-         putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
-         putRawKV(ret, NUM_TASKS, sumOr0(
++        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
++        putKV(ret, NUM_TASKS, sumOr0(
 +                getByKeywordOr0(accBoltStats, NUM_TASKS), getByKeywordOr0(boltStats, NUM_TASKS)));
-         putRawKV(ret, EMITTED, sumOr0(
++        putKV(ret, EMITTED, sumOr0(
 +                getByKeywordOr0(accBoltStats, EMITTED), getByKeywordOr0(boltStats, EMITTED)));
-         putRawKV(ret, TRANSFERRED, sumOr0(
++        putKV(ret, TRANSFERRED, sumOr0(
 +                getByKeywordOr0(accBoltStats, TRANSFERRED), getByKeywordOr0(boltStats, TRANSFERRED)));
-         putRawKV(ret, EXEC_LAT_TOTAL, sumOr0(
++        putKV(ret, EXEC_LAT_TOTAL, sumOr0(
 +                getByKeywordOr0(accBoltStats, EXEC_LAT_TOTAL), getByKeywordOr0(boltStats, EXEC_LAT_TOTAL)));
-         putRawKV(ret, PROC_LAT_TOTAL, sumOr0(
++        putKV(ret, PROC_LAT_TOTAL, sumOr0(
 +                getByKeywordOr0(accBoltStats, PROC_LAT_TOTAL), getByKeywordOr0(boltStats, PROC_LAT_TOTAL)));
-         putRawKV(ret, EXECUTED, sumOr0(
++        putKV(ret, EXECUTED, sumOr0(
 +                getByKeywordOr0(accBoltStats, EXECUTED), getByKeywordOr0(boltStats, EXECUTED)));
-         putRawKV(ret, ACKED, sumOr0(
++        putKV(ret, ACKED, sumOr0(
 +                getByKeywordOr0(accBoltStats, ACKED), getByKeywordOr0(boltStats, ACKED)));
-         putRawKV(ret, FAILED, sumOr0(
++        putKV(ret, FAILED, sumOr0(
 +                getByKeywordOr0(accBoltStats, FAILED), getByKeywordOr0(boltStats, FAILED)));
-         putRawKV(ret, CAPACITY, maxOr0(
++        putKV(ret, CAPACITY, maxOr0(
 +                getByKeywordOr0(accBoltStats, CAPACITY), getByKeywordOr0(boltStats, CAPACITY)));
 +
 +        return ret;
 +    }
 +
 +    public static Map mergeAggCompStatsTopoPageSpout(Map accSpoutStats, Map spoutStats) {
 +        Map ret = new HashMap();
 +        Integer numExecutors = getByKeywordOr0(accSpoutStats, NUM_EXECUTORS).intValue();
-         putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
-         putRawKV(ret, NUM_TASKS, sumOr0(
++        putKV(ret, NUM_EXECUTORS, numExecutors + 1);
++        putKV(ret, NUM_TASKS, sumOr0(
 +                getByKeywordOr0(accSpoutStats, NUM_TASKS), getByKeywordOr0(spoutStats, NUM_TASKS)));
-         putRawKV(ret, EMITTED, sumOr0(
++        putKV(ret, EMITTED, sumOr0(
 +                getByKeywordOr0(accSpoutStats, EMITTED), getByKeywordOr0(spoutStats, EMITTED)));
-         putRawKV(ret, TRANSFERRED, sumOr0(
++        putKV(ret, TRANSFERRED, sumOr0(
 +                getByKeywordOr0(accSpoutStats, TRANSFERRED), getByKeywordOr0(spoutStats, TRANSFERRED)));
-         putRawKV(ret, COMP_LAT_TOTAL, sumOr0(
++        putKV(ret, COMP_LAT_TOTAL, sumOr0(
 +                getByKeywordOr0(accSpoutStats, COMP_LAT_TOTAL), getByKeywordOr0(spoutStats, COMP_LAT_TOTAL)));
-         putRawKV(ret, ACKED, sumOr0(
++        putKV(ret, ACKED, sumOr0(
 +                getByKeywordOr0(accSpoutStats, ACKED), getByKeywordOr0(spoutStats, ACKED)));
-         putRawKV(ret, FAILED, sumOr0(
++        putKV(ret, FAILED, sumOr0(
 +                getByKeywordOr0(accSpoutStats, FAILED), getByKeywordOr0(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 aggTopoExecStats(String window, boolean includeSys, Map accStats, Map newData, String compType) {
 +        Map ret = new HashMap();
 +
-         Set workerSet = (Set) getByKeyword(accStats, WORKERS_SET);
-         Map bolt2stats = getMapByKeyword(accStats, BOLT_TO_STATS);
-         Map spout2stats = getMapByKeyword(accStats, SPOUT_TO_STATS);
-         Map win2emitted = getMapByKeyword(accStats, WIN_TO_EMITTED);
-         Map win2transferred = getMapByKeyword(accStats, WIN_TO_TRANSFERRED);
-         Map win2compLatWgtAvg = getMapByKeyword(accStats, WIN_TO_COMP_LAT_WGT_AVG);
-         Map win2acked = getMapByKeyword(accStats, WIN_TO_ACKED);
-         Map win2failed = getMapByKeyword(accStats, WIN_TO_FAILED);
-         Map stats = getMapByKeyword(newData, STATS);
++        Set workerSet = (Set) getByKey(accStats, 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);
++        Map stats = getMapByKey(newData, STATS);
 +
 +        boolean isSpout = compType.equals(SPOUT);
 +        Map cid2stat2num;
 +        if (isSpout) {
 +            cid2stat2num = aggPreMergeTopoPageSpout(newData, window, includeSys);
 +        } else {
 +            cid2stat2num = aggPreMergeTopoPageBolt(newData, window, includeSys);
 +        }
 +
 +        Map w2compLatWgtAvg, w2acked;
-         Map compLatStats = getMapByKeyword(stats, COMP_LATENCIES);
++        Map compLatStats = getMapByKey(stats, COMP_LATENCIES);
 +        if (isSpout) { // agg spout stats
 +            Map mm = new HashMap();
 +
-             Map acked = getMapByKeyword(stats, ACKED);
++            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 = getMapByKeyword(mm, COMP_LAT_TOTAL);
-             w2acked = getMapByKeyword(mm, ACKED);
++            w2compLatWgtAvg = getMapByKey(mm, COMP_LAT_TOTAL);
++            w2acked = getMapByKey(mm, ACKED);
 +        } else {
 +            w2compLatWgtAvg = null;
-             w2acked = aggregateCountStreams(getMapByKeyword(stats, ACKED));
++            w2acked = aggregateCountStreams(getMapByKey(stats, ACKED));
 +        }
 +
-         workerSet.add(Lists.newArrayList(getByKeyword(newData, HOST), getByKeyword(newData, PORT)));
-         putRawKV(ret, WORKERS_SET, workerSet);
-         putRawKV(ret, BOLT_TO_STATS, bolt2stats);
-         putRawKV(ret, SPOUT_TO_STATS, spout2stats);
-         putRawKV(ret, WIN_TO_EMITTED, mergeWithSum(win2emitted, aggregateCountStreams(
-                 filterSysStreams(getMapByKeyword(stats, EMITTED), includeSys))));
-         putRawKV(ret, WIN_TO_TRANSFERRED, mergeWithSum(win2transferred, aggregateCountStreams(
-                 filterSysStreams(getMapByKeyword(stats, TRANSFERRED), includeSys))));
-         putRawKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(win2compLatWgtAvg, w2compLatWgtAvg));
++        workerSet.add(Lists.newArrayList(getByKey(newData, HOST), getByKey(newData, PORT)));
++        putKV(ret, WORKERS_SET, workerSet);
++        putKV(ret, BOLT_TO_STATS, bolt2stats);
++        putKV(ret, SPOUT_TO_STATS, spout2stats);
++        putKV(ret, WIN_TO_EMITTED, mergeWithSum(win2emitted, aggregateCountStreams(
++                filterSysStreams(getMapByKey(stats, EMITTED), includeSys))));
++        putKV(ret, WIN_TO_TRANSFERRED, mergeWithSum(win2transferred, aggregateCountStreams(
++                filterSysStreams(getMapByKey(stats, TRANSFERRED), includeSys))));
++        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(win2compLatWgtAvg, w2compLatWgtAvg));
 +
-         //boolean isSpoutStat = SPOUT.equals(((Keyword) getByKeyword(stats, TYPE)).getName());
-         putRawKV(ret, WIN_TO_ACKED, isSpout ? mergeWithSum(win2acked, w2acked) : win2acked);
-         putRawKV(ret, WIN_TO_FAILED, isSpout ?
-                 mergeWithSum(aggregateCountStreams(getMapByKeyword(stats, FAILED)), win2failed) : win2failed);
-         putRawKV(ret, TYPE, getByKeyword(stats, TYPE));
++        //boolean isSpoutStat = SPOUT.equals(((Keyword) getByKey(stats, TYPE)).getName());
++        putKV(ret, WIN_TO_ACKED, isSpout ? mergeWithSum(win2acked, w2acked) : win2acked);
++        putKV(ret, WIN_TO_FAILED, isSpout ?
++                mergeWithSum(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) {
 +            Set<Object> keySet = new HashSet<>();
 +            keySet.addAll(spout2stats.keySet());
 +            keySet.addAll(cid2stat2num.keySet());
 +
 +            Map mm = new HashMap();
 +            for (Object k : keySet) {
 +                mm.put(k, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(k), (Map) cid2stat2num.get(k)));
 +            }
-             putRawKV(ret, SPOUT_TO_STATS, mm);
++            putKV(ret, SPOUT_TO_STATS, mm);
 +        } else {
 +            Set<Object> keySet = new HashSet<>();
 +            keySet.addAll(bolt2stats.keySet());
 +            keySet.addAll(cid2stat2num.keySet());
 +
 +            Map mm = new HashMap();
 +            for (Object k : keySet) {
 +                mm.put(k, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(k), (Map) cid2stat2num.get(k)));
 +            }
-             putRawKV(ret, BOLT_TO_STATS, mm);
++            putKV(ret, BOLT_TO_STATS, mm);
 +        }
 +
 +        return ret;
 +    }
 +
 +    public static TopologyPageInfo aggTopoExecsStats(
 +            String topologyId, Map exec2nodePort, Map task2component,
 +            Map beats, StormTopology topology, String window, boolean includeSys, IStormClusterState clusterState) {
 +        List beatList = extractDataFromHb(exec2nodePort, task2component, beats, includeSys, topology);
 +        Map topoStats = aggregateTopoStats(window, includeSys, beatList);
 +        topoStats = postAggregateTopoStats(task2component, exec2nodePort, topoStats, topologyId, clusterState);
 +
 +        return thriftifyTopoPageData(topologyId, topoStats);
 +    }
 +
 +    public static Map aggregateTopoStats(String win, boolean includeSys, List data) {
 +        Map initVal = new HashMap();
-         putRawKV(initVal, WORKERS_SET, new HashSet());
-         putRawKV(initVal, BOLT_TO_STATS, new HashMap());
-         putRawKV(initVal, SPOUT_TO_STATS, new HashMap());
-         putRawKV(initVal, WIN_TO_EMITTED, new HashMap());
-         putRawKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
-         putRawKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
-         putRawKV(initVal, WIN_TO_ACKED, new HashMap());
-         putRawKV(initVal, WIN_TO_FAILED, 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 (Object o : data) {
 +            Map newData = (Map) o;
-             String compType = ((Keyword) getByKeyword(newData, TYPE)).getName();
++            String compType = ((Keyword) getByKey(newData, TYPE)).getName();
 +            initVal = aggTopoExecStats(win, includeSys, initVal, newData, compType);
 +        }
 +
 +        return initVal;
 +    }
 +
 +    public static Map postAggregateTopoStats(
 +            Map task2comp, Map exec2nodePort, Map accData, String topologyId, IStormClusterState clusterState) {
 +        Map ret = new HashMap();
-         putRawKV(ret, NUM_TASKS, task2comp.size());
-         putRawKV(ret, NUM_WORKERS, ((Set) getByKeyword(accData, WORKERS_SET)).size());
-         putRawKV(ret, NUM_EXECUTORS, exec2nodePort != null ? exec2nodePort.size() : 0);
++        putKV(ret, NUM_TASKS, task2comp.size());
++        putKV(ret, NUM_WORKERS, ((Set) getByKey(accData, WORKERS_SET)).size());
++        putKV(ret, NUM_EXECUTORS, exec2nodePort != null ? exec2nodePort.size() : 0);
 +
-         Map bolt2stats = getMapByKeyword(accData, BOLT_TO_STATS);
++        Map bolt2stats = getMapByKey(accData, BOLT_TO_STATS);
 +        Map 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 = getByKeywordOr0(m, EXECUTED).longValue();
 +            if (executed > 0) {
 +                double execLatencyTotal = getByKeywordOr0(m, EXEC_LAT_TOTAL).doubleValue();
-                 putRawKV(m, EXEC_LATENCY, execLatencyTotal / executed);
++                putKV(m, EXEC_LATENCY, execLatencyTotal / executed);
 +
 +                double procLatencyTotal = getByKeywordOr0(m, PROC_LAT_TOTAL).doubleValue();
-                 putRawKV(m, PROC_LATENCY, procLatencyTotal / executed);
++                putKV(m, PROC_LATENCY, procLatencyTotal / executed);
 +            }
-             removeByKeyword(m, EXEC_LAT_TOTAL);
-             removeByKeyword(m, PROC_LAT_TOTAL);
-             putRawKV(m, "last-error", getLastError(clusterState, topologyId, id));
++            remove(m, EXEC_LAT_TOTAL);
++            remove(m, PROC_LAT_TOTAL);
++            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
 +
 +            aggBolt2stats.put(id, m);
 +        }
-         putRawKV(ret, BOLT_TO_STATS, aggBolt2stats);
++        putKV(ret, BOLT_TO_STATS, aggBolt2stats);
 +
-         Map spout2stats = getMapByKeyword(accData, SPOUT_TO_STATS);
++        Map spout2stats = getMapByKey(accData, SPOUT_TO_STATS);
 +        Map spoutBolt2stats = 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 = getByKeywordOr0(m, ACKED).longValue();
 +            if (acked > 0) {
 +                double compLatencyTotal = getByKeywordOr0(m, COMP_LAT_TOTAL).doubleValue();
-                 putRawKV(m, COMP_LATENCY, compLatencyTotal / acked);
++                putKV(m, COMP_LATENCY, compLatencyTotal / acked);
 +            }
-             removeByKeyword(m, COMP_LAT_TOTAL);
-             putRawKV(m, "last-error", getLastError(clusterState, topologyId, id));
++            remove(m, COMP_LAT_TOTAL);
++            putKV(m, "last-error", getLastError(clusterState, topologyId, id));
 +
 +            spoutBolt2stats.put(id, m);
 +        }
-         putRawKV(ret, SPOUT_TO_STATS, spoutBolt2stats);
++        putKV(ret, SPOUT_TO_STATS, spoutBolt2stats);
 +
-         putRawKV(ret, WIN_TO_EMITTED, mapKeyStr(getMapByKeyword(accData, WIN_TO_EMITTED)));
-         putRawKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKeyword(accData, WIN_TO_TRANSFERRED)));
-         putRawKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKeyword(accData, WIN_TO_ACKED)));
-         putRawKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKeyword(accData, WIN_TO_FAILED)));
-         putRawKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
++        putKV(ret, WIN_TO_EMITTED, mapKeyStr(getMapByKey(accData, WIN_TO_EMITTED)));
++        putKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKey(accData, WIN_TO_TRANSFERRED)));
++        putKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKey(accData, WIN_TO_ACKED)));
++        putKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKey(accData, WIN_TO_FAILED)));
++        putKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
 +                accData, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
 +        return ret;
 +    }
 +
 +    /**
 +     * aggregate bolt stats
 +     *
 +     * @param statsSeq   a seq of ExecutorStats
 +     * @param includeSys whether to include system streams
 +     * @return aggregated bolt stats
 +     */
 +    public static Map aggregateBoltStats(List statsSeq, boolean includeSys) {
 +        Map ret = new HashMap();
 +
 +        Map commonStats = preProcessStreamSummary(aggregateCommonStats(statsSeq), includeSys);
 +        List acked = new ArrayList();
 +        List failed = new ArrayList();
 +        List executed = new ArrayList();
 +        List processLatencies = new ArrayList();
 +        List executeLatencies = new ArrayList();
 +        for (Object o : statsSeq) {
 +            ExecutorStats stat = (ExecutorStats) o;
 +            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);
-         putRawKV(ret, ACKED, aggregateCounts(acked));
-         putRawKV(ret, FAILED, aggregateCounts(failed));
-         putRawKV(ret, EXECUTED, aggregateCounts(executed));
-         putRawKV(ret, PROC_LATENCIES, aggregateAverages(processLatencies, acked));
-         putRawKV(ret, EXEC_LATENCIES, aggregateAverages(executeLatencies, executed));
++        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
 +     */
 +    public static Map aggregateSpoutStats(List statsSeq, boolean includeSys) {
 +        Map ret = new HashMap();
 +
 +        Map commonStats = preProcessStreamSummary(aggregateCommonStats(statsSeq), includeSys);
 +        List acked = new ArrayList();
 +        List failed = new ArrayList();
 +        List completeLatencies = new ArrayList();
 +        for (Object o : statsSeq) {
 +            ExecutorStats stat = (ExecutorStats) o;
 +            acked.add(stat.get_specific().get_spout().get_acked());
 +            failed.add(stat.get_specific().get_spout().get_failed());
 +            completeLatencies.add(stat.get_specific().get_spout().get_complete_ms_avg());
 +        }
 +        mergeMaps(ret, commonStats);
-         putRawKV(ret, ACKED, aggregateCounts(acked));
-         putRawKV(ret, FAILED, aggregateCounts(failed));
-         putRawKV(ret, COMP_LATENCIES, aggregateAverages(completeLatencies, acked));
++        putKV(ret, ACKED, aggregateCounts(acked));
++        putKV(ret, FAILED, aggregateCounts(failed));
++        putKV(ret, COMP_LATENCIES, aggregateAverages(completeLatencies, acked));
 +
 +        return ret;
 +    }
 +
 +    public static Map aggregateCommonStats(List statsSeq) {
 +        Map ret = new HashMap();
 +
 +        List emitted = new ArrayList();
 +        List transferred = new ArrayList();
 +        for (Object o : statsSeq) {
 +            ExecutorStats stat = (ExecutorStats) o;
 +            emitted.add(stat.get_emitted());
 +            transferred.add(stat.get_transferred());
 +        }
 +
-         putRawKV(ret, EMITTED, aggregateCounts(emitted));
-         putRawKV(ret, TRANSFERRED, aggregateCounts(transferred));
++        putKV(ret, EMITTED, aggregateCounts(emitted));
++        putKV(ret, TRANSFERRED, aggregateCounts(transferred));
 +        return ret;
 +    }
 +
 +    public static Map preProcessStreamSummary(Map streamSummary, boolean includeSys) {
-         Map emitted = getMapByKeyword(streamSummary, EMITTED);
-         Map transferred = getMapByKeyword(streamSummary, TRANSFERRED);
++        Map emitted = getMapByKey(streamSummary, EMITTED);
++        Map transferred = getMapByKey(streamSummary, TRANSFERRED);
 +
-         putRawKV(streamSummary, EMITTED, filterSysStreams(emitted, includeSys));
-         putRawKV(streamSummary, TRANSFERRED, filterSysStreams(transferred, includeSys));
++        putKV(streamSummary, EMITTED, filterSysStreams(emitted, includeSys));
++        putKV(streamSummary, TRANSFERRED, filterSysStreams(transferred, includeSys));
 +
 +        return streamSummary;
 +    }
 +
 +    public static Map aggregateCountStreams(Map stats) {
 +        Map ret = new HashMap();
 +        for (Object o : stats.entrySet()) {
 +            Map.Entry entry = (Map.Entry) o;
 +            Map value = (Map) entry.getValue();
 +            long sum = 0l;
 +            for (Object num : value.values()) {
 +                sum += ((Number) num).longValue();
 +            }
 +            ret.put(entry.getKey(), sum);
 +        }
 +        return ret;
 +    }
 +
 +    public static Map aggregateAverages(List avgSeq, List countSeq) {
 +        Map ret = new HashMap();
 +
 +        Map expands = expandAveragesSeq(avgSeq, countSeq);
 +        for (Object o : expands.entrySet()) {
 +            Map.Entry entry = (Map.Entry) o;
 +            Object k = entry.getKey();
 +
 +            Map tmp = new HashMap();
 +            Map inner = (Map) entry.getValue();
 +            for (Object kk : inner.keySet()) {
 +                List vv = (List) inner.get(kk);
 +                tmp.put(kk, valAvg(((Number) vv.get(0)).doubleValue(), ((Number) vv.get(1)).longValue()));
 +            }
 +            ret.put(k, tmp);
 +        }
 +
 +        return ret;
 +    }
 +
 +    public static Map aggregateAvgStreams(Map avgs, Map counts) {
 +        Map ret = new HashMap();
 +
 +        Map expands = expandAverages(avgs, counts);
 +        for (Object o : expands.entrySet()) {
 +            Map.Entry e = (Map.Entry) o;
 +            Object win = e.getKey();
 +
 +            double avgTotal = 0.0;
 +            long cntTotal = 0l;
 +            Map inner = (Map) e.getValue();
 +            for (Object kk : inner.keySet()) {
 +                List vv = (List) inner.get(kk);
 +                avgTotal += ((Number) vv.get(0)).doubleValue();
 +                cntTotal += ((Number) vv.get(1)).longValue();
 +            }
 +            ret.put(win, valAvg(avgTotal, cntTotal));
 +        }
 +
 +        return ret;
 +    }
 +
 +    public static Map spoutStreamsStats(List summs, boolean includeSys) {
 +        List statsSeq = getFilledStats(summs);
 +        return aggregateSpoutStreams(aggregateSpoutStats(statsSeq, includeSys));
 +    }
 +
 +    public static Map boltStreamsStats(List summs, boolean includeSys) {
 +        List statsSeq = getFilledStats(summs);
 +        return aggregateBoltStreams(aggregateBoltStats(statsSeq, includeSys));
 +    }
 +
 +    public static Map aggregateSpoutStreams(Map stats) {
 +        Map ret = new HashMap();
-         putRawKV(ret, ACKED, aggregateCountStreams(getMapByKeyword(stats, ACKED)));
-         putRawKV(ret, FAILED, aggregateCountStreams(getMapByKeyword(stats, FAILED)));
-         putRawKV(ret, EMITTED, aggregateCountStreams(getMapByKeyword(stats, EMITTED)));
-         putRawKV(ret, TRANSFERRED, aggregateCountStreams(getMapByKeyword(stats, TRANSFERRED)));
-         putRawKV(ret, COMP_LATENCIES, aggregateAvgStreams(
-                 getMapByKeyword(stats, COMP_LATENCIES), getMapByKeyword(stats, ACKED)));
++        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;
 +    }
 +
 +    public static Map aggregateBoltStreams(Map stats) {
 +        Map ret = new HashMap();
-         putRawKV(ret, ACKED, aggregateCountStreams(getMapByKeyword(stats, ACKED)));
-         putRawKV(ret, FAILED, aggregateCountStreams(getMapByKeyword(stats, FAILED)));
-         putRawKV(ret, EMITTED, aggregateCountStreams(getMapByKeyword(stats, EMITTED)));
-         putRawKV(ret, TRANSFERRED, aggregateCountStreams(getMapByKeyword(stats, TRANSFERRED)));
-         putRawKV(ret, EXECUTED, aggregateCountStreams(getMapByKeyword(stats, EXECUTED)));
-         putRawKV(ret, PROC_LATENCIES, aggregateAvgStreams(
-                 getMapByKeyword(stats, PROC_LATENCIES), getMapByKeyword(stats, ACKED)));
-         putRawKV(ret, EXEC_LATENCIES, aggregateAvgStreams(
-                 getMapByKeyword(stats, EXEC_LATENCIES), getMapByKeyword(stats, EXECUTED)));
++        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;
 +    }
 +
 +    /**
 +     * A helper function that aggregates windowed stats from one spout executor.
 +     */
 +    public static Map aggBoltExecWinStats(Map accStats, Map newStats, boolean includeSys) {
 +        Map ret = new HashMap();
 +
 +        Map m = new HashMap();
-         for (Object win : getMapByKeyword(newStats, EXECUTED).keySet()) {
++        for (Object win : getMapByKey(newStats, EXECUTED).keySet()) {
 +            m.put(win, aggBoltLatAndCount(
-                     (Map) (getMapByKeyword(newStats, EXEC_LATENCIES)).get(win),
-                     (Map) (getMapByKeyword(newStats, PROC_LATENCIES)).get(win),
-                     (Map) (getMapByKeyword(newStats, EXECUTED)).get(win)));
++                    (Map) (getMapByKey(newStats, EXEC_LATENCIES)).get(win),
++                    (Map) (getMapByKey(newStats, PROC_LATENCIES)).get(win),
++                    (Map) (getMapByKey(newStats, EXECUTED)).get(win)));
 +        }
 +        m = swapMapOrder(m);
 +
-         Map win2execLatWgtAvg = getMapByKeyword(m, EXEC_LAT_TOTAL);
-         Map win2procLatWgtAvg = getMapByKeyword(m, PROC_LAT_TOTAL);
-         Map win2executed = getMapByKeyword(m, EXECUTED);
++        Map win2execLatWgtAvg = getMapByKey(m, EXEC_LAT_TOTAL);
++        Map win2procLatWgtAvg = getMapByKey(m, PROC_LAT_TOTAL);
++        Map win2executed = getMapByKey(m, EXECUTED);
 +
-         Map emitted = getMapByKeyword(newStats, EMITTED);
++        Map emitted = getMapByKey(newStats, EMITTED);
 +        emitted = mergeWithSum(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
-                 getMapByKeyword(accStats, WIN_TO_EMITTED));
-         putRawKV(ret, WIN_TO_EMITTED, emitted);
++                getMapByKey(accStats, WIN_TO_EMITTED));
++        putKV(ret, WIN_TO_EMITTED, emitted);
 +
-         Map transferred = getMapByKeyword(newStats, TRANSFERRED);
++        Map transferred = getMapByKey(newStats, TRANSFERRED);
 +        transferred = mergeWithSum(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
-                 getMapByKeyword(accStats, WIN_TO_TRANSFERRED));
-         putRawKV(ret, WIN_TO_TRANSFERRED, transferred);
- 
-         putRawKV(ret, WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSum(
-                 getMapByKeyword(accStats, WIN_TO_EXEC_LAT_WGT_AVG), win2execLatWgtAvg));
-         putRawKV(ret, WIN_TO_PROC_LAT_WGT_AVG, mergeWithSum(
-                 getMapByKeyword(accStats, WIN_TO_PROC_LAT_WGT_AVG), win2procLatWgtAvg));
-         putRawKV(ret, WIN_TO_EXECUTED, mergeWithSum(
-                 getMapByKeyword(accStats, WIN_TO_EXECUTED), win2executed));
-         putRawKV(ret, WIN_TO_ACKED, mergeWithSum(
-                 aggregateCountStreams(getMapByKeyword(newStats, ACKED)), getMapByKeyword(accStats, WIN_TO_ACKED)));
-         putRawKV(ret, WIN_TO_FAILED, mergeWithSum(
-                 aggregateCountStreams(getMapByKeyword(newStats, FAILED)), getMapByKeyword(accStats, WIN_TO_FAILED)));
++                getMapByKey(accStats, WIN_TO_TRANSFERRED));
++        putKV(ret, WIN_TO_TRANSFERRED, transferred);
++
++        putKV(ret, WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSum(
++                getMapByKey(accStats, WIN_TO_EXEC_LAT_WGT_AVG), win2execLatWgtAvg));
++        putKV(ret, WIN_TO_PROC_LAT_WGT_AVG, mergeWithSum(
++                getMapByKey(accStats, WIN_TO_PROC_LAT_WGT_AVG), win2procLatWgtAvg));
++        putKV(ret, WIN_TO_EXECUTED, mergeWithSum(
++                getMapByKey(accStats, WIN_TO_EXECUTED), win2executed));
++        putKV(ret, WIN_TO_ACKED, mergeWithSum(
++                aggregateCountStreams(getMapByKey(newStats, ACKED)), getMapByKey(accStats, WIN_TO_ACKED)));
++        putKV(ret, WIN_TO_FAILED, mergeWithSum(
++                aggregateCountStreams(getMapByKey(newStats, FAILED)), getMapByKey(accStats, WIN_TO_FAILED)));
 +
 +        return ret;
 +    }
 +
 +    /**
 +     * A helper function that aggregates windowed stats from one spout executor.
 +     */
 +    public static Map aggSpoutExecWinStats(Map accStats, Map newStats, boolean includeSys) {
 +        Map ret = new HashMap();
 +
 +        Map m = new HashMap();
-         for (Object win : getMapByKeyword(newStats, ACKED).keySet()) {
++        for (Object win : getMapByKey(newStats, ACKED).keySet()) {
 +            m.put(win, aggSpoutLatAndCount(
-                     (Map) (getMapByKeyword(newStats, COMP_LATENCIES)).get(win),
-                     (Map) (getMapByKeyword(newStats, ACKED)).get(win)));
++                    (Map) (getMapByKey(newStats, COMP_LATENCIES)).get(win),
++                    (Map) (getMapByKey(newStats, ACKED)).get(win)));
 +        }
 +        m = swapMapOrder(m);
 +
-         Map win2compLatWgtAvg = getMapByKeyword(m, COMP_LAT_TOTAL);
-         Map win2acked = getMapByKeyword(m, ACKED);
++        Map win2compLatWgtAvg = getMapByKey(m, COMP_LAT_TOTAL);
++        Map win2acked = getMapByKey(m, ACKED);
 +
-         Map emitted = getMapByKeyword(newStats, EMITTED);
++        Map emitted = getMapByKey(newStats, EMITTED);
 +        emitted = mergeWithSum(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
-                 getMapByKeyword(accStats, WIN_TO_EMITTED));
-         putRawKV(ret, WIN_TO_EMITTED, emitted);
++                getMapByKey(accStats, WIN_TO_EMITTED));
++        putKV(ret, WIN_TO_EMITTED, emitted);
 +
-         Map transferred = getMapByKeyword(newStats, TRANSFERRED);
++        Map transferred = getMapByKey(newStats, TRANSFERRED);
 +        transferred = mergeWithSum(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
-                 getMapByKeyword(accStats, WIN_TO_TRANSFERRED));
-         putRawKV(ret, WIN_TO_TRANSFERRED, transferred);
++                getMapByKey(accStats, WIN_TO_TRANSFERRED));
++        putKV(ret, WIN_TO_TRANSFERRED, transferred);
 +
-         putRawKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(
-                 getMapByKeyword(accStats, WIN_TO_COMP_LAT_WGT_AVG), win2compLatWgtAvg));
-         putRawKV(ret, WIN_TO_ACKED, mergeWithSum(
-                 getMapByKeyword(accStats, WIN_TO_ACKED), win2acked));
-         putRawKV(ret, WIN_TO_FAILED, mergeWithSum(
-                 aggregateCountStreams(getMapByKeyword(newStats, FAILED)), getMapByKeyword(accStats, WIN_TO_FAILED)));
++        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(
++                getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG), win2compLatWgtAvg));
++        putKV(ret, WIN_TO_ACKED, mergeWithSum(
++                getMapByKey(accStats, WIN_TO_ACKED), win2acked));
++        putKV(ret, WIN_TO_FAILED, mergeWithSum(
++                aggregateCountStreams(getMapByKey(newStats, FAILED)), getMapByKey(accStats, WIN_TO_FAILED)));
 +
 +        return ret;
 +    }
 +
 +
 +    /**
 +     * aggregate counts
 +     *
 +     * @param countsSeq a seq of {win -> GlobalStreamId -> value}
 +     */
 +    public static Map aggregateCounts(List countsSeq) {
 +        Map ret = new HashMap();
 +        for (Object counts : countsSeq) {
 +            for (Object o : ((Map) counts).entrySet()) {
 +                Map.Entry e = (Map.Entry) o;
 +                Object win = e.getKey();
 +                Map stream2count = (Map) e.getValue();
 +
 +                if (!ret.containsKey(win)) {
 +                    ret.put(win, stream2count);
 +                } else {
 +                    Map existing = (Map) ret.get(win);
 +                    for (Object oo : stream2count.entrySet()) {
 +                        Map.Entry ee = (Map.Entry) oo;
 +                        Object stream = ee.getKey();
 +                        if (!existing.containsKey(stream)) {
 +                            existing.put(stream, ee.getValue());
 +                        } else {
 +                            existing.put(stream, (Long) ee.getValue() + (Long) existing.get(stream));
 +                        }
 +                    }
 +                }
 +            }
 +        }
 +        return ret;
 +    }
 +
 +    public static Map aggregateCompStats(String window, boolean includeSys, List data, String compType) {
 +        boolean isSpout = SPOUT.equals(compType);
 +
 +        Map initVal = new HashMap();
-         putRawKV(initVal, WIN_TO_ACKED, new HashMap());
-         putRawKV(initVal, WIN_TO_FAILED, new HashMap());
-         putRawKV(initVal, WIN_TO_EMITTED, new HashMap());
-         putRawKV(initVal, WIN_TO_TRANSFERRED, 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 stats = new HashMap();
-         putRawKV(stats, EXECUTOR_STATS, new ArrayList());
-         putRawKV(stats, SID_TO_OUT_STATS, new HashMap());
++        putKV(stats, EXECUTOR_STATS, new ArrayList());
++        putKV(stats, SID_TO_OUT_STATS, new HashMap());
 +        if (isSpout) {
-             putRawKV(initVal, TYPE, KW_SPOUT);
-             putRawKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
++            putKV(initVal, TYPE, KW_SPOUT);
++            putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
 +        } else {
-             putRawKV(initVal, TYPE, KW_BOLT);
-             putRawKV(initVal, WIN_TO_EXECUTED, new HashMap());
-             putRawKV(stats, CID_SID_TO_IN_STATS, new HashMap());
-             putRawKV(initVal, WIN_TO_EXEC_LAT_WGT_AVG, new HashMap());
-             putRawKV(initVal, WIN_TO_PROC_LAT_WGT_AVG, new HashMap());
++            putKV(initVal, TYPE, KW_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());
 +        }
-         putRawKV(initVal, STATS, stats);
++        putKV(initVal, STATS, stats);
 +
 +        for (Object o : data) {
 +            initVal = aggCompExecStats(window, includeSys, initVal, (Map) o, 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 aggCompExecStats(String window, boolean includeSys, Map accStats, Map newData, String compType) {
 +        Map ret = new HashMap();
 +        if (SPOUT.equals(compType)) {
-             ret.putAll(aggSpoutExecWinStats(accStats, getMapByKeyword(newData, STATS), includeSys));
-             putRawKV(ret, STATS, mergeAggCompStatsCompPageSpout(
-                     getMapByKeyword(accStats, STATS),
++            ret.putAll(aggSpoutExecWinStats(accStats, getMapByKey(newData, STATS), includeSys));
++            putKV(ret, STATS, mergeAggCompStatsCompPageSpout(
++                    getMapByKey(accStats, STATS),
 +                    aggPreMergeCompPageSpout(newData, window, includeSys)));
 +        } else {
-             ret.putAll(aggBoltExecWinStats(accStats, getMapByKeyword(newData, STATS), includeSys));
-             putRawKV(ret, STATS, mergeAggCompStatsCompPageBolt(
-                     getMapByKeyword(accStats, STATS),
++            ret.putAll(aggBoltExecWinStats(accStats, getMapByKey(newData, STATS), includeSys));
++            putKV(ret, STATS, mergeAggCompStatsCompPageBolt(
++                    getMapByKey(accStats, STATS),
 +                    aggPreMergeCompPageBolt(newData, window, includeSys)));
 +        }
-         putRawKV(ret, TYPE, keyword(compType));
++        putKV(ret, TYPE, keyword(compType));
 +
 +        return ret;
 +    }
 +
 +    public static Map postAggregateCompStats(Map task2component, Map exec2hostPort, Map accData) {
 +        Map ret = new HashMap();
 +
-         String compType = ((Keyword) getByKeyword(accData, TYPE)).getName();
-         Map stats = getMapByKeyword(accData, STATS);
++        String compType = ((Keyword) getByKey(accData, TYPE)).getName();
++        Map stats = getMapByKey(accData, STATS);
 +        Integer numTasks = getByKeywordOr0(stats, NUM_TASKS).intValue();
 +        Integer numExecutors = getByKeywordOr0(stats, NUM_EXECUTORS).intValue();
-         Map outStats = getMapByKeyword(stats, SID_TO_OUT_STATS);
++        Map outStats = getMapByKey(stats, SID_TO_OUT_STATS);
 +
-         putRawKV(ret, TYPE, keyword(compType));
-         putRawKV(ret, NUM_TASKS, numTasks);
-         putRawKV(ret, NUM_EXECUTORS, numExecutors);
-         putRawKV(ret, EXECUTOR_STATS, getByKeyword(stats, EXECUTOR_STATS));
-         putRawKV(ret, WIN_TO_EMITTED, mapKeyStr(getMapByKeyword(accData, WIN_TO_EMITTED)));
-         putRawKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKeyword(accData, WIN_TO_TRANSFERRED)));
-         putRawKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKeyword(accData, WIN_TO_ACKED)));
-         putRawKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKeyword(accData, WIN_TO_FAILED)));
++        putKV(ret, TYPE, keyword(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(accData, WIN_TO_EMITTED)));
++        putKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKey(accData, WIN_TO_TRANSFERRED)));
++        putKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKey(accData, WIN_TO_ACKED)));
++        putKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKey(accData, WIN_TO_FAILED)));
 +
 +        if (BOLT.equals(compType)) {
-             Map inStats = getMapByKeyword(stats, CID_SID_TO_IN_STATS);
++            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 = getByKeywordOr0(v, EXECUTED).longValue();
 +                if (executed > 0) {
 +                    double executeLatencyTotal = getByKeywordOr0(v, EXEC_LAT_TOTAL).doubleValue();
 +                    double processLatencyTotal = getByKeywordOr0(v, PROC_LAT_TOTAL).doubleValue();
-                     putRawKV(v, EXEC_LATENCY, executeLatencyTotal / executed);
-                     putRawKV(v, PROC_LATENCY, processLatencyTotal / executed);
++                    putKV(v, EXEC_LATENCY, executeLatencyTotal / executed);
++                    putKV(v, PROC_LATENCY, processLatencyTotal / executed);
 +                } else {
-                     putRawKV(v, EXEC_LATENCY, 0.0);
-                     putRawKV(v, PROC_LATENCY, 0.0);
++                    putKV(v, EXEC_LATENCY, 0.0);
++                    putKV(v, PROC_LATENCY, 0.0);
 +                }
-                 removeByKeyword(v, EXEC_LAT_TOTAL);
-                 removeByKeyword(v, PROC_LAT_TOTAL);
++                remove(v, EXEC_LAT_TOTAL);
++                remove(v, PROC_LAT_TOTAL);
 +                inStats2.put(k, v);
 +            }
-             putRawKV(ret, CID_SID_TO_IN_STATS, inStats2);
++            putKV(ret, CID_SID_TO_IN_STATS, inStats2);
 +
-             putRawKV(ret, SID_TO_OUT_STATS, outStats);
-             putRawKV(ret, WIN_TO_EXECUTED, mapKeyStr(getMapByKeyword(accData, WIN_TO_EXECUTED)));
-             putRawKV(ret, WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
++            putKV(ret, SID_TO_OUT_STATS, outStats);
++            putKV(ret, WIN_TO_EXECUTED, mapKeyStr(getMapByKey(accData, WIN_TO_EXECUTED)));
++            putKV(ret, WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
 +                    accData, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
-             putRawKV(ret, WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
++            putKV(ret, WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
 +                    accData, 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 = getByKeywordOr0(v, ACKED).longValue();
 +                if (acked > 0) {
 +                    double compLatencyTotal = getByKeywordOr0(v, COMP_LAT_TOTAL).doubleValue();
-                     putRawKV(v, COMP_LATENCY, compLatencyTotal / acked);
++                    putKV(v, COMP_LATENCY, compLatencyTotal / acked);
 +                } else {
-                     putRawKV(v, COMP_LATENCY, 0.0);
++                    putKV(v, COMP_LATENCY, 0.0);
 +                }
-                 removeByKeyword(v, COMP_LAT_TOTAL);
++                remove(v, COMP_LAT_TOTAL);
 +                outStats2.put(k, v);
 +            }
-             putRawKV(ret, SID_TO_OUT_STATS, outStats2);
-             putRawKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
++            putKV(ret, SID_TO_OUT_STATS, outStats2);
++            putKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
 +                    accData, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
 +        }
 +
 +        return ret;
 +    }
 +
 +    public static ComponentPageInfo aggCompExecsStats(
 +            Map exec2hostPort, Map task2component, Map beats, String window, boolean includeSys,
 +            String topologyId, StormTopology topology, String componentId) {
 +
 +        List beatList = extractDataFromHb(exec2hostPort, task2component, beats, includeSys, topology, componentId);
 +        Map compStats = aggregateCompStats(window, includeSys, beatList, componentType(topology, componentId).getName());
 +        compStats = postAggregateCompStats(task2component, exec2hostPort, compStats);
 +        return thriftifyCompPageData(topologyId, topology, componentId, compStats);
 +    }
 +
 +
 +    // =====================================================================================
 +    // clojurify stats methods
 +    // =====================================================================================
 +
 +    public static Map clojurifyStats(Map stats) {
 +        Map ret = new HashMap();
 +        for (Object o : stats.entrySet()) {
 +            Map.Entry entry = (Map.Entry) o;
 +            ExecutorInfo executorInfo = (ExecutorInfo) entry.getKey();
 +            ExecutorStats executorStats = (ExecutorStats) entry.getValue();
 +
 +            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
 +                    clojurifyExecutorStats(executorStats));
 +        }
 +        return ret;
 +    }
 +
 +    public static Map clojurifyExecutorStats(ExecutorStats stats) {
 +        Map ret = new HashMap();
 +
-         putRawKV(ret, EMITTED, stats.get_emitted());
-         putRawKV(ret, TRANSFERRED, stats.get_transferred());
-         putRawKV(ret, "rate", stats.get_rate());
++        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()) {
 +            mergeMaps(ret, clojurifySpecificStats(stats.get_specific().get_bolt()));
-             putRawKV(ret, TYPE, KW_BOLT);
++            putKV(ret, TYPE, KW_BOLT);
 +        } else {
 +            mergeMaps(ret, clojurifySpecificStats(stats.get_specific().get_spout()));
-             putRawKV(ret, TYPE, KW_SPOUT);
++            putKV(ret, TYPE, KW_SPOUT);
 +        }
 +
 +        return ret;
 +    }
 +
 +    public static Map clojurifySpecificStats(SpoutStats stats) {
 +        Map ret = new HashMap();
-         putRawKV(ret, ACKED, stats.get_acked());
-         putRawKV(ret, FAILED, stats.get_failed());
-         putRawKV(ret, COMP_LATENCIES, stats.get_complete_ms_avg());
++        putKV(ret, ACKED, stats.get_acked());
++        putKV(ret, FAILED, stats.get_failed());
++        putKV(ret, COMP_LATENCIES, stats.get_complete_ms_avg());
 +
 +        return ret;
 +    }
 +
 +    public static Map clojurifySpecificStats(BoltStats stats) {
 +        Map 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);
 +
-         putRawKV(ret, ACKED, acked);
-         putRawKV(ret, FAILED, failed);
-         putRawKV(ret, PROC_LATENCIES, processAvg);
-         putRawKV(ret, EXECUTED, executed);
-         putRawKV(ret, EXEC_LATENCIES, executeAvg);
++        putKV(ret, ACKED, acked);
++        putKV(ret, FAILED, failed);
++        putKV(ret, PROC_LATENCIES, processAvg);
++        putKV(ret, EXECUTED, executed);
++        putKV(ret, EXEC_LATENCIES, executeAvg);
 +
 +        return ret;
 +    }
 +
 +    public static List extractNodeInfosFromHbForComp(
 +            Map exec2hostPort, Map task2component, boolean includeSys, String compId) {
 +        List ret = new ArrayList();
 +
 +        Set<List> hostPorts = new HashSet<>();
 +        for (Object o : exec2hostPort.entrySet()) {
 +            Map.Entry entry = (Map.Entry) o;
 +            List key = (List) entry.getKey();
 +            List value = (List) entry.getValue();
 +
 +            Integer start = ((Number) key.get(0)).intValue();
 +            String host = (String) value.get(0);
 +            Integer port = (Integer) value.get(1);
 +            String comp = (String) task2component.get(start);
 +            if ((compId == null || compId.equals(comp)) && (includeSys || !Utils.isSystemId(comp))) {
 +                hostPorts.add(Lists.newArrayList(host, port));
 +            }
 +        }
 +
 +        for (List hostPort : hostPorts) {
 +            Map m = new HashMap();
-             putRawKV(m, HOST, hostPort.get(0));
-             putRawKV(m, PORT, hostPort.get(1));
++            putKV(m, HOST, hostPort.get(0));
++            putKV(m, PORT, hostPort.get(1));
 +            ret.add(m);
 +        }
 +
 +        return ret;
 +    }
 +
-     public static List extractDataFromHb(Map executor2hostPort, Map task2component, Map beats,
-                                          boolean includeSys, StormTopology topology) {
++    /**
++     * extracts a list of executor data from heart beats
++     */
++    public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component, Map beats,
++                                                              boolean includeSys, StormTopology topology) {
 +        return extractDataFromHb(executor2hostPort, task2component, beats, includeSys, topology, null);
 +    }
 +
-     public static List extractDataFromHb(Map executor2hostPort, Map task2component, Map beats,
-                                          boolean includeSys, StormTopology topology, String compId) {
-         List ret = new ArrayList();
++    public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component, Map beats,
++                                                              boolean includeSys, StormTopology topology, String compId) {
++        List<Map<String, Object>> ret = new ArrayList<>();
 +        if (executor2hostPort == null) {
 +            return ret;
 +        }
 +        for (Object o : executor2hostPort.entrySet()) {
 +            Map.Entry entry = (Map.Entry) o;
 +            List key = (List) entry.getKey();
 +            List value = (List) entry.getValue();
 +
 +            Integer start = ((Number) key.get(0)).intValue();
 +            Integer end = ((Number) key.get(1)).intValue();
 +
 +            String host = (String) value.get(0);
 +            Integer port = ((Number) value.get(1)).intValue();
 +
 +            Map beat = (Map) beats.get(key);
 +            if (beat == null) {
 +                continue;
 +            }
 +            String id = (String) task2component.get(start);
 +
-             Map m = new HashMap();
++            Map<String, Object> m = new HashMap<>();
 +            if ((compId == null || compId.equals(id)) && (includeSys || !Utils.isSystemId(id))) {
-                 putRawKV(m, "exec-id", entry.getKey());
-                 putRawKV(m, "comp-id", id);
-                 putRawKV(m, NUM_TASKS, end - start + 1);
-                 putRawKV(m, HOST, host);
-                 putRawKV(m, PORT, port);
-                 putRawKV(m, UPTIME, beat.get(keyword(UPTIME)));
-                 putRawKV(m, STATS, beat.get(keyword(STATS)));
++                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);
++                putKV(m, UPTIME, beat.get(keyword(UPTIME)));
++                putKV(m, STATS, beat.get(keyword(STATS)));
 +
 +                Keyword type = componentType(topology, compId);
 +                if (type != null) {
-                     putRawKV(m, TYPE, type);
++                    putKV(m, TYPE, type);
 +                } else {
-                     putRawKV(m, TYPE, getByKeyword(getMapByKeyword(beat, STATS), TYPE));
++                    putKV(m, TYPE, getByKey(getMapByKey(beat, STATS), TYPE));
 +                }
 +                ret.add(m);
 +            }
 +        }
 +        return ret;
 +    }
 +
 +    private static Map computeWeightedAveragesPerWindow(Map accData, String wgtAvgKey, String divisorKey) {
 +        Map ret = new HashMap();
-         for (Object o : getMapByKeyword(accData, wgtAvgKey).entrySet()) {
++        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) getMapByKeyword(accData, divisorKey).get(window)).longValue();
++            long divisor = ((Number) 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 executorSumms) {
 +        double max = 0.0;
 +        for (Object o : executorSumms) {
 +            ExecutorSummary summary = (ExecutorSummary) o;
 +            double capacity = computeExecutorCapacity(summary);
 +            if (capacity > max) {
 +                max = capacity;
 +            }
 +        }
 +        return max;
 +    }
 +
 +    public static double computeExecutorCapacity(ExecutorSummary summ) {
 +        ExecutorStats stats = summ.get_stats();
 +        if (stats == null) {
 +            return 0.0;
 +        } else {
 +            Map m = aggregateBoltStats(Lists.newArrayList(stats), true);
 +            m = swapMapOrder(aggregateBoltStreams(m));
-             Map data = getMapByKeyword(m, TEN_MIN_IN_SECONDS_STR);
++            Map data = getMapByKey(m, TEN_MIN_IN_SECONDS_STR);
 +
 +            int uptime = summ.get_uptime_secs();
 +            int win = Math.min(uptime, TEN_MIN_IN_SECONDS);
 +            long executed = getByKeywordOr0(data, EXECUTED).longValue();
 +            double latency = getByKeywordOr0(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 getFilledStats(List summs) {
 +        for (Iterator itr = summs.iterator(); itr.hasNext(); ) {
 +            ExecutorSummary summ = (ExecutorSummary) itr.next();
 +            if (summ.get_stats() == null) {
 +                itr.remove();
 +            }
 +        }
 +        return summs;
 +    }
 +
 +    private static Map mapKeyStr(Map m) {
 +        Map ret = new HashMap();
 +        for (Object k : m.keySet()) {
 +            ret.put(k.toString(), m.get(k));
 +        }
 +        return ret;
 +    }
 +
 +    private static long sumStreamsLong(Map m, String key) {
 +        long sum = 0;
 +        if (m == null) {
 +            return sum;
 +        }
 +        for (Object v : m.values()) {
 +            Map sub = (Map) v;
 +            for (Object o : sub.entrySet()) {
 +                Map.Entry e = (Map.Entry) o;
 +                if (((Keyword) e.getKey()).getName().equals(key)) {
 +                    sum += ((Number) e.getValue()).longValue();
 +                }
 +            }
 +        }
 +        return sum;
 +    }
 +
 +    private static double sumStreamsDouble(Map m, String key) {
 +        double sum = 0;
 +        if (m == null) {
 +            return sum;
 +        }
 +        for (Object v : m.values()) {
 +            Map sub = (Map) v;
 +            for (Object o : sub.entrySet()) {
 +                Map.Entry e = (Map.Entry) o;
 +                if (((Keyword) e.getKey()).getName().equals(key)) {
 +                    sum += ((Number) e.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 stats      { win -> stream id -> value }
 +     * @param includeSys whether to filter system streams
 +     * @return filtered stats
 +     */
 +    private static Map filterSysStreams(Map stats, boolean includeSys) {
 +        if (!includeSys) {
 +            for (Iterator itr = stats.keySet().iterator(); itr.hasNext(); ) {
 +                Object winOrStream = itr.next();
 +                if (isWindow(winOrStream)) {
 +                    Map stream2stat = (Map) 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();
 +                        }
 +                    }
 +                } else {
 +                    if (winOrStream instanceof String && Utils.isSystemId((String) winOrStream)) {
 +                        itr.remove();
 +                    }
 +                }
 +            }
 +        }
 +        return stats;
 +    }
 +
 +    private static boolean isWindow(Object key) {
 +        return key.equals("600") || key.equals("10800") || key.equals("86400") || key.equals(":all-time");
 +    }
 +
 +    /**
 +     * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out)
 +     */
 +    private static Map fullMergeWithSum(Map m1, Map m2) {
 +        Set<Object> allKeys = new HashSet<>();
 +        if (m1 != null) {
 +            allKeys.addAll(m1.keySet());
 +        }
 +        if (m2 != null) {
 +            allKeys.addAll(m2.keySet());
 +        }
 +
 +        Map ret = new HashMap();
 +        for (Object k : allKeys) {
 +            Map mm1 = null, mm2 = null;
 +            if (m1 != null) {
 +                mm1 = (Map) m1.get(k);
 +            }
 +            if (m2 != null) {
 +                mm2 = (Map) m2.get(k);
 +            }
 +            ret.put(k, mergeWithSum(mm1, mm2));
 +        }
 +
 +        return ret;
 +    }
 +
 +    private static Map mergeWithSum(Map m1, Map m2) {
 +        Map ret = new HashMap();
 +
 +        Set<Object> allKeys = new HashSet<>();
 +        if (m1 != null) {
 +            allKeys.addAll(m1.keySet());
 +        }
 +        if (m2 != null) {
 +            allKeys.addAll(m2.keySet());
 +        }
 +
 +        for (Object k : allKeys) {
 +            Number n1 = getOr0(m1, k);
 +            Number n2 = getOr0(m2, k);
 +            ret.put(k, add(n1, n2));
 +        }
 +        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 Map mergeWithAddPair(Map m1, Map m2) {
 +        Map ret = new HashMap();
 +
 +        Set<Object> allKeys = new HashSet<>();
 +        if (m1 != null) {
 +            allKeys.addAll(m1.keySet());
 +        }
 +        if (m2 != null) {
 +            allKeys.addAll(m2.keySet());
 +        }
 +
 +        for (Object k : allKeys) {
 +            Map mm1 = (m1 != null) ? (Map) m1.get(k) : null;
 +            Map mm2 = (m2 != null) ? (Map) 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 tmp = new HashMap();
 +                for (Object kk : mm1.keySet()) {
 +                    List seq1 = (List) mm1.get(kk);
 +                    List seq2 = (List) mm2.get(kk);
 +                    List sums = new ArrayList();
 +                    for (int i = 0; i < seq1.size(); i++) {
 +                        sums.add(add((Number) seq1.get(i), (Number) seq2.get(i)));
 +                    }
 +                    tmp.put(kk, sums);
 +                }
 +                ret.put(k, t

<TRUNCATED>

[23/30] storm git commit: 1. changed heartbeat structure to java HashMap 2. use HashMaps in StatsUtil instead of clojure map 3. changed tests accordingly

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index 351e830..7650ab1 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -17,8 +17,6 @@
  */
 package org.apache.storm.stats;
 
-import clojure.lang.Keyword;
-import clojure.lang.RT;
 import com.google.common.collect.Lists;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -27,10 +25,12 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 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;
@@ -48,19 +48,18 @@ import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.generated.TopologyPageInfo;
 import org.apache.storm.generated.TopologyStats;
+import org.apache.storm.utils.Time;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@SuppressWarnings("unchecked, unused")
+@SuppressWarnings("unchecked")
 public class StatsUtil {
     private static final Logger logger = LoggerFactory.getLogger(StatsUtil.class);
 
     public static final String TYPE = "type";
     private static final String SPOUT = "spout";
     private static final String BOLT = "bolt";
-    public static final Keyword KW_SPOUT = keyword(SPOUT);
-    public static final Keyword KW_BOLT = keyword(BOLT);
 
     private static final String UPTIME = "uptime";
     private static final String HOST = "host";
@@ -73,7 +72,10 @@ public class StatsUtil {
     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";
@@ -130,8 +132,10 @@ public class StatsUtil {
      * @param id2procAvg { global stream id -> proc avg value }
      * @param id2numExec { global stream id -> executed }
      */
-    public static Map aggBoltLatAndCount(Map id2execAvg, Map id2procAvg, Map id2numExec) {
-        Map ret = new HashMap();
+    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));
@@ -142,8 +146,8 @@ public class StatsUtil {
     /**
      * Aggregates number acked and complete latencies across all streams.
      */
-    public static Map aggSpoutLatAndCount(Map id2compAvg, Map id2numAcked) {
-        Map ret = new HashMap();
+    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));
 
@@ -185,7 +189,7 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggPreMergeCompPageBolt(Map m, String window, boolean includeSys) {
+    public static Map aggPreMergeCompPageBolt(Map<String, Object> m, String window, boolean includeSys) {
         Map ret = new HashMap();
         putKV(ret, EXECUTOR_ID, getByKey(m, "exec-id"));
         putKV(ret, HOST, getByKey(m, HOST));
@@ -195,7 +199,7 @@ public class StatsUtil {
         putKV(ret, NUM_TASKS, getByKey(m, NUM_TASKS));
 
         Map stat2win2sid2num = getMapByKey(m, STATS);
-        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
+        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(m, UPTIME).intValue()));
 
         // calc cid+sid->input_stats
         Map inputStats = new HashMap();
@@ -232,8 +236,8 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggPreMergeCompPageSpout(Map m, String window, boolean includeSys) {
-        Map ret = new HashMap();
+    public static Map<String, Object> aggPreMergeCompPageSpout(Map<String, Object> m, String window, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
         putKV(ret, EXECUTOR_ID, getByKey(m, "exec-id"));
         putKV(ret, HOST, getByKey(m, HOST));
         putKV(ret, PORT, getByKey(m, PORT));
@@ -265,97 +269,111 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggPreMergeTopoPageBolt(Map m, String window, boolean includeSys) {
-        Map ret = new HashMap();
+    public static <K, V extends Number> Map<String, Object> aggPreMergeTopoPageBolt(
+            Map<String, Object> m, String window, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
 
-        Map subRet = new HashMap();
+        Map<String, Object> subRet = new HashMap<>();
         putKV(subRet, NUM_EXECUTORS, 1);
         putKV(subRet, NUM_TASKS, getByKey(m, NUM_TASKS));
 
-        Map stat2win2sid2num = getMapByKey(m, STATS);
-        putKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
+        Map<String, Object> stat2win2sid2num = getMapByKey(m, STATS);
+        putKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(m, UPTIME).intValue()));
 
         for (String key : new String[]{EMITTED, TRANSFERRED, ACKED, FAILED}) {
-            Map stat = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, key), TO_STRING).get(window);
+            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 (stat != null) {
-                for (Object o : stat.values()) {
-                    sum += ((Number) o).longValue();
+            if (winStat != null) {
+                for (V v : winStat.values()) {
+                    sum += v.longValue();
                 }
             }
             putKV(subRet, key, sum);
         }
 
-        Map win2sid2execLat = windowSetConverter(getMapByKey(stat2win2sid2num, EXEC_LATENCIES), TO_STRING);
-        Map win2sid2procLat = windowSetConverter(getMapByKey(stat2win2sid2num, PROC_LATENCIES), TO_STRING);
-        Map win2sid2exec = windowSetConverter(getMapByKey(stat2win2sid2num, EXECUTED), TO_STRING);
+        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(
-                (Map) win2sid2execLat.get(window), (Map) win2sid2procLat.get(window), (Map) win2sid2exec.get(window)));
+                win2sid2execLat.get(window), win2sid2procLat.get(window), win2sid2exec.get(window)));
 
-        ret.put(getByKey(m, "comp-id"), subRet);
+        ret.put((String) getByKey(m, "comp-id"), subRet);
         return ret;
     }
 
-    public static Map aggPreMergeTopoPageSpout(Map m, String window, boolean includeSys) {
-        Map ret = new HashMap();
+    /**
+     * 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 subRet = 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 stat2win2sid2num = getMapByKey(m, STATS);
+        Map<String, Map<String, Map<String, V>>> stat2win2sid2num = getMapByKey(m, STATS);
         for (String key : new String[]{EMITTED, TRANSFERRED, FAILED}) {
-            Map stat = (Map) windowSetConverter(getMapByKey(stat2win2sid2num, key), TO_STRING).get(window);
+            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 (stat != null) {
-                for (Object o : stat.values()) {
-                    sum += ((Number) o).longValue();
+            if (winStat != null) {
+                for (V v : winStat.values()) {
+                    sum += v.longValue();
                 }
             }
             putKV(subRet, key, sum);
         }
 
-        Map win2sid2compLat = windowSetConverter(getMapByKey(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
-        Map win2sid2acked = windowSetConverter(getMapByKey(stat2win2sid2num, ACKED), TO_STRING);
-        subRet.putAll(aggSpoutLatAndCount((Map) win2sid2compLat.get(window), (Map) win2sid2acked.get(window)));
+        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(getByKey(m, "comp-id"), subRet);
+        ret.put((String) getByKey(m, "comp-id"), subRet);
         return ret;
     }
 
-    public static Map mergeAggCompStatsCompPageBolt(Map accBoltStats, Map boltStats) {
-        Map ret = new HashMap();
+    public static Map<String, Object> mergeAggCompStatsCompPageBolt(
+            Map<String, Object> accBoltStats, Map<String, Object> boltStats) {
+        Map<String, Object> ret = new HashMap<>();
 
-        Map accIn = getMapByKey(accBoltStats, CID_SID_TO_IN_STATS);
-        Map accOut = getMapByKey(accBoltStats, SID_TO_OUT_STATS);
-        Map boltIn = getMapByKey(boltStats, CID_SID_TO_IN_STATS);
-        Map boltOut = getMapByKey(boltStats, SID_TO_OUT_STATS);
+        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 = getByKeywordOr0(accBoltStats, NUM_EXECUTORS).intValue();
+        int numExecutors = getByKeyOr0(accBoltStats, NUM_EXECUTORS).intValue();
         putKV(ret, NUM_EXECUTORS, numExecutors + 1);
         putKV(ret, NUM_TASKS, sumOr0(
-                getByKeywordOr0(accBoltStats, NUM_TASKS), getByKeywordOr0(boltStats, NUM_TASKS)));
+                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 executorStats = new HashMap();
-        putKV(executorStats, EXECUTOR_ID, getByKey(boltStats, EXECUTOR_ID));
-        putKV(executorStats, UPTIME, getByKey(boltStats, UPTIME));
-        putKV(executorStats, HOST, getByKey(boltStats, HOST));
-        putKV(executorStats, PORT, getByKey(boltStats, PORT));
-        putKV(executorStats, CAPACITY, getByKey(boltStats, CAPACITY));
+        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));
@@ -377,16 +395,18 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map mergeAggCompStatsCompPageSpout(Map accSpoutStats, Map spoutStats) {
-        Map ret = new HashMap();
+    public static Map<String, Object> mergeAggCompStatsCompPageSpout(
+            Map<String, Object> accSpoutStats, Map<String, Object> spoutStats) {
+        Map<String, Object> ret = new HashMap<>();
 
-        Map accOut = getMapByKey(accSpoutStats, SID_TO_OUT_STATS);
-        Map spoutOut = getMapByKey(spoutStats, SID_TO_OUT_STATS);
+        // {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 = getByKeywordOr0(accSpoutStats, NUM_EXECUTORS).intValue();
+        int numExecutors = getByKeyOr0(accSpoutStats, NUM_EXECUTORS).intValue();
         putKV(ret, NUM_EXECUTORS, numExecutors + 1);
         putKV(ret, NUM_TASKS, sumOr0(
-                getByKeywordOr0(accSpoutStats, NUM_TASKS), getByKeywordOr0(spoutStats, NUM_TASKS)));
+                getByKeyOr0(accSpoutStats, NUM_TASKS), getByKeyOr0(spoutStats, NUM_TASKS)));
         putKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
 
         Map executorStats = new HashMap();
@@ -412,48 +432,50 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map mergeAggCompStatsTopoPageBolt(Map accBoltStats, Map boltStats) {
-        Map ret = new HashMap();
-        Integer numExecutors = getByKeywordOr0(accBoltStats, NUM_EXECUTORS).intValue();
+    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(
-                getByKeywordOr0(accBoltStats, NUM_TASKS), getByKeywordOr0(boltStats, NUM_TASKS)));
-        putKV(ret, EMITTED, sumOr0(
-                getByKeywordOr0(accBoltStats, EMITTED), getByKeywordOr0(boltStats, EMITTED)));
-        putKV(ret, TRANSFERRED, sumOr0(
-                getByKeywordOr0(accBoltStats, TRANSFERRED), getByKeywordOr0(boltStats, TRANSFERRED)));
-        putKV(ret, EXEC_LAT_TOTAL, sumOr0(
-                getByKeywordOr0(accBoltStats, EXEC_LAT_TOTAL), getByKeywordOr0(boltStats, EXEC_LAT_TOTAL)));
-        putKV(ret, PROC_LAT_TOTAL, sumOr0(
-                getByKeywordOr0(accBoltStats, PROC_LAT_TOTAL), getByKeywordOr0(boltStats, PROC_LAT_TOTAL)));
-        putKV(ret, EXECUTED, sumOr0(
-                getByKeywordOr0(accBoltStats, EXECUTED), getByKeywordOr0(boltStats, EXECUTED)));
-        putKV(ret, ACKED, sumOr0(
-                getByKeywordOr0(accBoltStats, ACKED), getByKeywordOr0(boltStats, ACKED)));
-        putKV(ret, FAILED, sumOr0(
-                getByKeywordOr0(accBoltStats, FAILED), getByKeywordOr0(boltStats, FAILED)));
-        putKV(ret, CAPACITY, maxOr0(
-                getByKeywordOr0(accBoltStats, CAPACITY), getByKeywordOr0(boltStats, CAPACITY)));
+        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;
     }
 
-    public static Map mergeAggCompStatsTopoPageSpout(Map accSpoutStats, Map spoutStats) {
-        Map ret = new HashMap();
-        Integer numExecutors = getByKeywordOr0(accSpoutStats, NUM_EXECUTORS).intValue();
+    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(
-                getByKeywordOr0(accSpoutStats, NUM_TASKS), getByKeywordOr0(spoutStats, NUM_TASKS)));
-        putKV(ret, EMITTED, sumOr0(
-                getByKeywordOr0(accSpoutStats, EMITTED), getByKeywordOr0(spoutStats, EMITTED)));
-        putKV(ret, TRANSFERRED, sumOr0(
-                getByKeywordOr0(accSpoutStats, TRANSFERRED), getByKeywordOr0(spoutStats, TRANSFERRED)));
-        putKV(ret, COMP_LAT_TOTAL, sumOr0(
-                getByKeywordOr0(accSpoutStats, COMP_LAT_TOTAL), getByKeywordOr0(spoutStats, COMP_LAT_TOTAL)));
-        putKV(ret, ACKED, sumOr0(
-                getByKeywordOr0(accSpoutStats, ACKED), getByKeywordOr0(spoutStats, ACKED)));
-        putKV(ret, FAILED, sumOr0(
-                getByKeywordOr0(accSpoutStats, FAILED), getByKeywordOr0(spoutStats, FAILED)));
+        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;
     }
@@ -462,10 +484,11 @@ public class StatsUtil {
      * 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 aggTopoExecStats(String window, boolean includeSys, Map accStats, Map newData, String compType) {
-        Map ret = new HashMap();
+    public static Map<String, Object> aggTopoExecStats(
+            String window, boolean includeSys, Map<String, Object> accStats, Map<String, Object> newData, String compType) {
+        Map<String, Object> ret = new HashMap<>();
 
-        Set workerSet = (Set) getByKey(accStats, WORKERS_SET);
+        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);
@@ -473,16 +496,17 @@ public class StatsUtil {
         Map win2compLatWgtAvg = getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG);
         Map win2acked = getMapByKey(accStats, WIN_TO_ACKED);
         Map win2failed = getMapByKey(accStats, WIN_TO_FAILED);
-        Map stats = getMapByKey(newData, STATS);
 
         boolean isSpout = compType.equals(SPOUT);
-        Map cid2stat2num;
+        // component id -> stats
+        Map<String, Object> cid2stats;
         if (isSpout) {
-            cid2stat2num = aggPreMergeTopoPageSpout(newData, window, includeSys);
+            cid2stats = aggPreMergeTopoPageSpout(newData, window, includeSys);
         } else {
-            cid2stat2num = aggPreMergeTopoPageBolt(newData, window, includeSys);
+            cid2stats = aggPreMergeTopoPageBolt(newData, window, includeSys);
         }
 
+        Map stats = getMapByKey(newData, STATS);
         Map w2compLatWgtAvg, w2acked;
         Map compLatStats = getMapByKey(stats, COMP_LATENCIES);
         if (isSpout) { // agg spout stats
@@ -504,38 +528,38 @@ public class StatsUtil {
         putKV(ret, WORKERS_SET, workerSet);
         putKV(ret, BOLT_TO_STATS, bolt2stats);
         putKV(ret, SPOUT_TO_STATS, spout2stats);
-        putKV(ret, WIN_TO_EMITTED, mergeWithSum(win2emitted, aggregateCountStreams(
+        putKV(ret, WIN_TO_EMITTED, mergeWithSumLong(win2emitted, aggregateCountStreams(
                 filterSysStreams(getMapByKey(stats, EMITTED), includeSys))));
-        putKV(ret, WIN_TO_TRANSFERRED, mergeWithSum(win2transferred, aggregateCountStreams(
+        putKV(ret, WIN_TO_TRANSFERRED, mergeWithSumLong(win2transferred, aggregateCountStreams(
                 filterSysStreams(getMapByKey(stats, TRANSFERRED), includeSys))));
-        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(win2compLatWgtAvg, w2compLatWgtAvg));
+        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 ? mergeWithSum(win2acked, w2acked) : win2acked);
+        putKV(ret, WIN_TO_ACKED, isSpout ? mergeWithSumLong(win2acked, w2acked) : win2acked);
         putKV(ret, WIN_TO_FAILED, isSpout ?
-                mergeWithSum(aggregateCountStreams(getMapByKey(stats, FAILED)), win2failed) : win2failed);
+                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) {
-            Set<Object> keySet = new HashSet<>();
+            Set<String> keySet = new HashSet<>();
             keySet.addAll(spout2stats.keySet());
-            keySet.addAll(cid2stat2num.keySet());
+            keySet.addAll(cid2stats.keySet());
 
             Map mm = new HashMap();
-            for (Object k : keySet) {
-                mm.put(k, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(k), (Map) cid2stat2num.get(k)));
+            for (String k : keySet) {
+                mm.put(k, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(k), (Map) cid2stats.get(k)));
             }
             putKV(ret, SPOUT_TO_STATS, mm);
         } else {
-            Set<Object> keySet = new HashSet<>();
+            Set<String> keySet = new HashSet<>();
             keySet.addAll(bolt2stats.keySet());
-            keySet.addAll(cid2stat2num.keySet());
+            keySet.addAll(cid2stats.keySet());
 
             Map mm = new HashMap();
-            for (Object k : keySet) {
-                mm.put(k, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(k), (Map) cid2stat2num.get(k)));
+            for (String k : keySet) {
+                mm.put(k, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(k), (Map) cid2stats.get(k)));
             }
             putKV(ret, BOLT_TO_STATS, mm);
         }
@@ -543,18 +567,30 @@ public class StatsUtil {
         return ret;
     }
 
+    /**
+     * aggregate topo executors stats
+     * TODO: change clojure maps to java HashMap's when nimbus.clj is translated to java
+     *
+     * @param topologyId     topology id
+     * @param exec2nodePort  executor -> host+port, note it's a clojure map
+     * @param task2component task -> component, note it's a clojure map
+     * @param beats          executor[start, end] -> executor heartbeat, note it's a java HashMap
+     * @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 beats, StormTopology topology, String window, boolean includeSys, IStormClusterState clusterState) {
-        List beatList = extractDataFromHb(exec2nodePort, task2component, beats, includeSys, topology);
-        Map topoStats = aggregateTopoStats(window, includeSys, beatList);
-        topoStats = postAggregateTopoStats(task2component, exec2nodePort, topoStats, topologyId, clusterState);
-
-        return thriftifyTopoPageData(topologyId, topoStats);
+            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 aggregateTopoStats(String win, boolean includeSys, List data) {
-        Map initVal = new HashMap();
+    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());
@@ -564,68 +600,72 @@ public class StatsUtil {
         putKV(initVal, WIN_TO_ACKED, new HashMap());
         putKV(initVal, WIN_TO_FAILED, new HashMap());
 
-        for (Object o : data) {
-            Map newData = (Map) o;
-            String compType = ((Keyword) getByKey(newData, TYPE)).getName();
-            initVal = aggTopoExecStats(win, includeSys, initVal, newData, compType);
+        for (Map<String, Object> heartbeat : heartbeats) {
+            String compType = (String) getByKey(heartbeat, TYPE);
+            initVal = aggTopoExecStats(win, includeSys, initVal, heartbeat, compType);
         }
 
         return initVal;
     }
 
-    public static Map postAggregateTopoStats(
-            Map task2comp, Map exec2nodePort, Map accData, String topologyId, IStormClusterState clusterState) {
-        Map ret = new HashMap();
-        putKV(ret, NUM_TASKS, task2comp.size());
-        putKV(ret, NUM_WORKERS, ((Set) getByKey(accData, WORKERS_SET)).size());
-        putKV(ret, NUM_EXECUTORS, exec2nodePort != null ? exec2nodePort.size() : 0);
+    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 aggBolt2stats = new HashMap();
+        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 = getByKeywordOr0(m, EXECUTED).longValue();
+            long executed = getByKeyOr0(m, EXECUTED).longValue();
             if (executed > 0) {
-                double execLatencyTotal = getByKeywordOr0(m, EXEC_LAT_TOTAL).doubleValue();
+                double execLatencyTotal = getByKeyOr0(m, EXEC_LAT_TOTAL).doubleValue();
                 putKV(m, EXEC_LATENCY, execLatencyTotal / executed);
 
-                double procLatencyTotal = getByKeywordOr0(m, PROC_LAT_TOTAL).doubleValue();
+                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, m);
+            aggBolt2stats.put(id, thriftifyBoltAggStats(m));
         }
-        putKV(ret, BOLT_TO_STATS, aggBolt2stats);
 
         Map spout2stats = getMapByKey(accData, SPOUT_TO_STATS);
-        Map spoutBolt2stats = new HashMap();
+        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 = getByKeywordOr0(m, ACKED).longValue();
+            long acked = getByKeyOr0(m, ACKED).longValue();
             if (acked > 0) {
-                double compLatencyTotal = getByKeywordOr0(m, COMP_LAT_TOTAL).doubleValue();
+                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));
 
-            spoutBolt2stats.put(id, m);
+            aggSpout2stats.put(id, thriftifySpoutAggStats(m));
         }
-        putKV(ret, SPOUT_TO_STATS, spoutBolt2stats);
 
-        putKV(ret, WIN_TO_EMITTED, mapKeyStr(getMapByKey(accData, WIN_TO_EMITTED)));
-        putKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKey(accData, WIN_TO_TRANSFERRED)));
-        putKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKey(accData, WIN_TO_ACKED)));
-        putKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKey(accData, WIN_TO_FAILED)));
-        putKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
+        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;
     }
 
@@ -636,17 +676,19 @@ public class StatsUtil {
      * @param includeSys whether to include system streams
      * @return aggregated bolt stats
      */
-    public static Map aggregateBoltStats(List statsSeq, boolean includeSys) {
-        Map ret = new HashMap();
-
-        Map commonStats = preProcessStreamSummary(aggregateCommonStats(statsSeq), includeSys);
-        List acked = new ArrayList();
-        List failed = new ArrayList();
-        List executed = new ArrayList();
-        List processLatencies = new ArrayList();
-        List executeLatencies = new ArrayList();
-        for (Object o : statsSeq) {
-            ExecutorStats stat = (ExecutorStats) o;
+    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);
+        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());
@@ -670,20 +712,23 @@ public class StatsUtil {
      * @param includeSys whether to include system streams
      * @return aggregated spout stats
      */
-    public static Map aggregateSpoutStats(List statsSeq, boolean includeSys) {
-        Map ret = new HashMap();
-
-        Map commonStats = preProcessStreamSummary(aggregateCommonStats(statsSeq), includeSys);
-        List acked = new ArrayList();
-        List failed = new ArrayList();
-        List completeLatencies = new ArrayList();
-        for (Object o : statsSeq) {
-            ExecutorStats stat = (ExecutorStats) o;
-            acked.add(stat.get_specific().get_spout().get_acked());
-            failed.add(stat.get_specific().get_spout().get_failed());
-            completeLatencies.add(stat.get_specific().get_spout().get_complete_ms_avg());
-        }
-        mergeMaps(ret, commonStats);
+    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);
+        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));
@@ -691,25 +736,25 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggregateCommonStats(List statsSeq) {
-        Map ret = new HashMap();
+    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 emitted = new ArrayList();
-        List transferred = new ArrayList();
-        for (Object o : statsSeq) {
-            ExecutorStats stat = (ExecutorStats) o;
-            emitted.add(stat.get_emitted());
-            transferred.add(stat.get_transferred());
+        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;
     }
 
-    public static Map preProcessStreamSummary(Map streamSummary, boolean includeSys) {
-        Map emitted = getMapByKey(streamSummary, EMITTED);
-        Map transferred = getMapByKey(streamSummary, TRANSFERRED);
+    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));
@@ -717,32 +762,32 @@ public class StatsUtil {
         return streamSummary;
     }
 
-    public static Map aggregateCountStreams(Map stats) {
-        Map ret = new HashMap();
-        for (Object o : stats.entrySet()) {
-            Map.Entry entry = (Map.Entry) o;
-            Map value = (Map) entry.getValue();
+    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 (Object num : value.values()) {
-                sum += ((Number) num).longValue();
+            for (V num : value.values()) {
+                sum += num.longValue();
             }
             ret.put(entry.getKey(), sum);
         }
         return ret;
     }
 
-    public static Map aggregateAverages(List avgSeq, List countSeq) {
-        Map ret = new HashMap();
+    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 expands = expandAveragesSeq(avgSeq, countSeq);
-        for (Object o : expands.entrySet()) {
-            Map.Entry entry = (Map.Entry) o;
-            Object k = entry.getKey();
+        Map<String, Map<K, List>> expands = expandAveragesSeq(avgSeq, countSeq);
+        for (Map.Entry<String, Map<K, List>> entry : expands.entrySet()) {
+            String k = entry.getKey();
 
-            Map tmp = new HashMap();
-            Map inner = (Map) entry.getValue();
-            for (Object kk : inner.keySet()) {
-                List vv = (List) inner.get(kk);
+            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);
@@ -751,19 +796,19 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggregateAvgStreams(Map avgs, Map counts) {
-        Map ret = new HashMap();
+    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 expands = expandAverages(avgs, counts);
-        for (Object o : expands.entrySet()) {
-            Map.Entry e = (Map.Entry) o;
-            Object win = e.getKey();
+        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 inner = (Map) e.getValue();
-            for (Object kk : inner.keySet()) {
-                List vv = (List) inner.get(kk);
+            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();
             }
@@ -773,18 +818,25 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map spoutStreamsStats(List summs, boolean includeSys) {
-        List statsSeq = getFilledStats(summs);
+    public static Map<String, Map> spoutStreamsStats(List<ExecutorSummary> summs, boolean includeSys) {
+        if (summs == null) {
+            return new HashMap<>();
+        }
+        List<ExecutorSummary> statsSeq = getFilledStats(summs);
         return aggregateSpoutStreams(aggregateSpoutStats(statsSeq, includeSys));
     }
 
-    public static Map boltStreamsStats(List summs, boolean includeSys) {
-        List statsSeq = getFilledStats(summs);
+    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));
     }
 
-    public static Map aggregateSpoutStreams(Map stats) {
-        Map ret = new HashMap();
+    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)));
@@ -794,8 +846,8 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggregateBoltStreams(Map stats) {
-        Map ret = new HashMap();
+    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)));
@@ -811,41 +863,42 @@ public class StatsUtil {
     /**
      * A helper function that aggregates windowed stats from one spout executor.
      */
-    public static Map aggBoltExecWinStats(Map accStats, Map newStats, boolean includeSys) {
-        Map ret = new HashMap();
+    public static Map<String, Object> aggBoltExecWinStats(
+            Map<String, Object> accStats, Map<String, Object> newStats, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
 
-        Map m = new HashMap();
+        Map<String, Map<String, Number>> m = new HashMap<>();
         for (Object win : getMapByKey(newStats, EXECUTED).keySet()) {
-            m.put(win, aggBoltLatAndCount(
+            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 win2execLatWgtAvg = getMapByKey(m, EXEC_LAT_TOTAL);
-        Map win2procLatWgtAvg = getMapByKey(m, PROC_LAT_TOTAL);
-        Map win2executed = getMapByKey(m, EXECUTED);
+        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 emitted = getMapByKey(newStats, EMITTED);
-        emitted = mergeWithSum(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
+        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, emitted);
+        putKV(ret, WIN_TO_EMITTED, win2emitted);
 
-        Map transferred = getMapByKey(newStats, TRANSFERRED);
-        transferred = mergeWithSum(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
+        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, transferred);
+        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
 
-        putKV(ret, WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSum(
+        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, mergeWithSum(
+        putKV(ret, WIN_TO_PROC_LAT_WGT_AVG, mergeWithSumDouble(
                 getMapByKey(accStats, WIN_TO_PROC_LAT_WGT_AVG), win2procLatWgtAvg));
-        putKV(ret, WIN_TO_EXECUTED, mergeWithSum(
+        putKV(ret, WIN_TO_EXECUTED, mergeWithSumLong(
                 getMapByKey(accStats, WIN_TO_EXECUTED), win2executed));
-        putKV(ret, WIN_TO_ACKED, mergeWithSum(
+        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
                 aggregateCountStreams(getMapByKey(newStats, ACKED)), getMapByKey(accStats, WIN_TO_ACKED)));
-        putKV(ret, WIN_TO_FAILED, mergeWithSum(
+        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
                 aggregateCountStreams(getMapByKey(newStats, FAILED)), getMapByKey(accStats, WIN_TO_FAILED)));
 
         return ret;
@@ -854,36 +907,37 @@ public class StatsUtil {
     /**
      * A helper function that aggregates windowed stats from one spout executor.
      */
-    public static Map aggSpoutExecWinStats(Map accStats, Map newStats, boolean includeSys) {
-        Map ret = new HashMap();
+    public static Map<String, Object> aggSpoutExecWinStats(
+            Map<String, Object> accStats, Map<String, Object> beat, boolean includeSys) {
+        Map<String, Object> ret = new HashMap<>();
 
-        Map m = new HashMap();
-        for (Object win : getMapByKey(newStats, ACKED).keySet()) {
-            m.put(win, aggSpoutLatAndCount(
-                    (Map) (getMapByKey(newStats, COMP_LATENCIES)).get(win),
-                    (Map) (getMapByKey(newStats, ACKED)).get(win)));
+        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 win2compLatWgtAvg = getMapByKey(m, COMP_LAT_TOTAL);
-        Map win2acked = getMapByKey(m, ACKED);
+        Map<String, Double> win2compLatWgtAvg = getMapByKey(m, COMP_LAT_TOTAL);
+        Map<String, Long> win2acked = getMapByKey(m, ACKED);
 
-        Map emitted = getMapByKey(newStats, EMITTED);
-        emitted = mergeWithSum(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
+        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, emitted);
+        putKV(ret, WIN_TO_EMITTED, win2emitted);
 
-        Map transferred = getMapByKey(newStats, TRANSFERRED);
-        transferred = mergeWithSum(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
+        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, transferred);
+        putKV(ret, WIN_TO_TRANSFERRED, win2transferred);
 
-        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(
+        putKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSumDouble(
                 getMapByKey(accStats, WIN_TO_COMP_LAT_WGT_AVG), win2compLatWgtAvg));
-        putKV(ret, WIN_TO_ACKED, mergeWithSum(
+        putKV(ret, WIN_TO_ACKED, mergeWithSumLong(
                 getMapByKey(accStats, WIN_TO_ACKED), win2acked));
-        putKV(ret, WIN_TO_FAILED, mergeWithSum(
-                aggregateCountStreams(getMapByKey(newStats, FAILED)), getMapByKey(accStats, WIN_TO_FAILED)));
+        putKV(ret, WIN_TO_FAILED, mergeWithSumLong(
+                aggregateCountStreams(getMapByKey(beat, FAILED)), getMapByKey(accStats, WIN_TO_FAILED)));
 
         return ret;
     }
@@ -894,25 +948,23 @@ public class StatsUtil {
      *
      * @param countsSeq a seq of {win -> GlobalStreamId -> value}
      */
-    public static Map aggregateCounts(List countsSeq) {
-        Map ret = new HashMap();
-        for (Object counts : countsSeq) {
-            for (Object o : ((Map) counts).entrySet()) {
-                Map.Entry e = (Map.Entry) o;
-                Object win = e.getKey();
-                Map stream2count = (Map) e.getValue();
+    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 existing = (Map) ret.get(win);
-                    for (Object oo : stream2count.entrySet()) {
-                        Map.Entry ee = (Map.Entry) oo;
-                        Object stream = ee.getKey();
+                    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, ee.getValue());
+                            existing.put(stream, subEntry.getValue());
                         } else {
-                            existing.put(stream, (Long) ee.getValue() + (Long) existing.get(stream));
+                            existing.put(stream, subEntry.getValue() + existing.get(stream));
                         }
                     }
                 }
@@ -921,23 +973,24 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggregateCompStats(String window, boolean includeSys, List data, String compType) {
+    public static Map<String, Object> aggregateCompStats(String window, boolean includeSys,
+                                                         List<Map<String, Object>> beats, String compType) {
         boolean isSpout = SPOUT.equals(compType);
 
-        Map initVal = new HashMap();
+        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 stats = 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, KW_SPOUT);
+            putKV(initVal, TYPE, SPOUT);
             putKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
         } else {
-            putKV(initVal, TYPE, KW_BOLT);
+            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());
@@ -945,8 +998,8 @@ public class StatsUtil {
         }
         putKV(initVal, STATS, stats);
 
-        for (Object o : data) {
-            initVal = aggCompExecStats(window, includeSys, initVal, (Map) o, compType);
+        for (Map<String, Object> beat : beats) {
+            initVal = aggCompExecStats(window, includeSys, initVal, beat, compType);
         }
 
         return initVal;
@@ -956,41 +1009,50 @@ public class StatsUtil {
      * Combines the aggregate stats of one executor with the given map, selecting
      * the appropriate window and including system components as specified.
      */
-    public static Map aggCompExecStats(String window, boolean includeSys, Map accStats, Map newData, String compType) {
-        Map ret = new HashMap();
+    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(newData, STATS), includeSys));
+            ret.putAll(aggSpoutExecWinStats(accStats, getMapByKey(beat, STATS), includeSys));
             putKV(ret, STATS, mergeAggCompStatsCompPageSpout(
                     getMapByKey(accStats, STATS),
-                    aggPreMergeCompPageSpout(newData, window, includeSys)));
+                    aggPreMergeCompPageSpout(beat, window, includeSys)));
         } else {
-            ret.putAll(aggBoltExecWinStats(accStats, getMapByKey(newData, STATS), includeSys));
+            ret.putAll(aggBoltExecWinStats(accStats, getMapByKey(beat, STATS), includeSys));
             putKV(ret, STATS, mergeAggCompStatsCompPageBolt(
                     getMapByKey(accStats, STATS),
-                    aggPreMergeCompPageBolt(newData, window, includeSys)));
+                    aggPreMergeCompPageBolt(beat, window, includeSys)));
         }
-        putKV(ret, TYPE, keyword(compType));
+        putKV(ret, TYPE, compType);
 
         return ret;
     }
 
-    public static Map postAggregateCompStats(Map task2component, Map exec2hostPort, Map accData) {
-        Map ret = new HashMap();
+    /**
+     * post aggregate component stats
+     *
+     * @param task2component task -> component, note it's a clojure map
+     * @param exec2hostPort  executor -> host+port, note it's a clojure map
+     * @param compStats      accumulated comp stats
+     * @return
+     */
+    public static Map<String, Object> postAggregateCompStats(Map task2component, Map exec2hostPort, Map<String, Object> compStats) {
+        Map<String, Object> ret = new HashMap<>();
 
-        String compType = ((Keyword) getByKey(accData, TYPE)).getName();
-        Map stats = getMapByKey(accData, STATS);
-        Integer numTasks = getByKeywordOr0(stats, NUM_TASKS).intValue();
-        Integer numExecutors = getByKeywordOr0(stats, NUM_EXECUTORS).intValue();
+        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, keyword(compType));
+        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(accData, WIN_TO_EMITTED)));
-        putKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKey(accData, WIN_TO_TRANSFERRED)));
-        putKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKey(accData, WIN_TO_ACKED)));
-        putKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKey(accData, WIN_TO_FAILED)));
+        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);
@@ -1000,10 +1062,10 @@ public class StatsUtil {
                 Map.Entry e = (Map.Entry) o;
                 Object k = e.getKey();
                 Map v = (Map) e.getValue();
-                long executed = getByKeywordOr0(v, EXECUTED).longValue();
+                long executed = getByKeyOr0(v, EXECUTED).longValue();
                 if (executed > 0) {
-                    double executeLatencyTotal = getByKeywordOr0(v, EXEC_LAT_TOTAL).doubleValue();
-                    double processLatencyTotal = getByKeywordOr0(v, PROC_LAT_TOTAL).doubleValue();
+                    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 {
@@ -1017,20 +1079,20 @@ public class StatsUtil {
             putKV(ret, CID_SID_TO_IN_STATS, inStats2);
 
             putKV(ret, SID_TO_OUT_STATS, outStats);
-            putKV(ret, WIN_TO_EXECUTED, mapKeyStr(getMapByKey(accData, WIN_TO_EXECUTED)));
+            putKV(ret, WIN_TO_EXECUTED, mapKeyStr(getMapByKey(compStats, WIN_TO_EXECUTED)));
             putKV(ret, WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
-                    accData, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
+                    compStats, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
             putKV(ret, WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
-                    accData, WIN_TO_PROC_LAT_WGT_AVG, WIN_TO_EXECUTED));
+                    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 = getByKeywordOr0(v, ACKED).longValue();
+                long acked = getByKeyOr0(v, ACKED).longValue();
                 if (acked > 0) {
-                    double compLatencyTotal = getByKeywordOr0(v, COMP_LAT_TOTAL).doubleValue();
+                    double compLatencyTotal = getByKeyOr0(v, COMP_LAT_TOTAL).doubleValue();
                     putKV(v, COMP_LATENCY, compLatencyTotal / acked);
                 } else {
                     putKV(v, COMP_LATENCY, 0.0);
@@ -1040,60 +1102,103 @@ public class StatsUtil {
             }
             putKV(ret, SID_TO_OUT_STATS, outStats2);
             putKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
-                    accData, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
+                    compStats, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
         }
 
         return ret;
     }
 
     public static ComponentPageInfo aggCompExecsStats(
-            Map exec2hostPort, Map task2component, Map beats, String window, boolean includeSys,
-            String topologyId, StormTopology topology, String componentId) {
+            Map exec2hostPort, Map task2component, Map<List<Integer>, Map<String, Object>> beats,
+            String window, boolean includeSys, String topologyId, StormTopology topology, String componentId) {
 
-        List beatList = extractDataFromHb(exec2hostPort, task2component, beats, includeSys, topology, componentId);
-        Map compStats = aggregateCompStats(window, includeSys, beatList, componentType(topology, componentId).getName());
+        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(task2component, exec2hostPort, compStats);
         return thriftifyCompPageData(topologyId, topology, componentId, compStats);
     }
 
 
     // =====================================================================================
-    // clojurify stats methods
+    // convert thrift stats to java maps
     // =====================================================================================
 
-    public static Map clojurifyStats(Map stats) {
-        Map ret = new HashMap();
-        for (Object o : stats.entrySet()) {
-            Map.Entry entry = (Map.Entry) o;
-            ExecutorInfo executorInfo = (ExecutorInfo) entry.getKey();
-            ExecutorStats executorStats = (ExecutorStats) entry.getValue();
+    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 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;
+    }
+
+    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()),
-                    clojurifyExecutorStats(executorStats));
+                    executorStats);
         }
         return ret;
     }
 
-    public static Map clojurifyExecutorStats(ExecutorStats stats) {
-        Map ret = new 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());
+        putKV(ret, RATE, stats.get_rate());
 
         if (stats.get_specific().is_set_bolt()) {
-            mergeMaps(ret, clojurifySpecificStats(stats.get_specific().get_bolt()));
-            putKV(ret, TYPE, KW_BOLT);
+            ret.putAll(convertSpecificStats(stats.get_specific().get_bolt()));
+            putKV(ret, TYPE, BOLT);
         } else {
-            mergeMaps(ret, clojurifySpecificStats(stats.get_specific().get_spout()));
-            putKV(ret, TYPE, KW_SPOUT);
+            ret.putAll(convertSpecificStats(stats.get_specific().get_spout()));
+            putKV(ret, TYPE, SPOUT);
         }
 
         return ret;
     }
 
-    public static Map clojurifySpecificStats(SpoutStats stats) {
-        Map ret = new HashMap();
+    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());
@@ -1101,8 +1206,8 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map clojurifySpecificStats(BoltStats stats) {
-        Map ret = new HashMap();
+    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);
@@ -1119,9 +1224,9 @@ public class StatsUtil {
         return ret;
     }
 
-    public static List extractNodeInfosFromHbForComp(
+    public static List<Map<String, Object>> extractNodeInfosFromHbForComp(
             Map exec2hostPort, Map task2component, boolean includeSys, String compId) {
-        List ret = new ArrayList();
+        List<Map<String, Object>> ret = new ArrayList<>();
 
         Set<List> hostPorts = new HashSet<>();
         for (Object o : exec2hostPort.entrySet()) {
@@ -1139,7 +1244,7 @@ public class StatsUtil {
         }
 
         for (List hostPort : hostPorts) {
-            Map m = new HashMap();
+            Map<String, Object> m = new HashMap<>();
             putKV(m, HOST, hostPort.get(0));
             putKV(m, PORT, hostPort.get(1));
             ret.add(m);
@@ -1148,32 +1253,108 @@ public class StatsUtil {
         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
+     * @return a HashMap of updated executor heart beats
+     */
+    public static Map<List<Integer>, Object> updateHeartbeatCache(Map<List<Integer>, Map<String, Object>> cache,
+                                                                  Map<List<Integer>, Map<String, Object>> executorBeats,
+                                                                  Set<List<Integer>> executors, Integer timeout) {
+        Map<List<Integer>, 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;
+    }
+
+    // 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 = 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);
+        ret.put(HEARTBEAT, newBeat);
+
+        return ret;
+    }
+
+
     /**
      * extracts a list of executor data from heart beats
      */
-    public static List<Map<String, Object>> extractDataFromHb(Map executor2hostPort, Map task2component, Map 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 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) {
         List<Map<String, Object>> ret = new ArrayList<>();
-        if (executor2hostPort == null) {
+        if (executor2hostPort == null || beats == null) {
             return ret;
         }
         for (Object o : executor2hostPort.entrySet()) {
             Map.Entry entry = (Map.Entry) o;
-            List key = (List) entry.getKey();
-            List value = (List) entry.getValue();
+            List executor = (List) entry.getKey();
+            List hostPort = (List) entry.getValue();
 
-            Integer start = ((Number) key.get(0)).intValue();
-            Integer end = ((Number) key.get(1)).intValue();
+            Integer start = ((Number) executor.get(0)).intValue();
+            Integer end = ((Number) executor.get(1)).intValue();
 
-            String host = (String) value.get(0);
-            Integer port = ((Number) value.get(1)).intValue();
+            String host = (String) hostPort.get(0);
+            Integer port = ((Number) hostPort.get(1)).intValue();
 
-            Map beat = (Map) beats.get(key);
+            Map<String, Object> beat = beats.get(convertExecutor(executor));
             if (beat == null) {
                 continue;
             }
@@ -1186,14 +1367,16 @@ public class StatsUtil {
                 putKV(m, NUM_TASKS, end - start + 1);
                 putKV(m, HOST, host);
                 putKV(m, PORT, port);
-                putKV(m, UPTIME, beat.get(keyword(UPTIME)));
-                putKV(m, STATS, beat.get(keyword(STATS)));
 
-                Keyword type = componentType(topology, compId);
+                Map stats = getMapByKey(getMapByKey(beat, (HEARTBEAT)), STATS);
+                putKV(m, UPTIME, getMapByKey(beat, HEARTBEAT).get(UPTIME));
+                putKV(m, STATS, stats);
+
+                String type = componentType(topology, compId);
                 if (type != null) {
                     putKV(m, TYPE, type);
                 } else {
-                    putKV(m, TYPE, getByKey(getMapByKey(beat, STATS), TYPE));
+                    putKV(m, TYPE, stats.get(TYPE));
                 }
                 ret.add(m);
             }
@@ -1201,8 +1384,9 @@ public class StatsUtil {
         return ret;
     }
 
-    private static Map computeWeightedAveragesPerWindow(Map accData, String wgtAvgKey, String divisorKey) {
-        Map ret = new HashMap();
+    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();
@@ -1216,16 +1400,31 @@ public class StatsUtil {
     }
 
 
+    public static Set<List<Integer>> convertExecutors(Set executors) {
+        Set<List<Integer>> convertedExecutors = new HashSet<>();
+        for (Object executor : executors) {
+            List l = (List) executor;
+            convertedExecutors.add(convertExecutor(l));
+        }
+        return convertedExecutors;
+    }
+
+    /**
+     * convert a clojure executor to java List<Integer>
+     */
+    public static List<Integer> convertExecutor(List executor) {
+        return Lists.newArrayList(((Number) executor.get(0)).intValue(), ((Number) executor.get(1)).intValue());
+    }
+
     /**
      * computes max bolt capacity
      *
      * @param executorSumms a list of ExecutorSummary
      * @return max bolt capacity
      */
-    public static double computeBoltCapacity(List executorSumms) {
+    public static double computeBoltCapacity(List<ExecutorSummary> executorSumms) {
         double max = 0.0;
-        for (Object o : executorSumms) {
-            ExecutorSummary summary = (ExecutorSummary) o;
+        for (ExecutorSummary summary : executorSumms) {
             double capacity = computeExecutorCapacity(summary);
             if (capacity > max) {
                 max = capacity;
@@ -1234,19 +1433,22 @@ public class StatsUtil {
         return max;
     }
 
-    public static double computeExecutorCapacity(ExecutorSummary summ) {
-        ExecutorStats stats = summ.get_stats();
+    public static double computeExecutorCapacity(ExecutorSummary summary) {
+        ExecutorStats stats = summary.get_stats();
         if (stats == null) {
             return 0.0;
         } else {
-            Map m = aggregateBoltStats(Lists.newArrayList(stats), true);
+            // 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 = summ.get_uptime_secs();
+            int uptime = summary.get_uptime_secs();
             int win = Math.min(uptime, TEN_MIN_IN_SECONDS);
-            long executed = getByKeywordOr0(data, EXECUTED).longValue();
-            double latency = getByKeywordOr0(data, EXEC_LATENCIES).doubleValue();
+            long executed = getByKeyOr0(data, EXECUTED).longValue();
+            double latency = getByKeyOr0(data, EXEC_LATENCIES).doubleValue();
             if (win > 0) {
                 return executed * latency / (1000 * win);
             }
@@ -1260,35 +1462,33 @@ public class StatsUtil {
      * @param summs a list of ExecutorSummary
      * @return filtered summs
      */
-    public static List getFilledStats(List summs) {
-        for (Iterator itr = summs.iterator(); itr.hasNext(); ) {
-            ExecutorSummary summ = (ExecutorSummary) itr.next();
-            if (summ.get_stats() == null) {
-                itr.remove();
+    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 summs;
+        return ret;
     }
 
-    private static Map mapKeyStr(Map m) {
-        Map ret = new HashMap();
-        for (Object k : m.keySet()) {
-            ret.put(k.toString(), m.get(k));
+    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 long sumStreamsLong(Map m, String key) {
+    private static <K1, K2> long sumStreamsLong(Map<K1, Map<K2, ?>> m, String key) {
         long sum = 0;
         if (m == null) {
             return sum;
         }
-        for (Object v : m.values()) {
-            Map sub = (Map) v;
-            for (Object o : sub.entrySet()) {
-                Map.Entry e = (Map.Entry) o;
-                if (((Keyword) e.getKey()).getName().equals(key)) {
-                    sum += ((Number) e.getValue()).longValue();
+        for (Map<K2, ?> v : m.values()) {
+            for (Map.Entry<K2, ?> entry : v.entrySet()) {
+                if (entry.getKey().equals(key)) {
+                    sum += ((Number) entry.getValue()).longValue();
                 }
             }
         }
@@ -1304,7 +1504,7 @@ public class StatsUtil {
             Map sub = (Map) v;
             for (Object o : sub.entrySet()) {
                 Map.Entry e = (Map.Entry) o;
-                if (((Keyword) e.getKey()).getName().equals(key)) {
+                if (e.getKey().equals(key)) {
                     sum += ((Number) e.getValue()).doubleValue();
                 }
             }
@@ -1340,21 +1540,15 @@ public class StatsUtil {
      * @param includeSys whether to filter system streams
      * @return filtered stats
      */
-    private static Map filterSysStreams(Map stats, boolean includeSys) {
+    private static <K, V> Map<String, Map<K, V>> filterSysStreams(Map<String, Map<K, V>> stats, boolean includeSys) {
         if (!includeSys) {
-            for (Iterator itr = stats.keySet().iterator(); itr.hasNext(); ) {
-                Object winOrStream = itr.next();
-                if (isWindow(winOrStream)) {
-                    Map stream2stat = (Map) 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();
-                        }
-                    }
-                } else {
-                    if (winOrStream instanceof String && Utils.isSystemId((String) winOrStream)) {
-                        itr.remove();
+            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();
                     }
                 }
             }
@@ -1362,15 +1556,12 @@ public class StatsUtil {
         return stats;
     }
 
-    private static boolean isWindow(Object key) {
-        return key.equals("600") || key.equals("10800") || key.equals("86400") || key.equals(":all-time");
-    }
-
     /**
      * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out)
      */
-    private static Map fullMergeWithSum(Map m1, Map m2) {
-        Set<Object> allKeys = new HashSet<>();
+    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());
         }
@@ -1378,14 +1569,14 @@ public class StatsUtil {
             allKeys.addAll(m2.keySet());
         }
 
-        Map ret = new HashMap();
-        for (Object k : allKeys) {
-            Map mm1 = null, mm2 = null;
+        Map<K1, Map<K2, Number>> ret = new HashMap<>();
+        for (K1 k : allKeys) {
+            Map<K2, ?> mm1 = null, mm2 = null;
             if (m1 != null) {
-                mm1 = (Map) m1.get(k);
+                mm1 = m1.get(k);
             }
             if (m2 != null) {
-                mm2 = (Map) m2.get(k);
+                mm2 = m2.get(k);
             }
             ret.put(k, mergeWithSum(mm1, mm2));
         }
@@ -1393,10 +1584,10 @@ public class StatsUtil {
         return ret;
     }
 
-    private static Map mergeWithSum(Map m1, Map m2) {
-        Map ret = new HashMap();
+    private static <K> Map<K, Number> mergeWithSum(Map<K, ?> m1, Map<K, ?> m2) {
+        Map<K, Number> ret = new HashMap<>();
 
-        Set<Object> allKeys = new HashSet<>();
+        Set<K> allKeys = new HashSet<>();
         if (m1 != null) {
             allKeys.addAll(m1.keySet());
         }
@@ -1404,10 +1595,52 @@ public class StatsUtil {
             allKeys.addAll(m2.keySet());
         }
 
-        for (Object k : allKeys) {
+        for (K k : allKeys) {
             Number n1 = getOr0(m1, k);
             Number n2 = getOr0(m2, k);
-            ret.put(k, add(n1, n2));
+            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 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 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;
     }
@@ -1416,10 +1649,11 @@ 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
      */
-    private static Map mergeWithAddPair(Map m1, Map m2) {
-        Map ret = new HashMap();
+    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<Object> allKeys = new HashSet<>();
+        Set<String> allKeys = new HashSet<>();
         if (m1 != null) {
             allKeys.addAll(m1.keySet());
         }
@@ -1427,9 +1661,9 @@ public class StatsUtil {
             allKeys.addAll(m2.keySet());
         }
 
-        for (Object k : allKeys) {
-            Map mm1 = (m1 != null) ? (Map) m1.get(k) : null;
-            Map mm2 = (m2 != null) ? (Map) m2.get(k) : null;
+        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) {
@@ -1437,13 +1671,17 @@ public class StatsUtil {
             } else if (mm2 == null) {
                 ret.put(k, mm1);
             } else {
-                Map tmp = new HashMap();
-                for (Object kk : mm1.keySet()) {
-                    List seq1 = (List) mm1.get(kk);
-                    List seq2 = (List) mm2.get(kk);
+                Map<K, List> tmp = new HashMap<>();
+                for (K kk : mm1.keySet()) {
+                    List seq1 = mm1.get(kk);
+                    List seq2 = mm2.get(kk);
                     List sums = new ArrayList();
                     for (int i = 0; i < seq1.size(); i++) {
-                        sums.add(add((Number) seq1.get(i), (Number) seq2.get(i)));
+                        if (seq1.get(i) instanceof Long) {
+                            sums.add(((Number) seq1.get(i)).longValue() + ((Number) seq2.get(i)).longValue());
+                        } else {
+                            sums.add(((Number) seq1.get(i)).doubleValue() + ((Number) seq2.get(i)).doubleValue());
+                        }
                     }
                     tmp.put(kk, sums);
                 }
@@ -1457,65 +1695,36 @@ public class StatsUtil {
     // thriftify stats methods
     // =====================================================================================
 
-    private static TopologyPageInfo thriftifyTopoPageData(String topologyId, Map data) {
-        TopologyPageInfo ret = new TopologyPageInfo(topologyId);
-        Integer numTasks = getByKeywordOr0(data, NUM_TASKS).intValue();
-        Integer numWorkers = getByKeywordOr0(data, NUM_WORKERS).intValue();
-        Integer numExecutors = getByKeywordOr0(data, NUM_EXECUTORS).intValue();
-        Map spout2stats = getMapByKey(data, SPOUT_TO_STATS);
-        Map bolt2stats = getMapByKey(data, BOLT_TO_STATS);
-        Map win2emitted = getMapByKey(data, WIN_TO_EMITTED);
-        Map win2transferred = getMapByKey(data, WIN_TO_TRANSFERRED);
-        Map win2compLatency = getMapByKey(data, WIN_TO_COMP_LAT);
-        Map win2acked = getMapByKey(data, WIN_TO_ACKED);
-        Map win2failed = getMapByKey(data, WIN_TO_FAILED);
-
-        Map<String, ComponentAggregateStats> spoutAggStats = new HashMap<>();
-        for (Object o : spout2stats.entrySet()) {
-            Map.Entry e = (Map.Entry) o;
-            String id = (String) e.getKey();
-            Map v = (Map) e.getValue();
-            putKV(v, TYPE, KW_SPOUT);
-
-            spoutAggStats.put(id, thriftifySpoutAggStats(v));
-        }
+    public static ClusterWorkerHeartbeat thriftifyZkWorkerH

<TRUNCATED>

[20/30] storm git commit: upmerge from master

Posted by bo...@apache.org.
upmerge from master


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

Branch: refs/heads/master
Commit: 39ea23cdbbf64b3844205ce759701ec78d688c4a
Parents: 4e0ff2f 96f81d7
Author: 卫乐 <we...@taobao.com>
Authored: Sat Mar 5 20:02:42 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Sat Mar 5 20:02:42 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   7 +
 .../src/clj/org/apache/storm/MockAutoCred.clj   |  58 --
 .../clj/org/apache/storm/daemon/supervisor.clj  |  15 +
 .../storm/cluster/StormClusterStateImpl.java    |   7 +-
 .../storm/daemon/metrics/MetricsUtils.java      |   2 +-
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |   2 +
 .../apache/storm/security/auth/AuthUtils.java   |  40 +
 .../storm/security/auth/kerberos/AutoTGT.java   |  64 +-
 .../auth/kerberos/AutoTGTKrb5LoginModule.java   |   8 +-
 .../apache/storm/stats/BoltExecutorStats.java   |   6 +-
 .../jvm/org/apache/storm/stats/CommonStats.java |  12 +-
 .../apache/storm/stats/SpoutExecutorStats.java  |   5 +-
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 781 ++++++++++---------
 .../jvm/org/apache/storm/utils/ConfigUtils.java |  10 +
 .../test/clj/org/apache/storm/nimbus_test.clj   |  10 +-
 .../security/auth/auto_login_module_test.clj    |  24 +-
 .../clj/org/apache/storm/supervisor_test.clj    |   6 +
 .../test/jvm/org/apache/storm/MockAutoCred.java |  75 ++
 18 files changed, 623 insertions(+), 509 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/39ea23cd/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
index f6dad09,0000000..d8c7f06
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
@@@ -1,118 -1,0 +1,114 @@@
 +/**
 + * 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 com.google.common.collect.Lists;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
- import org.apache.storm.generated.BoltStats;
- import org.apache.storm.generated.ExecutorSpecificStats;
- import org.apache.storm.generated.ExecutorStats;
- import org.apache.storm.generated.SpoutStats;
 +import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 +import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 +
 +@SuppressWarnings("unchecked")
 +public class BoltExecutorStats extends CommonStats {
 +
 +    public static final String ACKED = "acked";
 +    public static final String FAILED = "failed";
 +    public static final String EXECUTED = "executed";
 +    public static final String PROCESS_LATENCIES = "process-latencies";
 +    public static final String EXECUTE_LATENCIES = "execute-latencies";
 +
 +    public static final String[] BOLT_FIELDS = {ACKED, FAILED, EXECUTED, PROCESS_LATENCIES, EXECUTE_LATENCIES};
 +
 +    public BoltExecutorStats(int rate) {
 +        super(rate);
 +
 +        this.put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
 +        this.put(FAILED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
 +        this.put(EXECUTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
 +        this.put(PROCESS_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
 +        this.put(EXECUTE_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
 +    }
 +
 +    public MultiCountStatAndMetric getAcked() {
 +        return (MultiCountStatAndMetric) this.get(ACKED);
 +    }
 +
 +    public MultiCountStatAndMetric getFailed() {
 +        return (MultiCountStatAndMetric) this.get(FAILED);
 +    }
 +
 +    public MultiCountStatAndMetric getExecuted() {
 +        return (MultiCountStatAndMetric) this.get(EXECUTED);
 +    }
 +
 +    public MultiLatencyStatAndMetric getProcessLatencies() {
 +        return (MultiLatencyStatAndMetric) this.get(PROCESS_LATENCIES);
 +    }
 +
 +    public MultiLatencyStatAndMetric getExecuteLatencies() {
 +        return (MultiLatencyStatAndMetric) this.get(EXECUTE_LATENCIES);
 +    }
 +
 +    public void boltExecuteTuple(String component, String stream, long latencyMs) {
 +        List key = Lists.newArrayList(component, stream);
 +        this.getExecuted().incBy(key, this.rate);
 +        this.getExecuteLatencies().record(key, latencyMs);
 +    }
 +
 +    public void boltAckedTuple(String component, String stream, long latencyMs) {
 +        List key = Lists.newArrayList(component, stream);
 +        this.getAcked().incBy(key, this.rate);
 +        this.getProcessLatencies().record(key, latencyMs);
 +    }
 +
 +    public void boltFailedTuple(String component, String stream, long latencyMs) {
 +        List key = Lists.newArrayList(component, stream);
 +        this.getFailed().incBy(key, this.rate);
 +
 +    }
 +
 +    public Map renderStats() {
 +        cleanupStats();
 +        Map ret = new HashMap();
 +        ret.putAll(valueStats(CommonStats.COMMON_FIELDS));
 +        ret.putAll(valueStats(BoltExecutorStats.BOLT_FIELDS));
-         StatsUtil.putRawKV(ret, StatsUtil.TYPE, StatsUtil.KW_BOLT);
++        StatsUtil.putKV(ret, StatsUtil.TYPE, StatsUtil.KW_BOLT);
 +
 +        return ret;
 +    }
 +
 +//    public ExecutorStats renderStats() {
 +//        cleanupStats();
 +//
 +//        ExecutorStats ret = new ExecutorStats();
 +//        ret.set_emitted(valueStat(EMITTED));
 +//        ret.set_transferred(valueStat(TRANSFERRED));
 +//        ret.set_rate(this.rate);
 +//
 +//        BoltStats boltStats = new BoltStats(
 +//                StatsUtil.windowSetConverter(valueStat(ACKED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
 +//                StatsUtil.windowSetConverter(valueStat(FAILED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
 +//                StatsUtil.windowSetConverter(valueStat(PROCESS_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
 +//                StatsUtil.windowSetConverter(valueStat(EXECUTED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
 +//                StatsUtil.windowSetConverter(valueStat(EXECUTE_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY));
 +//        ret.set_specific(ExecutorSpecificStats.bolt(boltStats));
 +//
 +//        return ret;
 +//    }
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/39ea23cd/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
index e386413,0000000..f7826f9
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
@@@ -1,112 -1,0 +1,112 @@@
 +/**
 + * 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.Map;
 +import org.apache.storm.metric.api.IMetric;
 +import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 +import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 +
 +@SuppressWarnings("unchecked")
 +public class CommonStats {
 +    public static final int NUM_STAT_BUCKETS = 20;
 +
 +    public static final String RATE = "rate";
 +
 +    public static final String EMITTED = "emitted";
 +    public static final String TRANSFERRED = "transferred";
 +    public static final String[] COMMON_FIELDS = {EMITTED, TRANSFERRED};
 +
 +    protected final int rate;
-     protected final Map metricMap = new HashMap();
++    protected final Map<String, IMetric> metricMap = new HashMap<>();
 +
 +    public CommonStats(int rate) {
 +        this.rate = rate;
 +        this.put(EMITTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
 +        this.put(TRANSFERRED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
 +    }
 +
 +    public int getRate() {
 +        return this.rate;
 +    }
 +
 +    public MultiCountStatAndMetric getEmitted() {
 +        return (MultiCountStatAndMetric) get(EMITTED);
 +    }
 +
 +    public MultiCountStatAndMetric getTransferred() {
 +        return (MultiCountStatAndMetric) get(TRANSFERRED);
 +    }
 +
 +    public IMetric get(String field) {
-         return (IMetric) StatsUtil.getByKeyword(metricMap, field);
++        return (IMetric) StatsUtil.getByKey(metricMap, field);
 +    }
 +
 +    protected void put(String field, Object value) {
-         StatsUtil.putRawKV(metricMap, field, value);
++        StatsUtil.putKV(metricMap, field, value);
 +    }
 +
 +    public void emittedTuple(String stream) {
 +        this.getEmitted().incBy(stream, this.rate);
 +    }
 +
 +    public void transferredTuples(String stream, int amount) {
 +        this.getTransferred().incBy(stream, this.rate * amount);
 +    }
 +
 +    public void cleanupStats() {
 +        for (Object imetric : this.metricMap.values()) {
 +            cleanupStat((IMetric) imetric);
 +        }
 +    }
 +
 +    private void cleanupStat(IMetric metric) {
 +        if (metric instanceof MultiCountStatAndMetric) {
 +            ((MultiCountStatAndMetric) metric).close();
 +        } else if (metric instanceof MultiLatencyStatAndMetric) {
 +            ((MultiLatencyStatAndMetric) metric).close();
 +        }
 +    }
 +
 +    protected Map valueStats(String[] fields) {
 +        Map ret = new HashMap();
 +        for (String field : fields) {
 +            IMetric metric = this.get(field);
 +            if (metric instanceof MultiCountStatAndMetric) {
-                 StatsUtil.putRawKV(ret, field, ((MultiCountStatAndMetric) metric).getTimeCounts());
++                StatsUtil.putKV(ret, field, ((MultiCountStatAndMetric) metric).getTimeCounts());
 +            } else if (metric instanceof MultiLatencyStatAndMetric) {
-                 StatsUtil.putRawKV(ret, field, ((MultiLatencyStatAndMetric) metric).getTimeLatAvg());
++                StatsUtil.putKV(ret, field, ((MultiLatencyStatAndMetric) metric).getTimeLatAvg());
 +            }
 +        }
-         StatsUtil.putRawKV(ret, CommonStats.RATE, this.getRate());
++        StatsUtil.putKV(ret, CommonStats.RATE, this.getRate());
 +
 +        return ret;
 +    }
 +
 +    protected Map valueStat(String field) {
 +        IMetric metric = this.get(field);
 +        if (metric instanceof MultiCountStatAndMetric) {
 +            return ((MultiCountStatAndMetric) metric).getTimeCounts();
 +        } else if (metric instanceof MultiLatencyStatAndMetric) {
 +            return ((MultiLatencyStatAndMetric) metric).getTimeLatAvg();
 +        }
 +        return null;
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/39ea23cd/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
index 918ae06,0000000..27c626e
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
@@@ -1,89 -1,0 +1,86 @@@
 +/**
 + * 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.Map;
- import org.apache.storm.generated.ExecutorSpecificStats;
- import org.apache.storm.generated.ExecutorStats;
- import org.apache.storm.generated.SpoutStats;
 +import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 +import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 +
 +@SuppressWarnings("unchecked")
 +public class SpoutExecutorStats extends CommonStats {
 +
 +    public static final String ACKED = "acked";
 +    public static final String FAILED = "failed";
 +    public static final String COMPLETE_LATENCIES = "complete-latencies";
 +
 +    public static final String[] SPOUT_FIELDS = {ACKED, FAILED, COMPLETE_LATENCIES};
 +
 +    public SpoutExecutorStats(int rate) {
 +        super(rate);
 +        this.put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
 +        this.put(FAILED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
 +        this.put(COMPLETE_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
 +    }
 +
 +    public MultiCountStatAndMetric getAcked() {
 +        return (MultiCountStatAndMetric) this.get(ACKED);
 +    }
 +
 +    public MultiCountStatAndMetric getFailed() {
 +        return (MultiCountStatAndMetric) this.get(FAILED);
 +    }
 +
 +    public MultiLatencyStatAndMetric getCompleteLatencies() {
 +        return (MultiLatencyStatAndMetric) this.get(COMPLETE_LATENCIES);
 +    }
 +
 +    public void spoutAckedTuple(String stream, long latencyMs) {
 +        this.getAcked().incBy(stream, this.rate);
 +        this.getCompleteLatencies().record(stream, latencyMs);
 +    }
 +
 +    public void spoutFailedTuple(String stream, long latencyMs) {
 +        this.getFailed().incBy(stream, this.rate);
 +    }
 +
 +    public Map renderStats() {
 +        cleanupStats();
 +        Map ret = new HashMap();
 +        ret.putAll(valueStats(CommonStats.COMMON_FIELDS));
 +        ret.putAll(valueStats(SpoutExecutorStats.SPOUT_FIELDS));
-         StatsUtil.putRawKV(ret, StatsUtil.TYPE, StatsUtil.KW_SPOUT);
++        StatsUtil.putKV(ret, StatsUtil.TYPE, StatsUtil.KW_SPOUT);
 +
 +        return ret;
 +    }
 +
 +//    public ExecutorStats renderStats() {
 +//        cleanupStats();
 +//
 +//        ExecutorStats ret = new ExecutorStats();
 +//        ret.set_emitted(valueStat(EMITTED));
 +//        ret.set_transferred(valueStat(TRANSFERRED));
 +//        ret.set_rate(this.rate);
 +//
 +//        SpoutStats spoutStats = new SpoutStats(
 +//                valueStat(ACKED), valueStat(FAILED), valueStat(COMPLETE_LATENCIES));
 +//        ret.set_specific(ExecutorSpecificStats.spout(spoutStats));
 +//
 +//        return ret;
 +//    }
 +}


[22/30] storm git commit: 1. changed heartbeat structure to java HashMap 2. use HashMaps in StatsUtil instead of clojure map 3. changed tests accordingly

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/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 904d0db..1f708cb 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -23,7 +23,7 @@
            [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
            [org.apache.storm.generated GlobalStreamId]
            [org.apache.storm Thrift MockAutoCred]
-           [org.apache.storm.stats BoltExecutorStats])
+           [org.apache.storm.stats BoltExecutorStats StatsUtil])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
   (:import [org.apache.storm.scheduler INimbus])
   (:import [org.mockito Mockito])
@@ -141,12 +141,17 @@
   (let [state (:storm-cluster-state cluster)
         executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
         [node port] (get executor->node+port executor)
-        curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
-        stats (:executor-stats curr-beat)]
+        curr-beat (StatsUtil/convertZkWorkerHb (.getWorkerHeartbeat state storm-id node port))
+        stats (if (get curr-beat "executor-stats")
+                (get curr-beat "executor-stats")
+                (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)))
+    (log-warn "merged:" stats)
+
     (.workerHeartbeat state storm-id node port
-      (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10
-                               :executor-stats (merge stats {executor (clojurify-structure (.renderStats (BoltExecutorStats. 20)))})})
-      )))
+      (StatsUtil/thriftifyZkWorkerHb (StatsUtil/mkZkWorkerHb storm-id stats (int 10))))))
 
 (defn slot-assignments [cluster storm-id]
   (let [state (:storm-cluster-state cluster)


[27/30] storm git commit: added method comments

Posted by bo...@apache.org.
added method comments


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

Branch: refs/heads/master
Commit: 7b354287227de358cc357ac45c18ac2b1a679202
Parents: 5bd5bd7
Author: 卫乐 <we...@taobao.com>
Authored: Wed Mar 9 13:05:36 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Wed Mar 9 13:05:36 2016 +0800

----------------------------------------------------------------------
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 313 ++++++++++++++-----
 1 file changed, 231 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7b354287/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index 7650ab1..aa1b234 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -144,9 +144,10 @@ public class StatsUtil {
     }
 
     /**
-     * Aggregates number acked and complete latencies across all streams.
+     * aggregate number acked and complete latencies across all streams.
      */
-    public static Map<String, Number> aggSpoutLatAndCount(Map<String, Double> id2compAvg, Map<String, Long> id2numAcked) {
+    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));
@@ -155,15 +156,17 @@ public class StatsUtil {
     }
 
     /**
-     * Aggregates number executed and process & execute latencies.
+     * aggregate number executed and process & execute latencies.
      */
-    public static Map aggBoltStreamsLatAndCount(Map id2execAvg, Map id2procAvg, Map id2numExec) {
-        Map ret = new HashMap();
+    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 (Object k : id2execAvg.keySet()) {
-            Map subMap = new HashMap();
+        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));
@@ -175,12 +178,13 @@ public class StatsUtil {
     /**
      * Aggregates number acked and complete latencies.
      */
-    public static Map aggSpoutStreamsLatAndCount(Map id2compAvg, Map id2acked) {
-        Map ret = new HashMap();
+    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 (Object k : id2compAvg.keySet()) {
+        for (K k : id2compAvg.keySet()) {
             Map subMap = new HashMap();
             putKV(subMap, COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
             putKV(subMap, ACKED, id2acked.get(k));
@@ -189,17 +193,29 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map aggPreMergeCompPageBolt(Map<String, Object> m, String window, boolean includeSys) {
-        Map ret = new HashMap();
-        putKV(ret, EXECUTOR_ID, getByKey(m, "exec-id"));
-        putKV(ret, HOST, getByKey(m, HOST));
-        putKV(ret, PORT, getByKey(m, PORT));
-        putKV(ret, UPTIME, getByKey(m, UPTIME));
+    /**
+     * 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(m, NUM_TASKS));
+        putKV(ret, NUM_TASKS, getByKey(beat, NUM_TASKS));
 
-        Map stat2win2sid2num = getMapByKey(m, STATS);
-        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(m, UPTIME).intValue()));
+        Map stat2win2sid2num = getMapByKey(beat, STATS);
+        putKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(beat, UPTIME).intValue()));
 
         // calc cid+sid->input_stats
         Map inputStats = new HashMap();
@@ -236,16 +252,27 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map<String, Object> aggPreMergeCompPageSpout(Map<String, Object> m, String window, boolean includeSys) {
+    /**
+     * 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(m, "exec-id"));
-        putKV(ret, HOST, getByKey(m, HOST));
-        putKV(ret, PORT, getByKey(m, PORT));
-        putKV(ret, UPTIME, getByKey(m, UPTIME));
+        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(m, NUM_TASKS));
+        putKV(ret, NUM_TASKS, getByKey(beat, NUM_TASKS));
 
-        Map stat2win2sid2num = getMapByKey(m, STATS);
+        Map stat2win2sid2num = getMapByKey(beat, STATS);
 
         // calc sid->output-stats
         Map outputStats = new HashMap();
@@ -269,16 +296,24 @@ public class StatsUtil {
         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> m, String window, boolean includeSys) {
+            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(m, NUM_TASKS));
+        putKV(subRet, NUM_TASKS, getByKey(beat, NUM_TASKS));
 
-        Map<String, Object> stat2win2sid2num = getMapByKey(m, STATS);
-        putKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeyOr0(m, UPTIME).intValue()));
+        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);
@@ -304,12 +339,12 @@ public class StatsUtil {
         subRet.putAll(aggBoltLatAndCount(
                 win2sid2execLat.get(window), win2sid2procLat.get(window), win2sid2exec.get(window)));
 
-        ret.put((String) getByKey(m, "comp-id"), subRet);
+        ret.put((String) getByKey(beat, "comp-id"), subRet);
         return ret;
     }
 
     /**
-     * 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) {
@@ -346,6 +381,13 @@ public class StatsUtil {
         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<>();
@@ -395,6 +437,9 @@ public class StatsUtil {
         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<>();
@@ -432,7 +477,15 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map<String, Object> mergeAggCompStatsTopoPageBolt(Map<String, Object> accBoltStats, Map<String, Object> boltStats) {
+    /**
+     * 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();
@@ -459,7 +512,11 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map<String, Object> mergeAggCompStatsTopoPageSpout(Map<String, Object> accSpoutStats, Map<String, Object> spoutStats) {
+    /**
+     * 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();
@@ -485,7 +542,7 @@ public class StatsUtil {
      * 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> newData, String compType) {
+            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);
@@ -501,12 +558,12 @@ public class StatsUtil {
         // component id -> stats
         Map<String, Object> cid2stats;
         if (isSpout) {
-            cid2stats = aggPreMergeTopoPageSpout(newData, window, includeSys);
+            cid2stats = aggPreMergeTopoPageSpout(beat, window, includeSys);
         } else {
-            cid2stats = aggPreMergeTopoPageBolt(newData, window, includeSys);
+            cid2stats = aggPreMergeTopoPageBolt(beat, window, includeSys);
         }
 
-        Map stats = getMapByKey(newData, STATS);
+        Map stats = getMapByKey(beat, STATS);
         Map w2compLatWgtAvg, w2acked;
         Map compLatStats = getMapByKey(stats, COMP_LATENCIES);
         if (isSpout) { // agg spout stats
@@ -524,7 +581,7 @@ public class StatsUtil {
             w2acked = aggregateCountStreams(getMapByKey(stats, ACKED));
         }
 
-        workerSet.add(Lists.newArrayList(getByKey(newData, HOST), getByKey(newData, PORT)));
+        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);
@@ -543,23 +600,23 @@ public class StatsUtil {
         // (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) {
-            Set<String> keySet = new HashSet<>();
-            keySet.addAll(spout2stats.keySet());
-            keySet.addAll(cid2stats.keySet());
+            Set<String> spouts = new HashSet<>();
+            spouts.addAll(spout2stats.keySet());
+            spouts.addAll(cid2stats.keySet());
 
-            Map mm = new HashMap();
-            for (String k : keySet) {
-                mm.put(k, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(k), (Map) cid2stats.get(k)));
+            Map<String, Object> mm = new HashMap<>();
+            for (String spout : spouts) {
+                mm.put(spout, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(spout), (Map) cid2stats.get(spout)));
             }
             putKV(ret, SPOUT_TO_STATS, mm);
         } else {
-            Set<String> keySet = new HashSet<>();
-            keySet.addAll(bolt2stats.keySet());
-            keySet.addAll(cid2stats.keySet());
+            Set<String> bolts = new HashSet<>();
+            bolts.addAll(bolt2stats.keySet());
+            bolts.addAll(cid2stats.keySet());
 
-            Map mm = new HashMap();
-            for (String k : keySet) {
-                mm.put(k, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(k), (Map) cid2stats.get(k)));
+            Map<String, Object> mm = new HashMap<>();
+            for (String bolt : bolts) {
+                mm.put(bolt, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(bolt), (Map) cid2stats.get(bolt)));
             }
             putKV(ret, BOLT_TO_STATS, mm);
         }
@@ -674,12 +731,13 @@ public class StatsUtil {
      *
      * @param statsSeq   a seq of ExecutorStats
      * @param includeSys whether to include system streams
-     * @return aggregated bolt stats
+     * @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<>();
@@ -710,13 +768,14 @@ public class StatsUtil {
      *
      * @param statsSeq   a seq of ExecutorStats
      * @param includeSys whether to include system streams
-     * @return aggregated spout stats
+     * @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<>();
@@ -736,6 +795,9 @@ public class StatsUtil {
         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<>();
 
@@ -751,6 +813,9 @@ public class StatsUtil {
         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);
@@ -762,6 +827,12 @@ public class StatsUtil {
         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<>();
@@ -776,6 +847,14 @@ public class StatsUtil {
         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<>();
@@ -796,8 +875,15 @@ public class StatsUtil {
         return ret;
     }
 
-    public static <K> Map<String, Double> aggregateAvgStreams(
-            Map<String, Map<K, Double>> avgs, Map<String, Map<K, Long>> counts) {
+    /**
+     * 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);
@@ -818,14 +904,21 @@ public class StatsUtil {
         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<>();
@@ -834,6 +927,12 @@ public class StatsUtil {
         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<>();
@@ -846,6 +945,12 @@ public class StatsUtil {
         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)));
@@ -861,7 +966,7 @@ public class StatsUtil {
     }
 
     /**
-     * A helper function that aggregates windowed stats from one spout executor.
+     * 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) {
@@ -905,7 +1010,7 @@ public class StatsUtil {
     }
 
     /**
-     * A helper function that aggregates windowed stats from one spout executor.
+     * 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) {
@@ -944,7 +1049,7 @@ public class StatsUtil {
 
 
     /**
-     * aggregate counts
+     * aggregate a list of count maps into one map
      *
      * @param countsSeq a seq of {win -> GlobalStreamId -> value}
      */
@@ -973,8 +1078,8 @@ public class StatsUtil {
         return ret;
     }
 
-    public static Map<String, Object> aggregateCompStats(String window, boolean includeSys,
-                                                         List<Map<String, Object>> beats, String compType) {
+    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<>();
@@ -998,6 +1103,7 @@ public class StatsUtil {
         }
         putKV(initVal, STATS, stats);
 
+        // iterate through all executor heartbeats
         for (Map<String, Object> beat : beats) {
             initVal = aggCompExecStats(window, includeSys, initVal, beat, compType);
         }
@@ -1029,14 +1135,14 @@ public class StatsUtil {
     }
 
     /**
-     * post aggregate component stats
+     * 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 task2component task -> component, note it's a clojure map
-     * @param exec2hostPort  executor -> host+port, note it's a clojure map
-     * @param compStats      accumulated comp stats
-     * @return
+     * @param compStats accumulated comp stats
      */
-    public static Map<String, Object> postAggregateCompStats(Map task2component, Map exec2hostPort, Map<String, Object> compStats) {
+    public static Map<String, Object> postAggregateCompStats(Map<String, Object> compStats) {
         Map<String, Object> ret = new HashMap<>();
 
         String compType = (String) compStats.get(TYPE);
@@ -1108,6 +1214,19 @@ public class StatsUtil {
         return ret;
     }
 
+    /**
+     * aggregate component executor stats
+     *
+     * @param exec2hostPort  a Map of {executor -> host+port}, note it's a clojure map
+     * @param task2component a Map of {task id -> component}, note it's a clojure map
+     * @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) {
@@ -1115,7 +1234,7 @@ public class StatsUtil {
         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(task2component, exec2hostPort, compStats);
+        compStats = postAggregateCompStats(compStats);
         return thriftifyCompPageData(topologyId, topology, componentId, compStats);
     }
 
@@ -1124,6 +1243,9 @@ public class StatsUtil {
     // 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()) {
@@ -1150,6 +1272,12 @@ public class StatsUtil {
         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) {
@@ -1224,6 +1352,15 @@ public class StatsUtil {
         return ret;
     }
 
+    /**
+     * extract a list of host port info for specified component
+     *
+     * @param exec2hostPort  {executor -> host+port}, note it's a clojure map
+     * @param task2component {task id -> component}, note it's a clojure map
+     * @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 exec2hostPort, Map task2component, boolean includeSys, String compId) {
         List<Map<String, Object>> ret = new ArrayList<>();
@@ -1384,6 +1521,14 @@ public class StatsUtil {
         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<>();
@@ -1400,6 +1545,9 @@ public class StatsUtil {
     }
 
 
+    /**
+     * convert a list of clojure executors to a java Set of List<Integer>
+     */
     public static Set<List<Integer>> convertExecutors(Set executors) {
         Set<List<Integer>> convertedExecutors = new HashSet<>();
         for (Object executor : executors) {
@@ -1438,7 +1586,7 @@ public class StatsUtil {
         if (stats == null) {
             return 0.0;
         } else {
-            // Map<String, Map<String/GlobalStreamId, Long/Double>> {win -> stream -> value}
+            // 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));
@@ -1495,17 +1643,15 @@ public class StatsUtil {
         return sum;
     }
 
-    private static double sumStreamsDouble(Map m, String key) {
+    private static <K1, K2> double sumStreamsDouble(Map<K1, Map<K2, ?>> m, String key) {
         double sum = 0;
         if (m == null) {
             return sum;
         }
-        for (Object v : m.values()) {
-            Map sub = (Map) v;
-            for (Object o : sub.entrySet()) {
-                Map.Entry e = (Map.Entry) o;
-                if (e.getKey().equals(key)) {
-                    sum += ((Number) e.getValue()).doubleValue();
+        for (Map<K2, ?> v : m.values()) {
+            for (Map.Entry<K2, ?> entry : v.entrySet()) {
+                if (entry.getKey().equals(key)) {
+                    sum += ((Number) entry.getValue()).doubleValue();
                 }
             }
         }
@@ -1607,7 +1753,7 @@ public class StatsUtil {
         return ret;
     }
 
-    private static <K> Map mergeWithSumLong(Map<K, Long> m1, Map<K, Long> m2) {
+    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<>();
@@ -1626,7 +1772,7 @@ public class StatsUtil {
         return ret;
     }
 
-    private static <K> Map mergeWithSumDouble(Map<K, Double> m1, Map<K, Double> m2) {
+    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<>();
@@ -2042,14 +2188,12 @@ public class StatsUtil {
 
         for (Map.Entry<T, V1> entry : id2Avg.entrySet()) {
             T k = entry.getKey();
-            double v = entry.getValue().doubleValue();
-            long n = id2num.get(k).longValue();
-            ret += productOr0(v, n);
+            ret += productOr0(entry.getValue(), id2num.get(k));
         }
         return ret;
     }
 
-    private static double weightAvg(Map id2Avg, Map id2num, Object key) {
+    private static <K, V1 extends Number, V2 extends Number> double weightAvg(Map<K, V1> id2Avg, Map<K, V2> id2num, K key) {
         if (id2Avg == null || id2num == null) {
             return 0.0;
         }
@@ -2087,7 +2231,7 @@ public class StatsUtil {
         return (Map) map.get(key);
     }
 
-    private static <T, V extends Number> long sumValues(Map<T, V> m) {
+    private static <K, V extends Number> long sumValues(Map<K, V> m) {
         long ret = 0L;
         if (m == null) {
             return ret;
@@ -2223,6 +2367,11 @@ public class StatsUtil {
         return stormClusterState.lastError(stormId, compId);
     }
 
+
+    // =====================================================================================
+    // key transformers
+    // =====================================================================================
+
     interface KeyTransformer<T> {
         T transform(Object key);
     }


[26/30] storm git commit: resolve conflict

Posted by bo...@apache.org.
resolve conflict


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

Branch: refs/heads/master
Commit: 5bd5bd7605d8bc74fb72d47bf44b3e5fadf3942e
Parents: 000fcb8
Author: 卫乐 <we...@taobao.com>
Authored: Tue Mar 8 20:57:13 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Tue Mar 8 20:57:13 2016 +0800

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/nimbus.clj | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5bd5bd76/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index f2e60bf..23ab435 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -2121,14 +2121,14 @@
          ^String component-id
          ^String window
          ^boolean include-sys?]
-        (mark! nimbus:num-getComponentPageInfo-calls)
+        (.mark nimbus:num-getComponentPageInfo-calls)
         (let [info (get-common-topo-info topo-id "getComponentPageInfo")
               {:keys [executor->node+port node->host]} (:assignment info)
               ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
               executor->host+port (map-val (fn [[node port]]
                                              [(node->host node) port])
                                            executor->node+port)
-              comp-page-info (stats/agg-comp-execs-stats executor->host+port
+              comp-page-info (StatsUtil/aggCompExecsStats executor->host+port
                                                          (:task->component info)
                                                          (:beats info)
                                                          window


[25/30] storm git commit: upmerge from master

Posted by bo...@apache.org.
upmerge from master


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

Branch: refs/heads/master
Commit: 000fcb86592772c6e51962c565c497c8b052aa47
Parents: 4c246d1 b477939
Author: 卫乐 <we...@taobao.com>
Authored: Tue Mar 8 20:51:34 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Tue Mar 8 20:51:34 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   4 +
 .../travis/print-errors-from-test-reports.py    |   4 +
 .../src/clj/org/apache/storm/daemon/common.clj  |  13 +-
 .../src/clj/org/apache/storm/daemon/drpc.clj    |  30 +--
 .../clj/org/apache/storm/daemon/logviewer.clj   |  27 ++-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  | 189 +++++++++----------
 .../clj/org/apache/storm/daemon/supervisor.clj  |  12 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |  81 ++++----
 .../src/clj/org/apache/storm/ui/helpers.clj     |  10 +-
 .../storm/metric/StormMetricsRegistry.java      |  84 +++++++++
 .../auth/AbstractSaslClientCallbackHandler.java |  76 ++++++++
 .../auth/AbstractSaslServerCallbackHandler.java |  94 +++++++++
 .../auth/digest/ClientCallbackHandler.java      |  60 +-----
 .../auth/digest/ServerCallbackHandler.java      |  61 +-----
 .../auth/plain/PlainClientCallbackHandler.java  |  31 +++
 .../auth/plain/PlainSaslTransportPlugin.java    |  71 +++++++
 .../auth/plain/PlainServerCallbackHandler.java  |  55 ++++++
 .../security/auth/plain/SaslPlainServer.java    | 158 ++++++++++++++++
 18 files changed, 763 insertions(+), 297 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/000fcb86/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 997f92c,0af12a2..f2e60bf
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -15,7 -15,7 +15,8 @@@
  ;; limitations under the License.
  (ns org.apache.storm.daemon.nimbus
    (:import [org.apache.thrift.server THsHaServer THsHaServer$Args]
-            [org.apache.storm.stats StatsUtil])
++           [org.apache.storm.stats StatsUtil]
+            [org.apache.storm.metric StormMetricsRegistry])
    (:import [org.apache.storm.generated KeyNotFoundException])
    (:import [org.apache.storm.blobstore LocalFsBlobStore])
    (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
@@@ -559,7 -558,36 +558,6 @@@
                        executor->component
                        (:launch-time-secs storm-base))))
  
 -;; Does not assume that clocks are synchronized. Executor heartbeat is only used so that
 -;; nimbus knows when it's received a new heartbeat. All timing is done by nimbus and
 -;; tracked through heartbeat-cache
 -(defn- update-executor-cache [curr hb timeout]
 -  (let [reported-time (:time-secs hb)
 -        {last-nimbus-time :nimbus-time
 -         last-reported-time :executor-reported-time} curr
 -        reported-time (cond reported-time reported-time
 -                            last-reported-time last-reported-time
 -                            :else 0)
 -        nimbus-time (if (or (not last-nimbus-time)
 -                        (not= last-reported-time reported-time))
 -                      (Time/currentTimeSecs)
 -                      last-nimbus-time
 -                      )]
 -      {:is-timed-out (and
 -                       nimbus-time
 -                       (>= (Time/deltaSecs nimbus-time) timeout))
 -       :nimbus-time nimbus-time
 -       :executor-reported-time reported-time
 -       :heartbeat hb}))
 -
 -(defn update-heartbeat-cache [cache executor-beats all-executors timeout]
 -  (let [cache (select-keys cache all-executors)]
 -    (into {}
 -      (for [executor all-executors :let [curr (cache executor)]]
 -        [executor
 -         (update-executor-cache curr (get executor-beats executor) timeout)]
 -         ))))
--
  (defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
    (log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
    (let [storm-cluster-state (:storm-cluster-state nimbus)
@@@ -1455,11 -1486,11 +1453,11 @@@
        (fn []
          (renew-credentials nimbus)))
  
-     (defgauge nimbus:num-supervisors
-       (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))
- 
-     (start-metrics-reporters conf)
+     (def nimbus:num-supervisors (StormMetricsRegistry/registerGauge "nimbus:num-supervisors"
+       (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil)))))
  
+     (StormMetricsRegistry/startMetricsReporters conf)
 -
++    
      (reify Nimbus$Iface
        (^void submitTopologyWithOpts
          [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology
@@@ -1887,12 -1918,11 +1885,11 @@@
                            (map (fn [c] [c (errors-fn storm-cluster-state storm-id c)]))
                            (into {}))
                executor-summaries (dofor [[executor [node port]] (:executor->node+port assignment)]
-                                      (let [host (-> assignment :node->host (get node))
-                                               heartbeat (.get beats (StatsUtil/convertExecutor executor))
-                                               excutorstats (.get (.get heartbeat "heartbeat") "stats")
 -                                        (let [host (-> assignment :node->host (get node))
 -                                              heartbeat (get beats executor)
 -                                              excutorstats (:stats heartbeat)
--                                              excutorstats (if excutorstats
-                                                       (StatsUtil/thriftifyExecutorStats excutorstats))]
-                                               
 -                                                      (stats/thriftify-executor-stats excutorstats))]
++                                   (let [host (-> assignment :node->host (get node))
++                                            heartbeat (.get beats (StatsUtil/convertExecutor executor))
++                                            excutorstats (.get (.get heartbeat "heartbeat") "stats")
++                                            excutorstats (if excutorstats
++                                                    (StatsUtil/thriftifyExecutorStats excutorstats))]
                                            (doto
                                                (ExecutorSummary. (thriftify-executor-id executor)
                                                                  (-> executor first task->component)
@@@ -2079,17 -2109,21 +2076,18 @@@
  
        (^TopologyPageInfo getTopologyPageInfo
          [this ^String topo-id ^String window ^boolean include-sys?]
-         (mark! nimbus:num-getTopologyPageInfo-calls)
+         (.mark nimbus:num-getTopologyPageInfo-calls)
          (let [info (get-common-topo-info topo-id "getTopologyPageInfo")
+ 
                exec->node+port (:executor->node+port (:assignment info))
 -              last-err-fn (partial get-last-error
 -                                   (:storm-cluster-state info)
 -                                   topo-id)
 -              topo-page-info (stats/agg-topo-execs-stats topo-id
 -                                                         exec->node+port
 -                                                         (:task->component info)
 -                                                         (:beats info)
 -                                                         (:topology info)
 -                                                         window
 -                                                         include-sys?
 -                                                         last-err-fn)]
 +              topo-page-info (StatsUtil/aggTopoExecsStats topo-id
-                                                          exec->node+port
-                                                          (:task->component info)
-                                                          (:beats info)
-                                                          (:topology info)
-                                                          window
-                                                          include-sys?
-                                                          (:storm-cluster-state info))]
++                                                          exec->node+port
++                                                          (:task->component info)
++                                                          (:beats info)
++                                                          (:topology info)
++                                                          window
++                                                          include-sys?
++                                                          (:storm-cluster-state info))]
            (when-let [owner (:owner (:base info))]
              (.set_owner topo-page-info owner))
            (when-let [sched-status (.get @(:id->sched-status nimbus) topo-id)]

http://git-wip-us.apache.org/repos/asf/storm/blob/000fcb86/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/ui/core.clj
index 0730d96,e1ab71f..a538876
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@@ -21,15 -21,14 +21,15 @@@
          ring.middleware.multipart-params)
    (:use [ring.middleware.json :only [wrap-json-params]])
    (:use [hiccup core page-helpers])
 -  (:use [org.apache.storm config util log stats converter])
 +  (:use [org.apache.storm config util log converter])
    (:use [org.apache.storm.ui helpers])
    (:use [org.apache.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
-                                               ACKER-FAIL-STREAM-ID mk-authorization-handler
-                                               start-metrics-reporters]]])
 -                                              ACKER-FAIL-STREAM-ID mk-authorization-handler]]])
++                                                ACKER-FAIL-STREAM-ID mk-authorization-handler]]])
    (:import [org.apache.storm.utils Time]
             [org.apache.storm.generated NimbusSummary]
 +           [org.apache.storm.stats StatsUtil]
-            [org.apache.storm.ui UIHelpers IConfigurator FilterConfiguration])
+            [org.apache.storm.ui UIHelpers IConfigurator FilterConfiguration]
+            [org.apache.storm.metric StormMetricsRegistry])
    (:use [clojure.string :only [blank? lower-case trim split]])
    (:import [org.apache.storm.generated ExecutorSpecificStats
              ExecutorStats ExecutorSummary ExecutorInfo TopologyInfo SpoutStats BoltStats


[05/30] storm git commit: move update tuple stat/renderStats methods to corresponding ExecutorStat classes

Posted by bo...@apache.org.
move update tuple stat/renderStats methods to corresponding ExecutorStat classes


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

Branch: refs/heads/master
Commit: f61ea0c0196da4f31126f3f96ffb2bf5551a01d2
Parents: 52d3b58
Author: 卫乐 <we...@taobao.com>
Authored: Thu Feb 25 10:59:42 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Thu Feb 25 10:59:42 2016 +0800

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    |  18 +--
 .../src/clj/org/apache/storm/daemon/task.clj    |   9 +-
 .../apache/storm/stats/BoltExecutorStats.java   |  45 ++++++
 .../jvm/org/apache/storm/stats/CommonStats.java |  40 +++++
 .../apache/storm/stats/SpoutExecutorStats.java  |  35 +++++
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 147 +------------------
 .../test/clj/org/apache/storm/nimbus_test.clj   |   4 +-
 7 files changed, 139 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f61ea0c0/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index bca03df..8009f6c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -17,7 +17,7 @@
   (:use [org.apache.storm.daemon common])
   (:import [org.apache.storm.generated Grouping Grouping$_Fields]
            [java.io Serializable]
-           [org.apache.storm.stats StatsUtil])
+           [org.apache.storm.stats BoltExecutorStats SpoutExecutorStats])
   (:use [org.apache.storm util config log])
   (:import [java.util List Random HashMap ArrayList LinkedList Map])
   (:import [org.apache.storm ICredentialsListener Thrift])
@@ -408,7 +408,7 @@
     (reify
       RunningExecutor
       (render-stats [this]
-        (clojurify-structure (StatsUtil/renderStats (:stats executor-data))))
+        (clojurify-structure (.renderStats (:stats executor-data))))
       (get-executor-id [this]
         executor-id)
       (credentials-changed [this creds]
@@ -448,7 +448,7 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (StatsUtil/spoutFailedTuple (:stats executor-data) (:stream tuple-info) time-delta))))
+      (.spoutFailedTuple (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id]
   (let [storm-conf (:storm-conf executor-data)
@@ -459,7 +459,7 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (StatsUtil/spoutAckedTuple (:stats executor-data) (:stream tuple-info) time-delta))))
+      (.spoutAckedTuple (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)
@@ -740,7 +740,7 @@
 
                                   (task/apply-hooks user-context .boltExecute (BoltExecuteInfo. tuple task-id delta))
                                   (when delta
-                                    (StatsUtil/boltExecuteTuple executor-stats
+                                    (.boltExecuteTuple executor-stats
                                                                (.getSourceComponent tuple)
                                                                (.getSourceStreamId tuple)
                                                                delta)))))))
@@ -813,7 +813,7 @@
                                                 (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                                               (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                                               (when delta
-                                                (StatsUtil/boltAckedTuple executor-stats
+                                                (.boltAckedTuple executor-stats
                                                                          (.getSourceComponent tuple)
                                                                          (.getSourceStreamId tuple)
                                                                          delta))))
@@ -828,7 +828,7 @@
                                                 (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                                               (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                                               (when delta
-                                                (StatsUtil/boltFailedTuple executor-stats
+                                                (.boltFailedTuple executor-stats
                                                                           (.getSourceComponent tuple)
                                                                           (.getSourceStreamId tuple)
                                                                           delta))))
@@ -863,7 +863,7 @@
 
 ;; TODO: refactor this to be part of an executor-specific map
 (defmethod mk-executor-stats :spout [_ rate]
-  (StatsUtil/mkSpoutStats rate))
+  (SpoutExecutorStats/mkSpoutStats rate))
 
 (defmethod mk-executor-stats :bolt [_ rate]
-  (StatsUtil/mkBoltStats rate))
+  (BoltExecutorStats/mkBoltStats rate))

http://git-wip-us.apache.org/repos/asf/storm/blob/f61ea0c0/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index c9f6828..707cdda 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -26,7 +26,6 @@
   (:import [org.apache.storm.utils Utils ConfigUtils])
   (:import [org.apache.storm.generated ShellComponent JavaObject])
   (:import [org.apache.storm.spout ShellSpout])
-  (:import [org.apache.storm.stats StatsUtil])
   (:import [java.util Collection List ArrayList])
   (:import [org.apache.storm Thrift])
   (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics]))
@@ -140,9 +139,9 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (StatsUtil/emittedTuple executor-stats stream)
+              (.emittedTuple executor-stats stream)
               (if out-task-id
-                (StatsUtil/transferredTuples executor-stats stream, 1)))
+                (.transferredTuples executor-stats stream, 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
@@ -162,8 +161,8 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (StatsUtil/emittedTuple executor-stats stream)
-               (StatsUtil/transferredTuples executor-stats stream (count out-tasks)))
+               (.emittedTuple executor-stats stream)
+               (.transferredTuples executor-stats stream (count out-tasks)))
              out-tasks)))
     ))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/f61ea0c0/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
index 7909a08..d694bc3 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
@@ -17,9 +17,13 @@
  */
 package org.apache.storm.stats;
 
+import clojure.lang.PersistentVector;
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 
+@SuppressWarnings("unchecked")
 public class BoltExecutorStats extends CommonStats {
 
     public static final String ACKED = "acked";
@@ -59,4 +63,45 @@ public class BoltExecutorStats extends CommonStats {
     public MultiLatencyStatAndMetric getExecuteLatencies() {
         return (MultiLatencyStatAndMetric) this.get(EXECUTE_LATENCIES);
     }
+
+    public void boltExecuteTuple(String component, String stream, long latencyMs) {
+        Object key = PersistentVector.create(component, stream);
+        this.getExecuted().incBy(key, this.rate);
+        this.getExecuteLatencies().record(key, latencyMs);
+    }
+
+    public void boltAckedTuple(String component, String stream, long latencyMs) {
+        Object key = PersistentVector.create(component, stream);
+        this.getAcked().incBy(key, this.rate);
+        this.getProcessLatencies().record(key, latencyMs);
+    }
+
+    public void boltFailedTuple(String component, String stream, long latencyMs) {
+        Object key = PersistentVector.create(component, stream);
+        this.getFailed().incBy(key, this.rate);
+
+    }
+
+    public Map renderStats() {
+        cleanupStats();
+        Map ret = new HashMap();
+        ret.putAll(valueStats(CommonStats.COMMON_FIELDS));
+        ret.putAll(valueStats(BoltExecutorStats.BOLT_FIELDS));
+        StatsUtil.putRawKV(ret, StatsUtil.TYPE, StatsUtil.KW_BOLT);
+
+        return ret;
+    }
+
+    public void cleanupStats() {
+        super.cleanupStats();
+        for (String field : BOLT_FIELDS) {
+            cleanupStat(this.get(field));
+        }
+    }
+
+    public static BoltExecutorStats mkBoltStats(int rate) {
+        BoltExecutorStats stats = new BoltExecutorStats();
+        stats.setRate(rate);
+        return stats;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/f61ea0c0/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java b/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
index a8bf706..93d42a4 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
@@ -21,7 +21,9 @@ import java.util.HashMap;
 import java.util.Map;
 import org.apache.storm.metric.api.IMetric;
 import org.apache.storm.metric.internal.MultiCountStatAndMetric;
+import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 
+@SuppressWarnings("unchecked")
 public class CommonStats {
     public static final int NUM_STAT_BUCKETS = 20;
 
@@ -62,4 +64,42 @@ public class CommonStats {
     protected void put(String field, Object value) {
         StatsUtil.putRawKV(metricMap, field, value);
     }
+
+    public void emittedTuple(String stream) {
+        this.getEmitted().incBy(stream, this.rate);
+    }
+
+    public void transferredTuples(String stream, int amount) {
+        this.getTransferred().incBy(stream, this.rate * amount);
+    }
+
+    protected void cleanupStats() {
+        for (String field : COMMON_FIELDS) {
+            cleanupStat(this.get(field));
+        }
+    }
+
+    protected void cleanupStat(IMetric metric) {
+        if (metric instanceof MultiCountStatAndMetric) {
+            ((MultiCountStatAndMetric) metric).close();
+        } else if (metric instanceof MultiLatencyStatAndMetric) {
+            ((MultiLatencyStatAndMetric) metric).close();
+        }
+    }
+
+    protected Map valueStats(String[] fields) {
+        Map ret = new HashMap();
+        for (String field : fields) {
+            IMetric metric = this.get(field);
+            if (metric instanceof MultiCountStatAndMetric) {
+                StatsUtil.putRawKV(ret, field, ((MultiCountStatAndMetric) metric).getTimeCounts());
+            } else if (metric instanceof MultiLatencyStatAndMetric) {
+                StatsUtil.putRawKV(ret, field, ((MultiLatencyStatAndMetric) metric).getTimeLatAvg());
+            }
+        }
+        StatsUtil.putRawKV(ret, CommonStats.RATE, this.getRate());
+
+        return ret;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/f61ea0c0/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
index 621ac24..d6d9162 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
@@ -17,9 +17,12 @@
  */
 package org.apache.storm.stats;
 
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 
+@SuppressWarnings("unchecked")
 public class SpoutExecutorStats extends CommonStats {
 
     public static final String ACKED = "acked";
@@ -46,4 +49,36 @@ public class SpoutExecutorStats extends CommonStats {
     public MultiLatencyStatAndMetric getCompleteLatencies() {
         return (MultiLatencyStatAndMetric) this.get(COMPLETE_LATENCIES);
     }
+
+    public void spoutAckedTuple(String stream, long latencyMs) {
+        this.getAcked().incBy(stream, this.rate);
+        this.getCompleteLatencies().record(stream, latencyMs);
+    }
+
+    public void spoutFailedTuple(String stream, long latencyMs) {
+        this.getFailed().incBy(stream, this.rate);
+    }
+
+    public Map renderStats() {
+        cleanupStats();
+        Map ret = new HashMap();
+        ret.putAll(valueStats(CommonStats.COMMON_FIELDS));
+        ret.putAll(valueStats(SpoutExecutorStats.SPOUT_FIELDS));
+        StatsUtil.putRawKV(ret, StatsUtil.TYPE, StatsUtil.KW_SPOUT);
+
+        return ret;
+    }
+
+    public void cleanupStats() {
+        super.cleanupStats();
+        for (String field : SpoutExecutorStats.SPOUT_FIELDS) {
+            cleanupStat(this.get(field));
+        }
+    }
+
+    public static SpoutExecutorStats mkSpoutStats(int rate) {
+        SpoutExecutorStats stats = new SpoutExecutorStats();
+        stats.setRate(rate);
+        return stats;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/f61ea0c0/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index 144872f..22ececf 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -48,9 +48,6 @@ import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.generated.TopologyPageInfo;
 import org.apache.storm.generated.TopologyStats;
-import org.apache.storm.metric.api.IMetric;
-import org.apache.storm.metric.internal.MultiCountStatAndMetric;
-import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -59,9 +56,11 @@ import org.slf4j.LoggerFactory;
 public class StatsUtil {
     private static final Logger logger = LoggerFactory.getLogger(StatsUtil.class);
 
-    private static final String TYPE = "type";
+    public static final String TYPE = "type";
     private static final String SPOUT = "spout";
     private static final String BOLT = "bolt";
+    public static final Keyword KW_SPOUT = keyword(SPOUT);
+    public static final Keyword KW_BOLT = keyword(BOLT);
 
     private static final String UPTIME = "uptime";
     private static final String HOST = "host";
@@ -111,9 +110,6 @@ public class StatsUtil {
     private static final String CID_SID_TO_IN_STATS = "cid+sid->input-stats";
     private static final String WORKERS_SET = "workers-set";
 
-    private static final Keyword KW_SPOUT = keyword(SPOUT);
-    private static final Keyword KW_BOLT = keyword(BOLT);
-
     public static final int TEN_MIN_IN_SECONDS = 60 * 10;
     public static final String TEN_MIN_IN_SECONDS_STR = TEN_MIN_IN_SECONDS + "";
 
@@ -124,120 +120,6 @@ public class StatsUtil {
 
 
     // =====================================================================================
-    // update stats methods
-    // =====================================================================================
-
-    public static BoltExecutorStats mkBoltStats(int rate) {
-        BoltExecutorStats stats = new BoltExecutorStats();
-        stats.setRate(rate);
-        return stats;
-    }
-
-    public static SpoutExecutorStats mkSpoutStats(int rate) {
-        SpoutExecutorStats stats = new SpoutExecutorStats();
-        stats.setRate(rate);
-        return stats;
-    }
-
-    public static void emittedTuple(CommonStats stats, String stream) {
-        stats.getEmitted().incBy(stream, stats.rate);
-    }
-
-    public static void transferredTuples(CommonStats stats, String stream, int amount) {
-        stats.getTransferred().incBy(stream, stats.rate * amount);
-    }
-
-    public static void boltExecuteTuple(BoltExecutorStats stats, String component, String stream, long latencyMs) {
-        Object key = PersistentVector.create(component, stream);
-        stats.getExecuted().incBy(key, stats.rate);
-        stats.getExecuteLatencies().record(key, latencyMs);
-    }
-
-    public static void boltAckedTuple(BoltExecutorStats stats, String component, String stream, long latencyMs) {
-        Object key = PersistentVector.create(component, stream);
-        stats.getAcked().incBy(key, stats.rate);
-        stats.getProcessLatencies().record(key, latencyMs);
-    }
-
-    public static void boltFailedTuple(BoltExecutorStats stats, String component, String stream, long latencyMs) {
-        Object key = PersistentVector.create(component, stream);
-        stats.getFailed().incBy(key, stats.rate);
-
-    }
-
-    public static void spoutAckedTuple(SpoutExecutorStats stats, String stream, long latencyMs) {
-        stats.getAcked().incBy(stream, stats.rate);
-        stats.getCompleteLatencies().record(stream, latencyMs);
-    }
-
-    public static void spoutFailedTuple(SpoutExecutorStats stats, String stream, long latencyMs) {
-        stats.getFailed().incBy(stream, stats.rate);
-    }
-
-    private static void cleanupStat(IMetric metric) {
-        if (metric instanceof MultiCountStatAndMetric) {
-            ((MultiCountStatAndMetric) metric).close();
-        } else if (metric instanceof MultiLatencyStatAndMetric) {
-            ((MultiLatencyStatAndMetric) metric).close();
-        }
-    }
-
-    public static Map renderStats(SpoutExecutorStats stats) {
-        cleanupSpoutStats(stats);
-        Map ret = new HashMap();
-        ret.putAll(valueStats(stats, CommonStats.COMMON_FIELDS));
-        ret.putAll(valueStats(stats, SpoutExecutorStats.SPOUT_FIELDS));
-        putRawKV(ret, TYPE, KW_SPOUT);
-
-        return ret;
-    }
-
-    public static Map renderStats(BoltExecutorStats stats) {
-        cleanupBoltStats(stats);
-        Map ret = new HashMap();
-        ret.putAll(valueStats(stats, CommonStats.COMMON_FIELDS));
-        ret.putAll(valueStats(stats, BoltExecutorStats.BOLT_FIELDS));
-        putRawKV(ret, TYPE, KW_BOLT);
-
-        return ret;
-    }
-
-    public static void cleanupSpoutStats(SpoutExecutorStats stats) {
-        cleanupCommonStats(stats);
-        for (String field : SpoutExecutorStats.SPOUT_FIELDS) {
-            cleanupStat(stats.get(field));
-        }
-    }
-
-    public static void cleanupBoltStats(BoltExecutorStats stats) {
-        cleanupCommonStats(stats);
-        for (String field : BoltExecutorStats.BOLT_FIELDS) {
-            cleanupStat(stats.get(field));
-        }
-    }
-
-    public static void cleanupCommonStats(CommonStats stats) {
-        for (String field : CommonStats.COMMON_FIELDS) {
-            cleanupStat(stats.get(field));
-        }
-    }
-
-    private static Map valueStats(CommonStats stats, String[] fields) {
-        Map ret = new HashMap();
-        for (String field : fields) {
-            IMetric metric = stats.get(field);
-            if (metric instanceof MultiCountStatAndMetric) {
-                putRawKV(ret, field, ((MultiCountStatAndMetric) metric).getTimeCounts());
-            } else if (metric instanceof MultiLatencyStatAndMetric) {
-                putRawKV(ret, field, ((MultiLatencyStatAndMetric) metric).getTimeLatAvg());
-            }
-        }
-        putRawKV(ret, CommonStats.RATE, stats.getRate());
-
-        return ret;
-    }
-
-    // =====================================================================================
     // aggregation stats methods
     // =====================================================================================
 
@@ -1166,9 +1048,6 @@ public class StatsUtil {
         return ret;
     }
 
-    /**
-     * called in nimbus.clj
-     */
     public static ComponentPageInfo aggCompExecsStats(
             Map exec2hostPort, Map task2component, Map beats, String window, boolean includeSys,
             String topologyId, StormTopology topology, String componentId) {
@@ -1184,9 +1063,6 @@ public class StatsUtil {
     // clojurify stats methods
     // =====================================================================================
 
-    /**
-     * called in converter.clj
-     */
     public static Map clojurifyStats(Map stats) {
         Map ret = new HashMap();
         for (Object o : stats.entrySet()) {
@@ -1245,9 +1121,6 @@ public class StatsUtil {
         return ret;
     }
 
-    /**
-     * caller: nimbus.clj
-     */
     public static List extractNodeInfosFromHbForComp(
             Map exec2hostPort, Map task2component, boolean includeSys, String compId) {
         List ret = new ArrayList();
@@ -1340,7 +1213,7 @@ public class StatsUtil {
 
 
     /**
-     * caller: core.clj
+     * computes max bolt capacity
      *
      * @param executorSumms a list of ExecutorSummary
      * @return max bolt capacity
@@ -1774,9 +1647,6 @@ public class StatsUtil {
         return ret;
     }
 
-    /**
-     * called in converter.clj
-     */
     public static Map thriftifyStats(List stats) {
         Map ret = new HashMap();
         for (Object o : stats) {
@@ -1791,9 +1661,6 @@ public class StatsUtil {
         return ret;
     }
 
-    /**
-     * called in nimbus.clj
-     */
     public static ExecutorStats thriftifyExecutorStats(Map stats) {
         ExecutorStats ret = new ExecutorStats();
         ExecutorSpecificStats specificStats = thriftifySpecificStats(stats);
@@ -2091,16 +1958,10 @@ public class StatsUtil {
         return t / c;
     }
 
-    /**
-     * caller: core.clj
-     */
     public static String floatStr(double n) {
         return String.format("%.3f", n);
     }
 
-    /**
-     * caller: core.clj
-     */
     public static String errorSubset(String errorStr) {
         return errorStr.substring(0, 200);
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/f61ea0c0/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 a76db54..5964e6f 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -23,7 +23,7 @@
            [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
            [org.apache.storm.generated GlobalStreamId]
            [org.apache.storm Thrift]
-           [org.apache.storm.stats StatsUtil])
+           [org.apache.storm.stats BoltExecutorStats])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
   (:import [org.apache.storm.scheduler INimbus])
   (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
@@ -141,7 +141,7 @@
         stats (:executor-stats curr-beat)]
     (.worker-heartbeat! state storm-id node port
       {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10
-       :executor-stats (merge stats {executor (clojurify-structure (StatsUtil/renderStats (StatsUtil/mkBoltStats 20)))})}
+       :executor-stats (merge stats {executor (clojurify-structure (.renderStats (BoltExecutorStats/mkBoltStats 20)))})}
       )))
 
 (defn slot-assignments [cluster storm-id]


[30/30] storm git commit: Added STORM-1252 to Changelog

Posted by bo...@apache.org.
Added STORM-1252 to Changelog


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

Branch: refs/heads/master
Commit: fa25f3d7fae52b7d3e951ab84e6a7fb8c10381f1
Parents: cf7ef7f
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Tue Mar 15 12:34:12 2016 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Tue Mar 15 12:34:12 2016 -0500

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/fa25f3d7/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 3908916..d99bf1d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 2.0.0
+ * STORM-1252: port backtype.storm.stats to java
  * STORM-1250: port backtype.storm.serialization-test to java
  * STORM-1605: use '/usr/bin/env python' to check python version
  * STORM-1618: Add the option of passing config directory


[10/30] storm git commit: merge from master

Posted by bo...@apache.org.
merge from master


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

Branch: refs/heads/master
Commit: f819b4999eac052840ff69552198b184afa0c1e0
Parents: 3fc80c4 8b622ce
Author: 卫乐 <we...@taobao.com>
Authored: Fri Feb 26 14:02:45 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Fri Feb 26 14:02:45 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   5 +
 bin/flight.bash                                 |   4 +-
 .../storm/hdfs/bolt/AbstractHdfsBolt.java       |  10 +-
 .../storm/hdfs/bolt/AvroGenericRecordBolt.java  |   8 +-
 .../org/apache/storm/hdfs/bolt/HdfsBolt.java    |   8 +-
 .../storm/hdfs/bolt/SequenceFileBolt.java       |   8 +-
 .../src/clj/org/apache/storm/daemon/drpc.clj    |  38 +--
 .../clj/org/apache/storm/daemon/logviewer.clj   |  65 ++---
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |   6 +-
 .../clj/org/apache/storm/daemon/supervisor.clj  |   2 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |   8 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj | 105 ++++----
 .../src/clj/org/apache/storm/ui/helpers.clj     | 199 +-------------
 .../storm/cluster/IStormClusterState.java       |   2 +
 .../storm/cluster/StormClusterStateImpl.java    |   8 +-
 .../storm/metric/FileBasedEventLogger.java      |  18 +-
 .../serialization/SerializationFactory.java     |   2 +
 .../apache/storm/trident/tuple/ConsList.java    |  20 +-
 .../apache/storm/ui/FilterConfiguration.java    |  63 +++++
 .../jvm/org/apache/storm/ui/IConfigurator.java  |  24 ++
 .../src/jvm/org/apache/storm/ui/UIHelpers.java  | 267 +++++++++++++++++++
 .../storm/utils/WorkerBackpressureCallback.java |   2 +-
 .../storm/utils/WorkerBackpressureThread.java   |  38 ++-
 .../utils/WorkerBackpressureThreadTest.java     |  50 ++++
 24 files changed, 617 insertions(+), 343 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f819b499/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/f819b499/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/f819b499/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------


[04/30] storm git commit: add translated java files

Posted by bo...@apache.org.
add translated java files


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

Branch: refs/heads/master
Commit: 52d3b587f07db7dcf66b774531e2face7247c7b6
Parents: afd2d52
Author: 卫乐 <we...@taobao.com>
Authored: Wed Feb 24 21:12:53 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Wed Feb 24 21:12:53 2016 +0800

----------------------------------------------------------------------
 .../apache/storm/stats/BoltExecutorStats.java   |   62 +
 .../jvm/org/apache/storm/stats/CommonStats.java |   65 +
 .../apache/storm/stats/SpoutExecutorStats.java  |   49 +
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 2178 ++++++++++++++++++
 4 files changed, 2354 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/52d3b587/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
new file mode 100644
index 0000000..7909a08
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
@@ -0,0 +1,62 @@
+/**
+ * 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 org.apache.storm.metric.internal.MultiCountStatAndMetric;
+import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
+
+public class BoltExecutorStats extends CommonStats {
+
+    public static final String ACKED = "acked";
+    public static final String FAILED = "failed";
+    public static final String EXECUTED = "executed";
+    public static final String PROCESS_LATENCIES = "process-latencies";
+    public static final String EXECUTE_LATENCIES = "execute-latencies";
+
+    public static final String[] BOLT_FIELDS = {ACKED, FAILED, EXECUTED, PROCESS_LATENCIES, EXECUTE_LATENCIES};
+
+    public BoltExecutorStats() {
+        super();
+
+        put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        put(FAILED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        put(EXECUTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        put(PROCESS_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
+        put(EXECUTE_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
+    }
+
+    public MultiCountStatAndMetric getAcked() {
+        return (MultiCountStatAndMetric) this.get(ACKED);
+    }
+
+    public MultiCountStatAndMetric getFailed() {
+        return (MultiCountStatAndMetric) this.get(FAILED);
+    }
+
+    public MultiCountStatAndMetric getExecuted() {
+        return (MultiCountStatAndMetric) this.get(EXECUTED);
+    }
+
+    public MultiLatencyStatAndMetric getProcessLatencies() {
+        return (MultiLatencyStatAndMetric) this.get(PROCESS_LATENCIES);
+    }
+
+    public MultiLatencyStatAndMetric getExecuteLatencies() {
+        return (MultiLatencyStatAndMetric) this.get(EXECUTE_LATENCIES);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/52d3b587/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java b/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
new file mode 100644
index 0000000..a8bf706
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
@@ -0,0 +1,65 @@
+/**
+ * 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.Map;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.internal.MultiCountStatAndMetric;
+
+public class CommonStats {
+    public static final int NUM_STAT_BUCKETS = 20;
+
+    public static final String RATE = "rate";
+
+    public static final String EMITTED = "emitted";
+    public static final String TRANSFERRED = "transferred";
+    public static final String[] COMMON_FIELDS = {EMITTED, TRANSFERRED};
+
+    protected int rate;
+    protected final Map metricMap = new HashMap();
+
+    public CommonStats() {
+        put(EMITTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        put(TRANSFERRED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+    }
+
+    public int getRate() {
+        return this.rate;
+    }
+
+    public void setRate(int rate) {
+        this.rate = rate;
+    }
+
+    public MultiCountStatAndMetric getEmitted() {
+        return (MultiCountStatAndMetric) get(EMITTED);
+    }
+
+    public MultiCountStatAndMetric getTransferred() {
+        return (MultiCountStatAndMetric) get(TRANSFERRED);
+    }
+
+    public IMetric get(String field) {
+        return (IMetric) StatsUtil.getByKeyword(metricMap, field);
+    }
+
+    protected void put(String field, Object value) {
+        StatsUtil.putRawKV(metricMap, field, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/52d3b587/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
new file mode 100644
index 0000000..621ac24
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
@@ -0,0 +1,49 @@
+/**
+ * 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 org.apache.storm.metric.internal.MultiCountStatAndMetric;
+import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
+
+public class SpoutExecutorStats extends CommonStats {
+
+    public static final String ACKED = "acked";
+    public static final String FAILED = "failed";
+    public static final String COMPLETE_LATENCIES = "complete-latencies";
+
+    public static final String[] SPOUT_FIELDS = {ACKED, FAILED, COMPLETE_LATENCIES};
+
+    public SpoutExecutorStats() {
+        super();
+        this.put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(FAILED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(COMPLETE_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
+    }
+
+    public MultiCountStatAndMetric getAcked() {
+        return (MultiCountStatAndMetric) this.get(ACKED);
+    }
+
+    public MultiCountStatAndMetric getFailed() {
+        return (MultiCountStatAndMetric) this.get(FAILED);
+    }
+
+    public MultiLatencyStatAndMetric getCompleteLatencies() {
+        return (MultiLatencyStatAndMetric) this.get(COMPLETE_LATENCIES);
+    }
+}


[11/30] storm git commit: merge from master

Posted by bo...@apache.org.
merge from master


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

Branch: refs/heads/master
Commit: e0e9de7d01bb09e6593093cc9324b09f03abb55c
Parents: f819b49
Author: 卫乐 <we...@taobao.com>
Authored: Fri Feb 26 14:03:35 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Fri Feb 26 14:03:35 2016 +0800

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/ui/core.clj | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e0e9de7d/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index d0d87b8..2df61f0 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -28,6 +28,7 @@
                                               start-metrics-reporters]]])
   (:import [org.apache.storm.utils Time]
            [org.apache.storm.generated NimbusSummary]
+           [org.apache.storm.stats StatsUtil]
            [org.apache.storm.ui UIHelpers IConfigurator FilterConfiguration])
   (:use [clojure.string :only [blank? lower-case trim split]])
   (:import [org.apache.storm.generated ExecutorSpecificStats
@@ -111,7 +112,7 @@
 
 (defn executor-summary-type
   [topology ^ExecutorSummary s]
-  (component-type topology (.get_component_id s)))
+  (StatsUtil/componentType topology (.get_component_id s)))
 
 (defn is-ack-stream
   [stream]


[14/30] storm git commit: fix possible NPE & ClassCastException

Posted by bo...@apache.org.
fix possible NPE & ClassCastException


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

Branch: refs/heads/master
Commit: faaacaee046bfa4f458c19cade678515a021d836
Parents: abe9b67
Author: 卫乐 <we...@taobao.com>
Authored: Tue Mar 1 11:47:51 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Tue Mar 1 11:47:51 2016 +0800

----------------------------------------------------------------------
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 23 +++++++++++++++-----
 1 file changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/faaacaee/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index efdf8e0..0ed2af9 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -578,7 +578,7 @@ public class StatsUtil {
         Map ret = new HashMap();
         putRawKV(ret, NUM_TASKS, task2comp.size());
         putRawKV(ret, NUM_WORKERS, ((Set) getByKeyword(accData, WORKERS_SET)).size());
-        putRawKV(ret, NUM_EXECUTORS, exec2nodePort.size());
+        putRawKV(ret, NUM_EXECUTORS, exec2nodePort != null ? exec2nodePort.size() : 0);
 
         Map bolt2stats = getMapByKeyword(accData, BOLT_TO_STATS);
         Map aggBolt2stats = new HashMap();
@@ -1339,11 +1339,18 @@ public class StatsUtil {
      */
     private static Map filterSysStreams(Map stats, boolean includeSys) {
         if (!includeSys) {
-            for (Object win : stats.keySet()) {
-                Map stream2stat = (Map) stats.get(win);
-                for (Iterator itr = stream2stat.keySet().iterator(); itr.hasNext(); ) {
-                    Object key = itr.next();
-                    if (key instanceof String && Utils.isSystemId((String) key)) {
+            for (Iterator itr = stats.keySet().iterator(); itr.hasNext(); ) {
+                Object winOrStream = itr.next();
+                if (isWindow(winOrStream)) {
+                    Map stream2stat = (Map) 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();
+                        }
+                    }
+                } else {
+                    if (winOrStream instanceof String && Utils.isSystemId((String) winOrStream)) {
                         itr.remove();
                     }
                 }
@@ -1352,6 +1359,10 @@ public class StatsUtil {
         return stats;
     }
 
+    private static boolean isWindow(Object key) {
+        return key.equals("600") || key.equals("10800") || key.equals("86400") || key.equals(":all-time");
+    }
+
     /**
      * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out)
      */


[16/30] storm git commit: upmerge from master

Posted by bo...@apache.org.
upmerge from master


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

Branch: refs/heads/master
Commit: b1c8188d8252ef613c04aeae9e9560a800f9ead3
Parents: 69f2906 df54280
Author: 卫乐 <we...@taobao.com>
Authored: Thu Mar 3 10:17:26 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Thu Mar 3 10:17:26 2016 +0800

----------------------------------------------------------------------
 .gitignore                                      |  1 +
 CHANGELOG.md                                    |  6 ++
 bin/storm.cmd                                   |  2 +-
 bin/storm.py                                    |  4 +-
 .../storm/starter/ThroughputVsLatency.java      |  2 +-
 .../apache/storm/sql/compiler/CompilerUtil.java |  7 ++-
 .../apache/storm/sql/compiler/ExprCompiler.java | 32 ++++++++--
 .../backends/standalone/RelNodeCompiler.java    |  6 +-
 .../apache/storm/sql/parser/StormParser.java    |  5 ++
 .../test/org/apache/storm/sql/TestStormSql.java | 64 +++++++++++++++++---
 .../storm/sql/compiler/TestCompilerUtils.java   | 62 ++++++++++++++++---
 .../storm/sql/compiler/TestExprSemantic.java    | 18 ++++++
 .../backends/standalone/TestPlanCompiler.java   | 20 ++++++
 .../backends/trident/TestPlanCompiler.java      |  4 +-
 .../test/org/apache/storm/sql/TestUtils.java    | 32 +++++++++-
 pom.xml                                         | 23 +++++++
 .../apache/storm/command/upload_credentials.clj | 35 -----------
 .../src/jvm/org/apache/storm/command/List.java  | 50 ---------------
 .../apache/storm/command/ListTopologies.java    | 52 ++++++++++++++++
 .../apache/storm/command/UploadCredentials.java | 61 +++++++++++++++++++
 .../jvm/org/apache/storm/utils/ConfigUtils.java |  8 +--
 .../src/jvm/org/apache/storm/utils/Utils.java   |  9 +--
 22 files changed, 375 insertions(+), 128 deletions(-)
----------------------------------------------------------------------



[17/30] storm git commit: revert unrelated changes to supervisor.clj

Posted by bo...@apache.org.
revert unrelated changes to supervisor.clj


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

Branch: refs/heads/master
Commit: 4e0ff2f6e238a59c13d9af6dc3db84ae5817365f
Parents: b1c8188
Author: 卫乐 <we...@taobao.com>
Authored: Thu Mar 3 10:21:55 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Thu Mar 3 10:21:55 2016 +0800

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/supervisor.clj | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4e0ff2f6/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index c1529c0..7295679 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -35,6 +35,7 @@
   (:use [org.apache.storm.daemon common])
   (:import [org.apache.storm.command HealthCheck])
   (:require [org.apache.storm.daemon [worker :as worker]]
+
             [clojure.set :as set])
   (:import [org.apache.thrift.transport TTransportException])
   (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
@@ -79,6 +80,7 @@
           new-profiler-actions
           (->>
             (dofor [sid (distinct storm-ids)]
+
                    (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid)] (clojurify-profile-request request)))]
                       {sid topo-profile-actions}))
            (apply merge))]


[07/30] storm git commit: merge conflicts from master

Posted by bo...@apache.org.
merge conflicts from master


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

Branch: refs/heads/master
Commit: 880134881566427e886b01d44890d22db483f6bd
Parents: 67a5878
Author: 卫乐 <we...@taobao.com>
Authored: Thu Feb 25 13:11:50 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Thu Feb 25 13:11:50 2016 +0800

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/converter.clj         | 4 ++--
 storm-core/src/clj/org/apache/storm/daemon/executor.clj   | 1 -
 storm-core/src/clj/org/apache/storm/daemon/nimbus.clj     | 7 +++----
 storm-core/src/clj/org/apache/storm/daemon/supervisor.clj | 2 --
 storm-core/src/clj/org/apache/storm/ui/core.clj           | 8 +++++++-
 storm-core/test/clj/org/apache/storm/nimbus_test.clj      | 8 +++++---
 6 files changed, 17 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/88013488/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index 54d906d..495fe7f 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -192,9 +192,9 @@
 (defn thriftify-storm-base [storm-base]
   (doto (StormBase.)
     (.set_name (:storm-name storm-base))
-    (.set_launch_time_secs (int (:launch-time-secs storm-base)))
+    (.set_launch_time_secs (if (:launch-time-secs storm-base) (int (:launch-time-secs storm-base)) 0))
     (.set_status (convert-to-status-from-symbol (:status storm-base)))
-    (.set_num_workers (int (:num-workers storm-base)))
+    (.set_num_workers (if (:num-workers storm-base) (int (:num-workers storm-base)) 0))
     (.set_component_executors (map-val int (:component->executors storm-base)))
     (.set_owner (:owner storm-base))
     (.set_topology_action_options (thriftify-topology-action-options storm-base))

http://git-wip-us.apache.org/repos/asf/storm/blob/88013488/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index edd1368..3b4e330 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -42,7 +42,6 @@
            [org.json.simple JSONValue]
            [com.lmax.disruptor.dsl ProducerType]
            [org.apache.storm StormTimer])
-  (:require [org.apache.storm [cluster :as cluster]])
   (:require [org.apache.storm.daemon [task :as task]])
   (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
   (:require [clojure.set :as set]))

http://git-wip-us.apache.org/repos/asf/storm/blob/88013488/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 735200f..a0e652b 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -50,10 +50,9 @@
             ProfileRequest ProfileAction NodeInfo LSTopoHistory])
   (:import [org.apache.storm.daemon Shutdownable])
   (:import [org.apache.storm.validation ConfigValidation])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
-  (:use [org.apache.storm util config log zookeeper])
-  (:require [org.apache.storm [cluster :as cluster]
-                            [converter :as converter]])
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
+  (:use [org.apache.storm util config log converter])
+  (:require [org.apache.storm [converter :as converter]])
   (:require [clojure.set :as set])
   (:import [org.apache.storm.daemon.common StormBase Assignment])
   (:import [org.apache.storm.zookeeper Zookeeper])

http://git-wip-us.apache.org/repos/asf/storm/blob/88013488/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 1446ac9..781bd94 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -35,7 +35,6 @@
   (:use [org.apache.storm.daemon common])
   (:import [org.apache.storm.command HealthCheck])
   (:require [org.apache.storm.daemon [worker :as worker]]
-
             [clojure.set :as set])
   (:import [org.apache.thrift.transport TTransportException])
   (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
@@ -80,7 +79,6 @@
           new-profiler-actions
           (->>
             (dofor [sid (distinct storm-ids)]
-
                    (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid)] (clojurify-profile-request request)))]
                       {sid topo-profile-actions}))
            (apply merge))]

http://git-wip-us.apache.org/repos/asf/storm/blob/88013488/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index 1e531c4..25aa717 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -21,7 +21,7 @@
         ring.middleware.multipart-params)
   (:use [ring.middleware.json :only [wrap-json-params]])
   (:use [hiccup core page-helpers])
-  (:use [org.apache.storm config util log zookeeper converter])
+  (:use [org.apache.storm config util log converter])
   (:use [org.apache.storm.ui helpers])
   (:use [org.apache.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
                                               ACKER-FAIL-STREAM-ID mk-authorization-handler
@@ -272,6 +272,12 @@
                            :grouping (clojure.core/name (thrift/grouping-type group))})})])]
     (into {} (doall components))))
 
+(defn mk-include-sys-fn
+  [include-sys?]
+  (if include-sys?
+    (fn [_] true)
+    (fn [stream] (and (string? stream) (not (Utils/isSystemId stream))))))
+
 (defn stream-boxes [datmap]
   (let [filter-fn (mk-include-sys-fn true)
         streams

http://git-wip-us.apache.org/repos/asf/storm/blob/88013488/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 3670fd1..8c383e5 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -15,14 +15,15 @@
 ;; limitations under the License.
 (ns org.apache.storm.nimbus-test
   (:use [clojure test])
-  (:require [org.apache.storm [util :as util] [stats :as stats]])
+  (:require [org.apache.storm [util :as util]])
   (:require [org.apache.storm.daemon [nimbus :as nimbus]])
   (:require [org.apache.storm [converter :as converter]])
   (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
             TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
            [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
            [org.apache.storm.generated GlobalStreamId]
-           [org.apache.storm Thrift])
+           [org.apache.storm Thrift]
+           [org.apache.storm.stats BoltExecutorStats])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
   (:import [org.apache.storm.scheduler INimbus])
   (:import [org.mockito Mockito])
@@ -143,7 +144,8 @@
         curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
         stats (:executor-stats curr-beat)]
     (.workerHeartbeat state storm-id node port
-      (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
+      (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10
+                               :executor-stats (merge stats {executor (clojurify-structure (.renderStats (BoltExecutorStats/mkBoltStats 20)))})})
       )))
 
 (defn slot-assignments [cluster storm-id]


[06/30] storm git commit: Merge https://github.com/apache/storm

Posted by bo...@apache.org.
Merge https://github.com/apache/storm

Conflicts:
	storm-core/src/clj/org/apache/storm/converter.clj
	storm-core/src/clj/org/apache/storm/daemon/executor.clj
	storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
	storm-core/src/clj/org/apache/storm/stats.clj
	storm-core/src/clj/org/apache/storm/ui/core.clj
	storm-core/test/clj/org/apache/storm/nimbus_test.clj


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

Branch: refs/heads/master
Commit: 67a5878e5f37ccd317c10ef8dcbd56b9de233997
Parents: f61ea0c dd00bc0
Author: 卫乐 <we...@taobao.com>
Authored: Thu Feb 25 13:10:07 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Thu Feb 25 13:10:07 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   8 +
 bin/storm.cmd                                   |   2 +-
 bin/storm.py                                    |   6 +-
 conf/defaults.yaml                              |   4 +-
 .../apache/storm/kafka/PartitionManager.java    |   5 +-
 .../kafka/trident/TridentKafkaEmitter.java      |   5 +-
 .../src/clj/org/apache/storm/thrift.clj         |   2 +-
 storm-core/src/clj/org/apache/storm/cluster.clj | 700 -------------------
 .../cluster_state/zookeeper_state_factory.clj   | 165 -----
 .../clj/org/apache/storm/command/heartbeats.clj |   6 +-
 .../clj/org/apache/storm/command/monitor.clj    |  37 -
 .../clj/org/apache/storm/command/rebalance.clj  |  47 --
 .../org/apache/storm/command/set_log_level.clj  |  76 --
 .../apache/storm/command/shell_submission.clj   |   2 +-
 .../src/clj/org/apache/storm/converter.clj      |  19 +-
 .../src/clj/org/apache/storm/daemon/common.clj  |  12 +-
 .../clj/org/apache/storm/daemon/executor.clj    |  11 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  | 151 ++--
 .../clj/org/apache/storm/daemon/supervisor.clj  |  38 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  59 +-
 .../clj/org/apache/storm/internal/thrift.clj    |   2 +-
 .../storm/pacemaker/pacemaker_state_factory.clj | 141 ----
 storm-core/src/clj/org/apache/storm/testing.clj |  12 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |   6 -
 storm-core/src/clj/org/apache/storm/util.clj    |  11 +
 .../src/clj/org/apache/storm/zookeeper.clj      |  74 --
 storm-core/src/jvm/org/apache/storm/Config.java |   9 +
 .../jvm/org/apache/storm/callback/Callback.java |  23 -
 .../storm/callback/ZKStateChangedCallback.java  |  25 +
 .../org/apache/storm/cluster/ClusterState.java  | 217 ------
 .../storm/cluster/ClusterStateContext.java      |   2 +-
 .../storm/cluster/ClusterStateFactory.java      |  28 -
 .../org/apache/storm/cluster/ClusterUtils.java  | 244 +++++++
 .../org/apache/storm/cluster/ExecutorBeat.java  |  44 ++
 .../org/apache/storm/cluster/IStateStorage.java | 222 ++++++
 .../storm/cluster/IStormClusterState.java       | 122 ++++
 .../storm/cluster/PaceMakerStateStorage.java    | 216 ++++++
 .../cluster/PaceMakerStateStorageFactory.java   |  64 ++
 .../storm/cluster/StateStorageFactory.java      |  28 +
 .../storm/cluster/StormClusterStateImpl.java    | 686 ++++++++++++++++++
 .../apache/storm/cluster/ZKStateStorage.java    | 244 +++++++
 .../storm/cluster/ZKStateStorageFactory.java    |  36 +
 .../src/jvm/org/apache/storm/command/CLI.java   |  34 +-
 .../jvm/org/apache/storm/command/Monitor.java   |  65 ++
 .../jvm/org/apache/storm/command/Rebalance.java |  86 +++
 .../org/apache/storm/command/SetLogLevel.java   | 116 +++
 .../apache/storm/pacemaker/PacemakerClient.java |   6 +-
 .../security/auth/ThriftConnectionType.java     |   2 +-
 .../testing/staticmocking/MockedCluster.java    |  31 +
 .../MockedPaceMakerStateStorageFactory.java     |  32 +
 .../src/jvm/org/apache/storm/utils/Utils.java   |  61 +-
 .../org/apache/storm/zookeeper/Zookeeper.java   |  77 +-
 .../org/apache/storm/integration_test.clj       |  13 +-
 .../test/clj/org/apache/storm/cluster_test.clj  | 202 +++---
 .../storm/messaging/netty_integration_test.clj  |   1 +
 .../test/clj/org/apache/storm/nimbus_test.clj   | 164 ++---
 .../storm/pacemaker_state_factory_test.clj      | 121 ++--
 .../storm/security/auth/nimbus_auth_test.clj    |   3 +-
 .../clj/org/apache/storm/supervisor_test.clj    | 163 +++--
 .../org/apache/storm/command/RebalanceTest.java |  41 ++
 .../apache/storm/command/SetLogLevelTest.java   |  54 ++
 .../jvm/org/apache/storm/command/TestCLI.java   |  44 +-
 .../storm/utils/staticmocking/package-info.java |   2 +-
 63 files changed, 3036 insertions(+), 2093 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/67a5878e/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/converter.clj
index 6e9eeb8,e269c5d..54d906d
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@@ -17,9 -17,9 +17,10 @@@
    (:import [org.apache.storm.generated SupervisorInfo NodeInfo Assignment WorkerResources
              StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
              TopologyActionOptions DebugOptions ProfileRequest]
 -           [org.apache.storm.utils Utils])
 +           [org.apache.storm.utils Utils]
 +           [org.apache.storm.stats StatsUtil])
+   (:import [org.apache.storm.cluster ExecutorBeat])
 -  (:use [org.apache.storm util stats log])
 +  (:use [org.apache.storm util log])
    (:require [org.apache.storm.daemon [common :as common]]))
  
  (defn thriftify-supervisor-info [supervisor-info]
@@@ -223,24 -239,22 +224,22 @@@
       }
      {}))
  
+ (defn clojurify-zk-executor-hb [^ExecutorBeat executor-hb]
+   (if executor-hb
 -    {:stats (clojurify-executor-stats (.getStats executor-hb))
++    {:stats (StatsUtil/clojurifyExecutorStats (.getStats executor-hb))
+      :uptime (.getUptime executor-hb)
+      :time-secs (.getTimeSecs executor-hb)
+      }
+     {}))
+ 
  (defn thriftify-zk-worker-hb [worker-hb]
    (if (not-empty (filter second (:executor-stats worker-hb)))
      (doto (ClusterWorkerHeartbeat.)
        (.set_uptime_secs (:uptime worker-hb))
        (.set_storm_id (:storm-id worker-hb))
 -      (.set_executor_stats (thriftify-stats (filter second (:executor-stats worker-hb))))
 +      (.set_executor_stats (StatsUtil/thriftifyStats (filter second (:executor-stats worker-hb))))
        (.set_time_secs (:time-secs worker-hb)))))
  
- (defn clojurify-error [^ErrorInfo error]
-   (if error
-     {
-       :error (.get_error error)
-       :time-secs (.get_error_time_secs error)
-       :host (.get_host error)
-       :port (.get_port error)
-       }
-     ))
- 
  (defn thriftify-error [error]
    (doto (ErrorInfo. (:error error) (:time-secs error))
      (.set_host (:host error))

http://git-wip-us.apache.org/repos/asf/storm/blob/67a5878e/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 8009f6c,9ff93f8..edd1368
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -258,9 -257,8 +258,8 @@@
       :batch-transfer-queue batch-transfer->worker
       :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
       :suicide-fn (:suicide-fn worker)
-      :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker) 
-                                                           :acls (Utils/getWorkerACL storm-conf)
-                                                           :context (ClusterStateContext. DaemonType/WORKER))
 -     :storm-cluster-state (ClusterUtils/mkStormClusterState (:state-store worker) (Utils/getWorkerACL storm-conf)
 -                            (ClusterStateContext. DaemonType/WORKER))
++      :storm-cluster-state (ClusterUtils/mkStormClusterState (:state-store worker) (Utils/getWorkerACL storm-conf)
++                             (ClusterStateContext. DaemonType/WORKER))
       :type executor-type
       ;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)
       :stats (mk-executor-stats <> (ConfigUtils/samplingRate storm-conf))

http://git-wip-us.apache.org/repos/asf/storm/blob/67a5878e/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 992a864,e524ec2..735200f
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -917,7 -916,7 +917,7 @@@
          storm-cluster-state (:storm-cluster-state nimbus)
          ^INimbus inimbus (:inimbus nimbus)
          ;; read all the topologies
-         topology-ids (.active-storms storm-cluster-state)
 -        topology-ids (.activeStorms storm-cluster-state)
++          topology-ids (.activeStorms storm-cluster-state)
          topologies (into {} (for [tid topology-ids]
                                {tid (read-topology-details nimbus tid)}))
          topologies (Topologies. topologies)
@@@ -1668,8 -1675,8 +1676,8 @@@
                executor->host+port (map-val (fn [[node port]]
                                               [(node->host node) port])
                                      executor->node+port)
 -              nodeinfos (stats/extract-nodeinfos-from-hb-for-comp executor->host+port task->component false component_id)
 +              nodeinfos (clojurify-structure (StatsUtil/extractNodeInfosFromHbForComp executor->host+port task->component false component_id))
-               all-pending-actions-for-topology (.get-topology-profile-requests storm-cluster-state id true)
+               all-pending-actions-for-topology (clojurify-profile-request (.getTopologyProfileRequests storm-cluster-state id))
                latest-profile-actions (remove nil? (map (fn [nodeInfo]
                                                           (->> all-pending-actions-for-topology
                                                                (filter #(and (= (:host nodeInfo) (.get_node (.get_nodeInfo %)))
@@@ -1705,7 -1712,7 +1713,7 @@@
                                 (.containsKey named-loggers logger-name))
                          (.remove named-loggers logger-name))))))
              (log-message "Setting log config for " storm-name ":" merged-log-config)
-             (.set-topology-log-config! storm-cluster-state id merged-log-config)))
 -            (.setTopologyLogConfig storm-cluster-state id merged-log-config)))
++          (.setTopologyLogConfig storm-cluster-state id merged-log-config)))
  
        (uploadNewCredentials [this storm-name credentials]
          (mark! nimbus:num-uploadNewCredentials-calls)
@@@ -1789,7 -1796,7 +1797,7 @@@
                storm-name (topology-conf TOPOLOGY-NAME)
                _ (check-authorization! nimbus storm-name topology-conf "getLogConfig")
               storm-cluster-state (:storm-cluster-state nimbus)
-              log-config (.topology-log-config storm-cluster-state id nil)]
 -             log-config (.topologyLogConfig storm-cluster-state id nil)]
++              log-config (.topologyLogConfig storm-cluster-state id nil)]
             (if log-config log-config (LogConfig.))))
  
        (^String getTopologyConf [this ^String id]
@@@ -1910,9 -1917,9 +1918,10 @@@
                executor-summaries (dofor [[executor [node port]] (:executor->node+port assignment)]
                                          (let [host (-> assignment :node->host (get node))
                                                heartbeat (get beats executor)
-                                               stats (:stats heartbeat)
-                                               stats (if stats
-                                                       (StatsUtil/thriftifyExecutorStats stats))]
+                                               excutorstats (:stats heartbeat)
+                                               excutorstats (if excutorstats
 -                                                      (stats/thriftify-executor-stats excutorstats))]
++                                                      (StatsUtil/thriftifyExecutorStats excutorstats))]
++                                              
                                            (doto
                                                (ExecutorSummary. (thriftify-executor-id executor)
                                                                  (-> executor first task->component)
@@@ -1975,7 -1982,7 +1984,7 @@@
                nimbus-host-port-info (:nimbus-host-port-info nimbus)
                conf (:conf nimbus)]
            (if (instance? LocalFsBlobStore blob-store)
-               (.setup-blobstore! storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
 -              (.setupBlobstore storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
++            (.setupBlobstore storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
            (log-debug "Created state in zookeeper" storm-cluster-state blob-store nimbus-host-port-info)))
  
        (^void uploadBlobChunk [this ^String session ^ByteBuffer blob-chunk]

http://git-wip-us.apache.org/repos/asf/storm/blob/67a5878e/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------


[02/30] storm git commit: port backtype.storm.stats to java

Posted by bo...@apache.org.
port backtype.storm.stats to java


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

Branch: refs/heads/master
Commit: afd2d525be396c6f430e6a4a13cd1f237496a473
Parents: 11232b5
Author: 卫乐 <we...@taobao.com>
Authored: Wed Feb 24 21:06:25 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Wed Feb 24 21:06:25 2016 +0800

----------------------------------------------------------------------
 .../src/clj/org/apache/storm/converter.clj      |   25 +-
 .../org/apache/storm/daemon/builtin_metrics.clj |   33 +-
 .../clj/org/apache/storm/daemon/executor.clj    |   23 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |   18 +-
 .../src/clj/org/apache/storm/daemon/task.clj    |   11 +-
 storm-core/src/clj/org/apache/storm/stats.clj   | 1567 ------------------
 storm-core/src/clj/org/apache/storm/ui/core.clj |   57 +-
 .../test/clj/org/apache/storm/nimbus_test.clj   |    8 +-
 8 files changed, 84 insertions(+), 1658 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index 5599d28..6e9eeb8 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -17,8 +17,9 @@
   (:import [org.apache.storm.generated SupervisorInfo NodeInfo Assignment WorkerResources
             StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
             TopologyActionOptions DebugOptions ProfileRequest]
-           [org.apache.storm.utils Utils])
-  (:use [org.apache.storm util stats log])
+           [org.apache.storm.utils Utils]
+           [org.apache.storm.stats StatsUtil])
+  (:use [org.apache.storm util log])
   (:require [org.apache.storm.daemon [common :as common]]))
 
 (defn thriftify-supervisor-info [supervisor-info]
@@ -213,26 +214,10 @@
       (convert-to-symbol-from-status (.get_prev_status storm-base))
       (map-val clojurify-debugoptions (.get_component_debug storm-base)))))
 
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn thriftify-stats [stats]
-  (if stats
-    (map-val thriftify-executor-stats
-      (map-key #(ExecutorInfo. (int (first %1)) (int (last %1)))
-        stats))
-    {}))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn clojurify-stats [stats]
-  (if stats
-    (map-val clojurify-executor-stats
-      (map-key (fn [x] (list (.get_task_start x) (.get_task_end x)))
-        stats))
-    {}))
-
 (defn clojurify-zk-worker-hb [^ClusterWorkerHeartbeat worker-hb]
   (if worker-hb
     {:storm-id (.get_storm_id worker-hb)
-     :executor-stats (clojurify-stats (into {} (.get_executor_stats worker-hb)))
+     :executor-stats (clojurify-structure (StatsUtil/clojurifyStats (into {} (.get_executor_stats worker-hb))))
      :uptime (.get_uptime_secs worker-hb)
      :time-secs (.get_time_secs worker-hb)
      }
@@ -243,7 +228,7 @@
     (doto (ClusterWorkerHeartbeat.)
       (.set_uptime_secs (:uptime worker-hb))
       (.set_storm_id (:storm-id worker-hb))
-      (.set_executor_stats (thriftify-stats (filter second (:executor-stats worker-hb))))
+      (.set_executor_stats (StatsUtil/thriftifyStats (filter second (:executor-stats worker-hb))))
       (.set_time_secs (:time-secs worker-hb)))))
 
 (defn clojurify-error [^ErrorInfo error]

http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj b/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj
index 14d0132..caa3b71 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj
@@ -16,8 +16,7 @@
 (ns org.apache.storm.daemon.builtin-metrics
   (:import [org.apache.storm.metric.api CountMetric StateMetric IMetric IStatefulObject])
   (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
-  (:import [org.apache.storm Config])
-  (:use [org.apache.storm.stats]))
+  (:import [org.apache.storm Config]))
 
 (defrecord BuiltinSpoutMetrics [^MultiCountStatAndMetric ack-count
                                 ^MultiLatencyStatAndMetric complete-latency
@@ -38,18 +37,18 @@
 
 (defn make-data [executor-type stats]
   (condp = executor-type
-    :spout (BuiltinSpoutMetrics. (stats-acked stats)
-                                 (stats-complete-latencies stats)
-                                 (stats-failed stats)
-                                 (stats-emitted stats)
-                                 (stats-transferred stats))
-    :bolt (BuiltinBoltMetrics. (stats-acked stats)
-                               (stats-process-latencies stats)
-                               (stats-failed stats)
-                               (stats-executed stats)
-                               (stats-execute-latencies stats)
-                               (stats-emitted stats)
-                               (stats-transferred stats))))
+    :spout (BuiltinSpoutMetrics. (.getAcked stats)
+                                 (.getCompleteLatencies stats)
+                                 (.getFailed stats)
+                                 (.getEmitted stats)
+                                 (.getTransferred stats))
+    :bolt (BuiltinBoltMetrics. (.getAcked stats)
+                               (.getProcessLatencies stats)
+                               (.getFailed stats)
+                               (.getExecuted stats)
+                               (.getExecuteLatencies stats)
+                               (.getEmitted stats)
+                               (.getTransferred stats))))
 
 (defn make-spout-throttling-data []
   (SpoutThrottlingMetrics. (CountMetric.)
@@ -89,10 +88,10 @@
                      (int (get storm-conf Config/TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)))))
 
 (defn skipped-max-spout! [^SpoutThrottlingMetrics m stats]
-  (-> m .skipped-max-spout (.incrBy (stats-rate stats))))
+  (-> m .skipped-max-spout (.incrBy (.getRate stats))))
 
 (defn skipped-throttle! [^SpoutThrottlingMetrics m stats]
-  (-> m .skipped-throttle (.incrBy (stats-rate stats))))
+  (-> m .skipped-throttle (.incrBy (.getRate stats))))
 
 (defn skipped-inactive! [^SpoutThrottlingMetrics m stats]
-  (-> m .skipped-inactive (.incrBy (stats-rate stats))))
+  (-> m .skipped-inactive (.incrBy (.getRate stats))))

http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 92cc003..bca03df 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -16,8 +16,9 @@
 (ns org.apache.storm.daemon.executor
   (:use [org.apache.storm.daemon common])
   (:import [org.apache.storm.generated Grouping Grouping$_Fields]
-           [java.io Serializable])
-  (:use [org.apache.storm util config log stats])
+           [java.io Serializable]
+           [org.apache.storm.stats StatsUtil])
+  (:use [org.apache.storm util config log])
   (:import [java.util List Random HashMap ArrayList LinkedList Map])
   (:import [org.apache.storm ICredentialsListener Thrift])
   (:import [org.apache.storm.hooks ITaskHook])
@@ -41,7 +42,7 @@
            [org.json.simple JSONValue]
            [com.lmax.disruptor.dsl ProducerType]
            [org.apache.storm StormTimer])
-  (:require [org.apache.storm [cluster :as cluster] [stats :as stats]])
+  (:require [org.apache.storm [cluster :as cluster]])
   (:require [org.apache.storm.daemon [task :as task]])
   (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
   (:require [clojure.set :as set]))
@@ -407,7 +408,7 @@
     (reify
       RunningExecutor
       (render-stats [this]
-        (stats/render-stats! (:stats executor-data)))
+        (clojurify-structure (StatsUtil/renderStats (:stats executor-data))))
       (get-executor-id [this]
         executor-id)
       (credentials-changed [this creds]
@@ -447,7 +448,7 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-failed-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
+      (StatsUtil/spoutFailedTuple (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id]
   (let [storm-conf (:storm-conf executor-data)
@@ -458,7 +459,7 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-acked-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
+      (StatsUtil/spoutAckedTuple (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)
@@ -739,7 +740,7 @@
 
                                   (task/apply-hooks user-context .boltExecute (BoltExecuteInfo. tuple task-id delta))
                                   (when delta
-                                    (stats/bolt-execute-tuple! executor-stats
+                                    (StatsUtil/boltExecuteTuple executor-stats
                                                                (.getSourceComponent tuple)
                                                                (.getSourceStreamId tuple)
                                                                delta)))))))
@@ -812,7 +813,7 @@
                                                 (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                                               (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                                               (when delta
-                                                (stats/bolt-acked-tuple! executor-stats
+                                                (StatsUtil/boltAckedTuple executor-stats
                                                                          (.getSourceComponent tuple)
                                                                          (.getSourceStreamId tuple)
                                                                          delta))))
@@ -827,7 +828,7 @@
                                                 (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                                               (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                                               (when delta
-                                                (stats/bolt-failed-tuple! executor-stats
+                                                (StatsUtil/boltFailedTuple executor-stats
                                                                           (.getSourceComponent tuple)
                                                                           (.getSourceStreamId tuple)
                                                                           delta))))
@@ -862,7 +863,7 @@
 
 ;; TODO: refactor this to be part of an executor-specific map
 (defmethod mk-executor-stats :spout [_ rate]
-  (stats/mk-spout-stats rate))
+  (StatsUtil/mkSpoutStats rate))
 
 (defmethod mk-executor-stats :bolt [_ rate]
-  (stats/mk-bolt-stats rate))
+  (StatsUtil/mkBoltStats rate))

http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 28a6fb8..992a864 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -14,7 +14,8 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns org.apache.storm.daemon.nimbus
-  (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
+  (:import [org.apache.thrift.server THsHaServer THsHaServer$Args]
+           [org.apache.storm.stats StatsUtil])
   (:import [org.apache.storm.generated KeyNotFoundException])
   (:import [org.apache.storm.blobstore LocalFsBlobStore])
   (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
@@ -52,8 +53,7 @@
   (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
   (:use [org.apache.storm util config log zookeeper])
   (:require [org.apache.storm [cluster :as cluster]
-                            [converter :as converter]
-                            [stats :as stats]])
+                            [converter :as converter]])
   (:require [clojure.set :as set])
   (:import [org.apache.storm.daemon.common StormBase Assignment])
   (:import [org.apache.storm.zookeeper Zookeeper])
@@ -1668,7 +1668,7 @@
               executor->host+port (map-val (fn [[node port]]
                                              [(node->host node) port])
                                     executor->node+port)
-              nodeinfos (stats/extract-nodeinfos-from-hb-for-comp executor->host+port task->component false component_id)
+              nodeinfos (clojurify-structure (StatsUtil/extractNodeInfosFromHbForComp executor->host+port task->component false component_id))
               all-pending-actions-for-topology (.get-topology-profile-requests storm-cluster-state id true)
               latest-profile-actions (remove nil? (map (fn [nodeInfo]
                                                          (->> all-pending-actions-for-topology
@@ -1912,7 +1912,7 @@
                                               heartbeat (get beats executor)
                                               stats (:stats heartbeat)
                                               stats (if stats
-                                                      (stats/thriftify-executor-stats stats))]
+                                                      (StatsUtil/thriftifyExecutorStats stats))]
                                           (doto
                                               (ExecutorSummary. (thriftify-executor-id executor)
                                                                 (-> executor first task->component)
@@ -2106,14 +2106,14 @@
               last-err-fn (partial get-last-error
                                    (:storm-cluster-state info)
                                    topo-id)
-              topo-page-info (stats/agg-topo-execs-stats topo-id
+              ;;TODO: add last-error-fn to aggTopoExecsStats method
+              topo-page-info (StatsUtil/aggTopoExecsStats topo-id
                                                          exec->node+port
                                                          (:task->component info)
                                                          (:beats info)
                                                          (:topology info)
                                                          window
-                                                         include-sys?
-                                                         last-err-fn)]
+                                                         include-sys?)]
           (when-let [owner (:owner (:base info))]
             (.set_owner topo-page-info owner))
           (when-let [sched-status (.get @(:id->sched-status nimbus) topo-id)]
@@ -2154,7 +2154,7 @@
               executor->host+port (map-val (fn [[node port]]
                                              [(node->host node) port])
                                            executor->node+port)
-              comp-page-info (stats/agg-comp-execs-stats executor->host+port
+              comp-page-info (StatsUtil/aggCompExecsStats executor->host+port
                                                          (:task->component info)
                                                          (:beats info)
                                                          window

http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 77abdec..c9f6828 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -26,10 +26,9 @@
   (:import [org.apache.storm.utils Utils ConfigUtils])
   (:import [org.apache.storm.generated ShellComponent JavaObject])
   (:import [org.apache.storm.spout ShellSpout])
+  (:import [org.apache.storm.stats StatsUtil])
   (:import [java.util Collection List ArrayList])
   (:import [org.apache.storm Thrift])
-  (:require [org.apache.storm
-             [stats :as stats]])
   (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics]))
 
 (defn mk-topology-context-builder [worker executor-data topology]
@@ -141,9 +140,9 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (stats/emitted-tuple! executor-stats stream)
+              (StatsUtil/emittedTuple executor-stats stream)
               (if out-task-id
-                (stats/transferred-tuples! executor-stats stream 1)))
+                (StatsUtil/transferredTuples executor-stats stream, 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
@@ -163,8 +162,8 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (stats/emitted-tuple! executor-stats stream)
-               (stats/transferred-tuples! executor-stats stream (count out-tasks)))
+               (StatsUtil/emittedTuple executor-stats stream)
+               (StatsUtil/transferredTuples executor-stats stream (count out-tasks)))
              out-tasks)))
     ))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/afd2d525/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
deleted file mode 100644
index 8b37fc3..0000000
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ /dev/null
@@ -1,1567 +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.
-
-(ns org.apache.storm.stats
-  (:import [org.apache.storm.generated Nimbus Nimbus$Processor Nimbus$Iface StormTopology ShellComponent
-            NotAliveException AlreadyAliveException InvalidTopologyException GlobalStreamId
-            ClusterSummary TopologyInfo TopologySummary ExecutorInfo ExecutorSummary ExecutorStats
-            ExecutorSpecificStats SpoutStats BoltStats ErrorInfo
-            SupervisorSummary CommonAggregateStats ComponentAggregateStats
-            ComponentPageInfo ComponentType BoltAggregateStats
-            ExecutorAggregateStats SpecificAggregateStats
-            SpoutAggregateStats TopologyPageInfo TopologyStats])
-  (:import [org.apache.storm.utils Utils])
-  (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric]
-           [java.util Collection])
-  (:use [org.apache.storm log util])
-  (:use [clojure.math.numeric-tower :only [ceil]]))
-
-(def TEN-MIN-IN-SECONDS (* 10 60))
-
-(def COMMON-FIELDS [:emitted :transferred])
-(defrecord CommonStats [^MultiCountStatAndMetric emitted
-                        ^MultiCountStatAndMetric transferred
-                        rate])
-
-(def BOLT-FIELDS [:acked :failed :process-latencies :executed :execute-latencies])
-;;acked and failed count individual tuples
-(defrecord BoltExecutorStats [^CommonStats common
-                              ^MultiCountStatAndMetric acked
-                              ^MultiCountStatAndMetric failed
-                              ^MultiLatencyStatAndMetric process-latencies
-                              ^MultiCountStatAndMetric executed
-                              ^MultiLatencyStatAndMetric execute-latencies])
-
-(def SPOUT-FIELDS [:acked :failed :complete-latencies])
-;;acked and failed count tuple completion
-(defrecord SpoutExecutorStats [^CommonStats common
-                               ^MultiCountStatAndMetric acked
-                               ^MultiCountStatAndMetric failed
-                               ^MultiLatencyStatAndMetric complete-latencies])
-
-(def NUM-STAT-BUCKETS 20)
-
-(defn- div
-  "Perform floating point division on the arguments."
-  [f & rest]
-  (apply / (double f) rest))
-
-(defn- mk-common-stats
-  [rate]
-  (CommonStats.
-    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
-    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
-    rate))
-
-(defn mk-bolt-stats
-  [rate]
-  (BoltExecutorStats.
-    (mk-common-stats rate)
-    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
-    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
-    (MultiLatencyStatAndMetric. NUM-STAT-BUCKETS)
-    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
-    (MultiLatencyStatAndMetric. NUM-STAT-BUCKETS)))
-
-(defn mk-spout-stats
-  [rate]
-  (SpoutExecutorStats.
-    (mk-common-stats rate)
-    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
-    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
-    (MultiLatencyStatAndMetric. NUM-STAT-BUCKETS)))
-
-(defmacro stats-rate
-  [stats]
-  `(-> ~stats :common :rate))
-
-(defmacro stats-emitted
-  [stats]
-  `(-> ~stats :common :emitted))
-
-(defmacro stats-transferred
-  [stats]
-  `(-> ~stats :common :transferred))
-
-(defmacro stats-executed
-  [stats]
-  `(:executed ~stats))
-
-(defmacro stats-acked
-  [stats]
-  `(:acked ~stats))
-
-(defmacro stats-failed
-  [stats]
-  `(:failed ~stats))
-
-(defmacro stats-execute-latencies
-  [stats]
-  `(:execute-latencies ~stats))
-
-(defmacro stats-process-latencies
-  [stats]
-  `(:process-latencies ~stats))
-
-(defmacro stats-complete-latencies
-  [stats]
-  `(:complete-latencies ~stats))
-
-(defn emitted-tuple!
-  [stats stream]
-  (.incBy ^MultiCountStatAndMetric (stats-emitted stats) ^Object stream ^long (stats-rate stats)))
-
-(defn transferred-tuples!
-  [stats stream amt]
-  (.incBy ^MultiCountStatAndMetric (stats-transferred stats) ^Object stream ^long (* (stats-rate stats) amt)))
-
-(defn bolt-execute-tuple!
-  [^BoltExecutorStats stats component stream latency-ms]
-  (let [key [component stream]
-        ^MultiCountStatAndMetric executed (stats-executed stats)
-        ^MultiLatencyStatAndMetric exec-lat (stats-execute-latencies stats)]
-    (.incBy executed key (stats-rate stats))
-    (.record exec-lat key latency-ms)))
-
-(defn bolt-acked-tuple!
-  [^BoltExecutorStats stats component stream latency-ms]
-  (let [key [component stream]
-        ^MultiCountStatAndMetric acked (stats-acked stats)
-        ^MultiLatencyStatAndMetric process-lat (stats-process-latencies stats)]
-    (.incBy acked key (stats-rate stats))
-    (.record process-lat key latency-ms)))
-
-(defn bolt-failed-tuple!
-  [^BoltExecutorStats stats component stream latency-ms]
-  (let [key [component stream]
-        ^MultiCountStatAndMetric failed (stats-failed stats)]
-    (.incBy failed key (stats-rate stats))))
-
-(defn spout-acked-tuple!
-  [^SpoutExecutorStats stats stream latency-ms]
-  (.incBy ^MultiCountStatAndMetric (stats-acked stats) stream (stats-rate stats))
-  (.record ^MultiLatencyStatAndMetric (stats-complete-latencies stats) stream latency-ms))
-
-(defn spout-failed-tuple!
-  [^SpoutExecutorStats stats stream latency-ms]
-  (.incBy ^MultiCountStatAndMetric (stats-failed stats) stream (stats-rate stats)))
-
-(defn- cleanup-stat! [stat]
-  (.close stat))
-
-(defn- cleanup-common-stats!
-  [^CommonStats stats]
-  (doseq [f COMMON-FIELDS]
-    (cleanup-stat! (f stats))))
-
-(defn cleanup-bolt-stats!
-  [^BoltExecutorStats stats]
-  (cleanup-common-stats! (:common stats))
-  (doseq [f BOLT-FIELDS]
-    (cleanup-stat! (f stats))))
-
-(defn cleanup-spout-stats!
-  [^SpoutExecutorStats stats]
-  (cleanup-common-stats! (:common stats))
-  (doseq [f SPOUT-FIELDS]
-    (cleanup-stat! (f stats))))
-
-(defn- value-stats
-  [stats fields]
-  (into {} (dofor [f fields]
-                  [f (if (instance? MultiCountStatAndMetric (f stats))
-                         (.getTimeCounts ^MultiCountStatAndMetric (f stats))
-                         (.getTimeLatAvg ^MultiLatencyStatAndMetric (f stats)))])))
-
-(defn- value-common-stats
-  [^CommonStats stats]
-  (merge
-    (value-stats stats COMMON-FIELDS)
-    {:rate (:rate stats)}))
-
-(defn value-bolt-stats!
-  [^BoltExecutorStats stats]
-  (cleanup-bolt-stats! stats)
-  (merge (value-common-stats (:common stats))
-         (value-stats stats BOLT-FIELDS)
-         {:type :bolt}))
-
-(defn value-spout-stats!
-  [^SpoutExecutorStats stats]
-  (cleanup-spout-stats! stats)
-  (merge (value-common-stats (:common stats))
-         (value-stats stats SPOUT-FIELDS)
-         {:type :spout}))
-
-(defn- class-selector
-  [obj & args]
-  (class obj))
-
-(defmulti render-stats! class-selector)
-
-(defmethod render-stats! SpoutExecutorStats
-  [stats]
-  (value-spout-stats! stats))
-
-(defmethod render-stats! BoltExecutorStats
-  [stats]
-  (value-bolt-stats! stats))
-
-(defmulti thriftify-specific-stats :type)
-(defmulti clojurify-specific-stats class-selector)
-
-(defn window-set-converter
-  ([stats key-fn first-key-fun]
-    (into {}
-      (for [[k v] stats]
-        ;apply the first-key-fun only to first key.
-        [(first-key-fun k)
-         (into {} (for [[k2 v2] v]
-                    [(key-fn k2) v2]))])))
-  ([stats first-key-fun]
-    (window-set-converter stats identity first-key-fun)))
-
-(defn to-global-stream-id
-  [[component stream]]
-  (GlobalStreamId. component stream))
-
-(defn from-global-stream-id [global-stream-id]
-  [(.get_componentId global-stream-id) (.get_streamId global-stream-id)])
-
-(defmethod clojurify-specific-stats BoltStats [^BoltStats stats]
-  [(window-set-converter (.get_acked stats) from-global-stream-id identity)
-   (window-set-converter (.get_failed stats) from-global-stream-id identity)
-   (window-set-converter (.get_process_ms_avg stats) from-global-stream-id identity)
-   (window-set-converter (.get_executed stats) from-global-stream-id identity)
-   (window-set-converter (.get_execute_ms_avg stats) from-global-stream-id identity)])
-
-(defmethod clojurify-specific-stats SpoutStats [^SpoutStats stats]
-  [(.get_acked stats)
-   (.get_failed stats)
-   (.get_complete_ms_avg stats)])
-
-
-(defn clojurify-executor-stats
-  [^ExecutorStats stats]
-  (let [ specific-stats (.get_specific stats)
-         is_bolt? (.is_set_bolt specific-stats)
-         specific-stats (if is_bolt? (.get_bolt specific-stats) (.get_spout specific-stats))
-         specific-stats (clojurify-specific-stats specific-stats)
-         common-stats (CommonStats. (.get_emitted stats)
-                                    (.get_transferred stats)
-                                    (.get_rate stats))]
-    (if is_bolt?
-      ; worker heart beat does not store the BoltExecutorStats or SpoutExecutorStats , instead it stores the result returned by render-stats!
-      ; which flattens the BoltExecutorStats/SpoutExecutorStats by extracting values from all atoms and merging all values inside :common to top
-      ;level map we are pretty much doing the same here.
-      (dissoc (merge common-stats {:type :bolt}  (apply ->BoltExecutorStats (into [nil] specific-stats))) :common)
-      (dissoc (merge common-stats {:type :spout} (apply ->SpoutExecutorStats (into [nil] specific-stats))) :common)
-      )))
-
-(defmethod thriftify-specific-stats :bolt
-  [stats]
-  (ExecutorSpecificStats/bolt
-    (BoltStats.
-      (window-set-converter (:acked stats) to-global-stream-id str)
-      (window-set-converter (:failed stats) to-global-stream-id str)
-      (window-set-converter (:process-latencies stats) to-global-stream-id str)
-      (window-set-converter (:executed stats) to-global-stream-id str)
-      (window-set-converter (:execute-latencies stats) to-global-stream-id str))))
-
-(defmethod thriftify-specific-stats :spout
-  [stats]
-  (ExecutorSpecificStats/spout
-    (SpoutStats. (window-set-converter (:acked stats) str)
-      (window-set-converter (:failed stats) str)
-      (window-set-converter (:complete-latencies stats) str))))
-
-(defn thriftify-executor-stats
-  [stats]
-  (let [specific-stats (thriftify-specific-stats stats)
-        rate (:rate stats)]
-    (ExecutorStats. (window-set-converter (:emitted stats) str)
-      (window-set-converter (:transferred stats) str)
-      specific-stats
-      rate)))
-
-(defn valid-number?
-  "Returns true if x is a number that is not NaN or Infinity, false otherwise"
-  [x]
-  (and (number? x)
-       (not (Double/isNaN x))
-       (not (Double/isInfinite x))))
-
-(defn apply-default
-  [f defaulting-fn & args]
-  (apply f (map defaulting-fn args)))
-
-(defn apply-or-0
-  [f & args]
-  (apply apply-default
-         f
-         #(if (valid-number? %) % 0)
-         args))
-
-(defn sum-or-0
-  [& args]
-  (apply apply-or-0 + args))
-
-(defn product-or-0
-  [& args]
-  (apply apply-or-0 * args))
-
-(defn max-or-0
-  [& args]
-  (apply apply-or-0 max args))
-
-(defn- agg-bolt-lat-and-count
-  "Aggregates number executed, process latency, and execute latency across all
-  streams."
-  [idk->exec-avg idk->proc-avg idk->num-executed]
-  (letfn [(weight-avg [[id avg]]
-            (let [num-e (get idk->num-executed id)]
-              (product-or-0 avg num-e)))]
-    {:executeLatencyTotal (reduce + (map weight-avg idk->exec-avg))
-     :processLatencyTotal (reduce + (map weight-avg idk->proc-avg))
-     :executed (reduce + (vals idk->num-executed))}))
-
-(defn- agg-spout-lat-and-count
-  "Aggregates number acked and complete latencies across all streams."
-  [sid->comp-avg sid->num-acked]
-  (letfn [(weight-avg [[id avg]]
-            (product-or-0 avg (get sid->num-acked id)))]
-    {:completeLatencyTotal (reduce + (map weight-avg sid->comp-avg))
-     :acked (reduce + (vals sid->num-acked))}))
-
-(defn add-pairs
-  ([] [0 0])
-  ([[a1 a2] [b1 b2]]
-   [(+ a1 b1) (+ a2 b2)]))
-
-(defn mk-include-sys-fn
-  [include-sys?]
-  (if include-sys?
-    (fn [_] true)
-    (fn [stream] (and (string? stream) (not (Utils/isSystemId stream))))))
-
-;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
-(defn mk-include-sys-filter
-  "Returns a function that includes or excludes map entries whose keys are
-  system ids."
-  [include-sys?]
-  (if include-sys?
-    identity
-    (partial filter-key (mk-include-sys-fn false))))
-
-(defn- agg-bolt-streams-lat-and-count
-  "Aggregates number executed and process & execute latencies."
-  [idk->exec-avg idk->proc-avg idk->executed]
-  (letfn [(weight-avg [id avg]
-            (let [num-e (idk->executed id)]
-              (product-or-0 avg num-e)))]
-    (into {}
-      (for [k (keys idk->exec-avg)]
-        [k {:executeLatencyTotal (weight-avg k (get idk->exec-avg k))
-            :processLatencyTotal (weight-avg k (get idk->proc-avg k))
-            :executed (idk->executed k)}]))))
-
-(defn- agg-spout-streams-lat-and-count
-  "Aggregates number acked and complete latencies."
-  [idk->comp-avg idk->acked]
-  (letfn [(weight-avg [id avg]
-            (let [num-e (get idk->acked id)]
-              (product-or-0 avg num-e)))]
-    (into {}
-      (for [k (keys idk->comp-avg)]
-        [k {:completeLatencyTotal (weight-avg k (get idk->comp-avg k))
-            :acked (get idk->acked k)}]))))
-
-(defn swap-map-order
-  "For a nested map, rearrange data such that the top-level keys become the
-  nested map's keys and vice versa.
-  Example:
-  {:a {:X :banana, :Y :pear}, :b {:X :apple, :Y :orange}}
-  -> {:Y {:a :pear, :b :orange}, :X {:a :banana, :b :apple}}"
-  [m]
-  (apply merge-with
-         merge
-         (map (fn [[k v]]
-                (into {}
-                      (for [[k2 v2] v]
-                        [k2 {k v2}])))
-              m)))
-
-(defn- compute-agg-capacity
-  "Computes the capacity metric for one executor given its heartbeat data and
-  uptime."
-  [m uptime]
-  (when uptime
-    (->>
-      ;; For each stream, create weighted averages and counts.
-      (merge-with (fn weighted-avg+count-fn
-                    [avg cnt]
-                    [(* avg cnt) cnt])
-                  (get (:execute-latencies m) (str TEN-MIN-IN-SECONDS))
-                  (get (:executed m) (str TEN-MIN-IN-SECONDS)))
-      vals ;; Ignore the stream ids.
-      (reduce add-pairs
-              [0. 0]) ;; Combine weighted averages and counts.
-      ((fn [[weighted-avg cnt]]
-        (div weighted-avg (* 1000 (min uptime TEN-MIN-IN-SECONDS))))))))
-
-(defn agg-pre-merge-comp-page-bolt
-  [{exec-id :exec-id
-    host :host
-    port :port
-    uptime :uptime
-    comp-id :comp-id
-    num-tasks :num-tasks
-    statk->w->sid->num :stats}
-   window
-   include-sys?]
-  ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-  (let [str-key (partial map-key str)
-        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
-    {:executor-id exec-id,
-     :host host,
-     :port port,
-     :uptime uptime,
-     :num-executors 1,
-     :num-tasks num-tasks,
-     :capacity (compute-agg-capacity statk->w->sid->num uptime)
-     :cid+sid->input-stats
-     (merge-with
-       merge
-       (swap-map-order
-         {:acked (-> statk->w->sid->num
-                     :acked
-                     str-key
-                     (get window))
-          :failed (-> statk->w->sid->num
-                      :failed
-                      str-key
-                      (get window))})
-       (agg-bolt-streams-lat-and-count (-> statk->w->sid->num
-                                           :execute-latencies
-                                           str-key
-                                           (get window))
-                                       (-> statk->w->sid->num
-                                           :process-latencies
-                                           str-key
-                                           (get window))
-                                       (-> statk->w->sid->num
-                                           :executed
-                                           str-key
-                                           (get window)))),
-     :sid->output-stats
-     (swap-map-order
-       {:emitted (-> statk->w->sid->num
-                     :emitted
-                     str-key
-                     (get window)
-                     handle-sys-components-fn)
-        :transferred (-> statk->w->sid->num
-                         :transferred
-                         str-key
-                         (get window)
-                         handle-sys-components-fn)})}))
-
-(defn agg-pre-merge-comp-page-spout
-  [{exec-id :exec-id
-    host :host
-    port :port
-    uptime :uptime
-    comp-id :comp-id
-    num-tasks :num-tasks
-    statk->w->sid->num :stats}
-   window
-   include-sys?]
-  ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-  (let [str-key (partial map-key str)
-        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
-    {:executor-id exec-id,
-     :host host,
-     :port port,
-     :uptime uptime,
-     :num-executors 1,
-     :num-tasks num-tasks,
-     :sid->output-stats
-     (merge-with
-       merge
-       (agg-spout-streams-lat-and-count (-> statk->w->sid->num
-                                            :complete-latencies
-                                            str-key
-                                            (get window))
-                                        (-> statk->w->sid->num
-                                            :acked
-                                            str-key
-                                            (get window)))
-       (swap-map-order
-         {:acked (-> statk->w->sid->num
-                     :acked
-                     str-key
-                     (get window))
-          :failed (-> statk->w->sid->num
-                      :failed
-                      str-key
-                      (get window))
-          :emitted (-> statk->w->sid->num
-                       :emitted
-                       str-key
-                       (get window)
-                       handle-sys-components-fn)
-          :transferred (-> statk->w->sid->num
-                           :transferred
-                           str-key
-                           (get window)
-                           handle-sys-components-fn)}))}))
-
-(defn agg-pre-merge-topo-page-bolt
-  [{comp-id :comp-id
-    num-tasks :num-tasks
-    statk->w->sid->num :stats
-    uptime :uptime}
-   window
-   include-sys?]
-  ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-  (let [str-key (partial map-key str)
-        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
-    {comp-id
-     (merge
-       (agg-bolt-lat-and-count (-> statk->w->sid->num
-                                   :execute-latencies
-                                   str-key
-                                   (get window))
-                               (-> statk->w->sid->num
-                                   :process-latencies
-                                   str-key
-                                   (get window))
-                               (-> statk->w->sid->num
-                                   :executed
-                                   str-key
-                                   (get window)))
-       {:num-executors 1
-        :num-tasks num-tasks
-        :emitted (-> statk->w->sid->num
-                     :emitted
-                     str-key
-                     (get window)
-                     handle-sys-components-fn
-                     vals
-                     (#(reduce + %)))
-        :transferred (-> statk->w->sid->num
-                         :transferred
-                         str-key
-                         (get window)
-                         handle-sys-components-fn
-                         vals
-                         (#(reduce + %)))
-        :capacity (compute-agg-capacity statk->w->sid->num uptime)
-        :acked (-> statk->w->sid->num
-                   :acked
-                   str-key
-                   (get window)
-                   vals
-                   (#(reduce + %)))
-        :failed (-> statk->w->sid->num
-                    :failed
-                    str-key
-                    (get window)
-                    vals
-                    (#(reduce + %)))})}))
-
-(defn agg-pre-merge-topo-page-spout
-  [{comp-id :comp-id
-    num-tasks :num-tasks
-    statk->w->sid->num :stats}
-   window
-   include-sys?]
-  ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-  (let [str-key (partial map-key str)
-        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
-    {comp-id
-     (merge
-       (agg-spout-lat-and-count (-> statk->w->sid->num
-                                    :complete-latencies
-                                    str-key
-                                    (get window))
-                                (-> statk->w->sid->num
-                                    :acked
-                                    str-key
-                                    (get window)))
-       {:num-executors 1
-        :num-tasks num-tasks
-        :emitted (-> statk->w->sid->num
-                     :emitted
-                     str-key
-                     (get window)
-                     handle-sys-components-fn
-                     vals
-                     (#(reduce + %)))
-        :transferred (-> statk->w->sid->num
-                         :transferred
-                         str-key
-                         (get window)
-                         handle-sys-components-fn
-                         vals
-                         (#(reduce + %)))
-        :failed (-> statk->w->sid->num
-                    :failed
-                    str-key
-                    (get window)
-                    vals
-                    (#(reduce + %)))})}))
-
-(defn merge-agg-comp-stats-comp-page-bolt
-  [{acc-in :cid+sid->input-stats
-    acc-out :sid->output-stats
-    :as acc-bolt-stats}
-   {bolt-in :cid+sid->input-stats
-    bolt-out :sid->output-stats
-    :as bolt-stats}]
-  {:num-executors (inc (or (:num-executors acc-bolt-stats) 0)),
-   :num-tasks (sum-or-0 (:num-tasks acc-bolt-stats) (:num-tasks bolt-stats)),
-   :sid->output-stats (merge-with (partial merge-with sum-or-0)
-                                  acc-out
-                                  bolt-out),
-   :cid+sid->input-stats (merge-with (partial merge-with sum-or-0)
-                                     acc-in
-                                     bolt-in),
-   :executor-stats
-   (let [sum-streams (fn [m k] (->> m vals (map k) (apply sum-or-0)))
-         executed (sum-streams bolt-in :executed)]
-     (conj (:executor-stats acc-bolt-stats)
-           (merge
-             (select-keys bolt-stats
-                          [:executor-id :uptime :host :port :capacity])
-             {:emitted (sum-streams bolt-out :emitted)
-              :transferred (sum-streams bolt-out :transferred)
-              :acked (sum-streams bolt-in :acked)
-              :failed (sum-streams bolt-in :failed)
-              :executed executed}
-             (->>
-               (if (and executed (pos? executed))
-                 [(div (sum-streams bolt-in :executeLatencyTotal) executed)
-                  (div (sum-streams bolt-in :processLatencyTotal) executed)]
-                 [nil nil])
-               (mapcat vector [:execute-latency :process-latency])
-               (apply assoc {})))))})
-
-(defn merge-agg-comp-stats-comp-page-spout
-  [{acc-out :sid->output-stats
-    :as acc-spout-stats}
-   {spout-out :sid->output-stats
-    :as spout-stats}]
-  {:num-executors (inc (or (:num-executors acc-spout-stats) 0)),
-   :num-tasks (sum-or-0 (:num-tasks acc-spout-stats) (:num-tasks spout-stats)),
-   :sid->output-stats (merge-with (partial merge-with sum-or-0)
-                                  acc-out
-                                  spout-out),
-   :executor-stats
-   (let [sum-streams (fn [m k] (->> m vals (map k) (apply sum-or-0)))
-         acked (sum-streams spout-out :acked)]
-     (conj (:executor-stats acc-spout-stats)
-           (merge
-             (select-keys spout-stats [:executor-id :uptime :host :port])
-             {:emitted (sum-streams spout-out :emitted)
-              :transferred (sum-streams spout-out :transferred)
-              :acked acked
-              :failed (sum-streams spout-out :failed)}
-             {:complete-latency (if (and acked (pos? acked))
-                                  (div (sum-streams spout-out
-                                                    :completeLatencyTotal)
-                                       acked)
-                                  nil)})))})
-
-(defn merge-agg-comp-stats-topo-page-bolt
-  [acc-bolt-stats bolt-stats]
-  {:num-executors (inc (or (:num-executors acc-bolt-stats) 0))
-   :num-tasks (sum-or-0 (:num-tasks acc-bolt-stats) (:num-tasks bolt-stats))
-   :emitted (sum-or-0 (:emitted acc-bolt-stats) (:emitted bolt-stats))
-   :transferred (sum-or-0 (:transferred acc-bolt-stats)
-                          (:transferred bolt-stats))
-   :capacity (max-or-0 (:capacity acc-bolt-stats) (:capacity bolt-stats))
-   ;; We sum average latency totals here to avoid dividing at each step.
-   ;; Compute the average latencies by dividing the total by the count.
-   :executeLatencyTotal (sum-or-0 (:executeLatencyTotal acc-bolt-stats)
-                                  (:executeLatencyTotal bolt-stats))
-   :processLatencyTotal (sum-or-0 (:processLatencyTotal acc-bolt-stats)
-                                  (:processLatencyTotal bolt-stats))
-   :executed (sum-or-0 (:executed acc-bolt-stats) (:executed bolt-stats))
-   :acked (sum-or-0 (:acked acc-bolt-stats) (:acked bolt-stats))
-   :failed (sum-or-0 (:failed acc-bolt-stats) (:failed bolt-stats))})
-
-(defn merge-agg-comp-stats-topo-page-spout
-  [acc-spout-stats spout-stats]
-  {:num-executors (inc (or (:num-executors acc-spout-stats) 0))
-   :num-tasks (sum-or-0 (:num-tasks acc-spout-stats) (:num-tasks spout-stats))
-   :emitted (sum-or-0 (:emitted acc-spout-stats) (:emitted spout-stats))
-   :transferred (sum-or-0 (:transferred acc-spout-stats) (:transferred spout-stats))
-   ;; We sum average latency totals here to avoid dividing at each step.
-   ;; Compute the average latencies by dividing the total by the count.
-   :completeLatencyTotal (sum-or-0 (:completeLatencyTotal acc-spout-stats)
-                            (:completeLatencyTotal spout-stats))
-   :acked (sum-or-0 (:acked acc-spout-stats) (:acked spout-stats))
-   :failed (sum-or-0 (:failed acc-spout-stats) (:failed spout-stats))})
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn aggregate-count-streams
-  [stats]
-  (->> stats
-    (map-val #(reduce + (vals %)))))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn- agg-topo-exec-stats*
-  "A helper function that does the common work to aggregate stats of one
-  executor with the given map for the topology page."
-  [window
-   include-sys?
-   {:keys [workers-set
-           bolt-id->stats
-           spout-id->stats
-           window->emitted
-           window->transferred
-           window->comp-lat-wgt-avg
-           window->acked
-           window->failed] :as acc-stats}
-   {:keys [stats] :as new-data}
-   pre-merge-fn
-   merge-fn
-   comp-key]
-  (let [cid->statk->num (pre-merge-fn new-data window include-sys?)
-        {w->compLatWgtAvg :completeLatencyTotal
-         w->acked :acked}
-          (if (:complete-latencies stats)
-            (swap-map-order
-              (into {}
-                    (for [w (keys (:acked stats))]
-                         [w (agg-spout-lat-and-count
-                              (get (:complete-latencies stats) w)
-                              (get (:acked stats) w))])))
-            {:completeLatencyTotal nil
-             :acks (aggregate-count-streams (:acked stats))})
-        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
-    (assoc {:workers-set (conj workers-set
-                               [(:host new-data) (:port new-data)])
-            :bolt-id->stats bolt-id->stats
-            :spout-id->stats spout-id->stats
-            :window->emitted (->> (:emitted stats)
-                                  (map-val handle-sys-components-fn)
-                                  aggregate-count-streams
-                                  (merge-with + window->emitted))
-            :window->transferred (->> (:transferred stats)
-                                      (map-val handle-sys-components-fn)
-                                      aggregate-count-streams
-                                      (merge-with + window->transferred))
-            :window->comp-lat-wgt-avg (merge-with +
-                                                  window->comp-lat-wgt-avg
-                                                  w->compLatWgtAvg)
-            :window->acked (if (= :spout (:type stats))
-                             (merge-with + window->acked w->acked)
-                             window->acked)
-            :window->failed (if (= :spout (:type stats))
-                              (->> (:failed stats)
-                                   aggregate-count-streams
-                                   (merge-with + window->failed))
-                              window->failed)}
-           comp-key (merge-with merge-fn
-                                (acc-stats comp-key)
-                                cid->statk->num)
-           :type (:type stats))))
-
-(defmulti agg-topo-exec-stats
-  "Combines the aggregate stats of one executor with the given map, selecting
-  the appropriate window and including system components as specified."
-  (fn dispatch-fn [& args] (:type (last args))))
-
-(defmethod agg-topo-exec-stats :bolt
-  [window include-sys? acc-stats new-data]
-  (agg-topo-exec-stats* window
-                        include-sys?
-                        acc-stats
-                        new-data
-                        agg-pre-merge-topo-page-bolt
-                        merge-agg-comp-stats-topo-page-bolt
-                        :bolt-id->stats))
-
-(defmethod agg-topo-exec-stats :spout
-  [window include-sys? acc-stats new-data]
-  (agg-topo-exec-stats* window
-                        include-sys?
-                        acc-stats
-                        new-data
-                        agg-pre-merge-topo-page-spout
-                        merge-agg-comp-stats-topo-page-spout
-                        :spout-id->stats))
-
-(defmethod agg-topo-exec-stats :default [_ _ acc-stats _] acc-stats)
-
-(defn get-last-error
-  [storm-cluster-state storm-id component-id]
-  (if-let [e (.last-error storm-cluster-state storm-id component-id)]
-    (ErrorInfo. (:error e) (:time-secs e))))
-
-(defn component-type
-  "Returns the component type (either :bolt or :spout) for a given
-  topology and component id. Returns nil if not found."
-  [^StormTopology topology id]
-  (let [bolts (.get_bolts topology)
-        spouts (.get_spouts topology)]
-    (cond
-      (Utils/isSystemId id) :bolt
-      (.containsKey bolts id) :bolt
-      (.containsKey spouts id) :spout)))
-
-(defn extract-nodeinfos-from-hb-for-comp
-  ([exec->host+port task->component include-sys? comp-id]
-   (distinct (for [[[start end :as executor] [host port]] exec->host+port
-         :let [id (task->component start)]
-         :when (and (or (nil? comp-id) (= comp-id id))
-                 (or include-sys? (not (Utils/isSystemId id))))]
-     {:host host
-      :port port}))))
-
-(defn extract-data-from-hb
-  ([exec->host+port task->component beats include-sys? topology comp-id]
-   (for [[[start end :as executor] [host port]] exec->host+port
-         :let [beat (beats executor)
-               id (task->component start)]
-         :when (and (or (nil? comp-id) (= comp-id id))
-                    (or include-sys? (not (Utils/isSystemId id))))]
-     {:exec-id executor
-      :comp-id id
-      :num-tasks (count (range start (inc end)))
-      :host host
-      :port port
-      :uptime (:uptime beat)
-      :stats (:stats beat)
-      :type (or (:type (:stats beat))
-                (component-type topology id))}))
-  ([exec->host+port task->component beats include-sys? topology]
-    (extract-data-from-hb exec->host+port
-                          task->component
-                          beats
-                          include-sys?
-                          topology
-                          nil)))
-
-(defn aggregate-topo-stats
-  [window include-sys? data]
-  (let [init-val {:workers-set #{}
-                  :bolt-id->stats {}
-                  :spout-id->stats {}
-                  :window->emitted {}
-                  :window->transferred {}
-                  :window->comp-lat-wgt-avg {}
-                  :window->acked {}
-                  :window->failed {}}
-        reducer-fn (partial agg-topo-exec-stats
-                            window
-                            include-sys?)]
-    (reduce reducer-fn init-val data)))
-
-(defn- compute-weighted-averages-per-window
-  [acc-data wgt-avg-key divisor-key]
-  (into {} (for [[window wgt-avg] (wgt-avg-key acc-data)
-                 :let [divisor ((divisor-key acc-data) window)]
-                 :when (and divisor (pos? divisor))]
-             [(str window) (div wgt-avg divisor)])))
-
-(defn- post-aggregate-topo-stats
-  [task->component exec->node+port last-err-fn acc-data]
-  {:num-tasks (count task->component)
-   :num-workers (count (:workers-set acc-data))
-   :num-executors (count exec->node+port)
-   :bolt-id->stats
-     (into {} (for [[id m] (:bolt-id->stats acc-data)
-                    :let [executed (:executed m)]]
-                     [id (-> m
-                             (assoc :execute-latency
-                                    (if (and executed (pos? executed))
-                                      (div (or (:executeLatencyTotal m) 0)
-                                           executed)
-                                      0)
-                                    :process-latency
-                                    (if (and executed (pos? executed))
-                                      (div (or (:processLatencyTotal m) 0)
-                                           executed)
-                                      0))
-                             (dissoc :executeLatencyTotal
-                                     :processLatencyTotal)
-                             (assoc :lastError (last-err-fn id)))]))
-   :spout-id->stats
-     (into {} (for [[id m] (:spout-id->stats acc-data)
-                    :let [acked (:acked m)]]
-                    [id (-> m
-                            (assoc :complete-latency
-                                   (if (and acked (pos? acked))
-                                     (div (:completeLatencyTotal m)
-                                          (:acked m))
-                                     0))
-                            (dissoc :completeLatencyTotal)
-                            (assoc :lastError (last-err-fn id)))]))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->emitted (map-key str (:window->emitted acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->transferred (map-key str (:window->transferred acc-data))
-   :window->complete-latency
-     (compute-weighted-averages-per-window acc-data
-                                           :window->comp-lat-wgt-avg
-                                           :window->acked)
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->acked (map-key str (:window->acked acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->failed (map-key str (:window->failed acc-data))})
-
-(defn- thriftify-common-agg-stats
-  [^ComponentAggregateStats s
-   {:keys [num-tasks
-           emitted
-           transferred
-           acked
-           failed
-           num-executors] :as statk->num}]
-  (let [cas (CommonAggregateStats.)]
-    (and num-executors (.set_num_executors cas num-executors))
-    (and num-tasks (.set_num_tasks cas num-tasks))
-    (and emitted (.set_emitted cas emitted))
-    (and transferred (.set_transferred cas transferred))
-    (and acked (.set_acked cas acked))
-    (and failed (.set_failed cas failed))
-    (.set_common_stats s cas)))
-
-(defn thriftify-bolt-agg-stats
-  [statk->num]
-  (let [{:keys [lastError
-                execute-latency
-                process-latency
-                executed
-                capacity]} statk->num
-        s (ComponentAggregateStats.)]
-    (.set_type s ComponentType/BOLT)
-    (and lastError (.set_last_error s lastError))
-    (thriftify-common-agg-stats s statk->num)
-    (.set_specific_stats s
-      (SpecificAggregateStats/bolt
-        (let [bas (BoltAggregateStats.)]
-          (and execute-latency (.set_execute_latency_ms bas execute-latency))
-          (and process-latency (.set_process_latency_ms bas process-latency))
-          (and executed (.set_executed bas executed))
-          (and capacity (.set_capacity bas capacity))
-          bas)))
-    s))
-
-(defn thriftify-spout-agg-stats
-  [statk->num]
-  (let [{:keys [lastError
-                complete-latency]} statk->num
-        s (ComponentAggregateStats.)]
-    (.set_type s ComponentType/SPOUT)
-    (and lastError (.set_last_error s lastError))
-    (thriftify-common-agg-stats s statk->num)
-    (.set_specific_stats s
-      (SpecificAggregateStats/spout
-        (let [sas (SpoutAggregateStats.)]
-          (and complete-latency (.set_complete_latency_ms sas complete-latency))
-          sas)))
-    s))
-
-(defn thriftify-topo-page-data
-  [topology-id data]
-  (let [{:keys [num-tasks
-                num-workers
-                num-executors
-                spout-id->stats
-                bolt-id->stats
-                window->emitted
-                window->transferred
-                window->complete-latency
-                window->acked
-                window->failed]} data
-        spout-agg-stats (into {}
-                              (for [[id m] spout-id->stats
-                                    :let [m (assoc m :type :spout)]]
-                                [id
-                                 (thriftify-spout-agg-stats m)]))
-        bolt-agg-stats (into {}
-                             (for [[id m] bolt-id->stats
-                                   :let [m (assoc m :type :bolt)]]
-                              [id
-                               (thriftify-bolt-agg-stats m)]))
-        topology-stats (doto (TopologyStats.)
-                         (.set_window_to_emitted window->emitted)
-                         (.set_window_to_transferred window->transferred)
-                         (.set_window_to_complete_latencies_ms
-                           window->complete-latency)
-                         (.set_window_to_acked window->acked)
-                         (.set_window_to_failed window->failed))
-      topo-page-info (doto (TopologyPageInfo. topology-id)
-                       (.set_num_tasks num-tasks)
-                       (.set_num_workers num-workers)
-                       (.set_num_executors num-executors)
-                       (.set_id_to_spout_agg_stats spout-agg-stats)
-                       (.set_id_to_bolt_agg_stats bolt-agg-stats)
-                       (.set_topology_stats topology-stats))]
-    topo-page-info))
-
-(defn agg-topo-execs-stats
-  "Aggregate various executor statistics for a topology from the given
-  heartbeats."
-  [topology-id
-   exec->node+port
-   task->component
-   beats
-   topology
-   window
-   include-sys?
-   last-err-fn]
-  (->> ;; This iterates over each executor one time, because of lazy evaluation.
-    (extract-data-from-hb exec->node+port
-                          task->component
-                          beats
-                          include-sys?
-                          topology)
-    (aggregate-topo-stats window include-sys?)
-    (post-aggregate-topo-stats task->component exec->node+port last-err-fn)
-    (thriftify-topo-page-data topology-id)))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn- agg-bolt-exec-win-stats
-  "A helper function that aggregates windowed stats from one bolt executor."
-  [acc-stats new-stats include-sys?]
-  (let [{w->execLatWgtAvg :executeLatencyTotal
-         w->procLatWgtAvg :processLatencyTotal
-         w->executed :executed}
-          (swap-map-order
-            (into {} (for [w (keys (:executed new-stats))]
-                       [w (agg-bolt-lat-and-count
-                            (get (:execute-latencies new-stats) w)
-                            (get (:process-latencies new-stats) w)
-                            (get (:executed new-stats) w))])))
-        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
-    {:window->emitted (->> (:emitted new-stats)
-                           (map-val handle-sys-components-fn)
-                           aggregate-count-streams
-                           (merge-with + (:window->emitted acc-stats)))
-     :window->transferred (->> (:transferred new-stats)
-                               (map-val handle-sys-components-fn)
-                               aggregate-count-streams
-                               (merge-with + (:window->transferred acc-stats)))
-     :window->exec-lat-wgt-avg (merge-with +
-                                           (:window->exec-lat-wgt-avg acc-stats)
-                                           w->execLatWgtAvg)
-     :window->proc-lat-wgt-avg (merge-with +
-                                           (:window->proc-lat-wgt-avg acc-stats)
-                                           w->procLatWgtAvg)
-     :window->executed (merge-with + (:window->executed acc-stats) w->executed)
-     :window->acked (->> (:acked new-stats)
-                         aggregate-count-streams
-                         (merge-with + (:window->acked acc-stats)))
-     :window->failed (->> (:failed new-stats)
-                          aggregate-count-streams
-                          (merge-with + (:window->failed acc-stats)))}))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn- agg-spout-exec-win-stats
-  "A helper function that aggregates windowed stats from one spout executor."
-  [acc-stats new-stats include-sys?]
-  (let [{w->compLatWgtAvg :completeLatencyTotal
-         w->acked :acked}
-          (swap-map-order
-            (into {} (for [w (keys (:acked new-stats))]
-                       [w (agg-spout-lat-and-count
-                            (get (:complete-latencies new-stats) w)
-                            (get (:acked new-stats) w))])))
-        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
-    {:window->emitted (->> (:emitted new-stats)
-                           (map-val handle-sys-components-fn)
-                           aggregate-count-streams
-                           (merge-with + (:window->emitted acc-stats)))
-     :window->transferred (->> (:transferred new-stats)
-                               (map-val handle-sys-components-fn)
-                               aggregate-count-streams
-                               (merge-with + (:window->transferred acc-stats)))
-     :window->comp-lat-wgt-avg (merge-with +
-                                           (:window->comp-lat-wgt-avg acc-stats)
-                                           w->compLatWgtAvg)
-     :window->acked (->> (:acked new-stats)
-                         aggregate-count-streams
-                         (merge-with + (:window->acked acc-stats)))
-     :window->failed (->> (:failed new-stats)
-                          aggregate-count-streams
-                          (merge-with + (:window->failed acc-stats)))}))
-
-(defmulti agg-comp-exec-stats
-  "Combines the aggregate stats of one executor with the given map, selecting
-  the appropriate window and including system components as specified."
-  (fn dispatch-fn [_ _ init-val _] (:type init-val)))
-
-(defmethod agg-comp-exec-stats :bolt
-  [window include-sys? acc-stats new-data]
-  (assoc (agg-bolt-exec-win-stats acc-stats (:stats new-data) include-sys?)
-         :stats (merge-agg-comp-stats-comp-page-bolt
-                  (:stats acc-stats)
-                  (agg-pre-merge-comp-page-bolt new-data window include-sys?))
-         :type :bolt))
-
-(defmethod agg-comp-exec-stats :spout
-  [window include-sys? acc-stats new-data]
-  (assoc (agg-spout-exec-win-stats acc-stats (:stats new-data) include-sys?)
-         :stats (merge-agg-comp-stats-comp-page-spout
-                  (:stats acc-stats)
-                  (agg-pre-merge-comp-page-spout new-data window include-sys?))
-         :type :spout))
-
-(defn- aggregate-comp-stats*
-  [window include-sys? data init-val]
-  (-> (partial agg-comp-exec-stats
-               window
-               include-sys?)
-      (reduce init-val data)))
-
-(defmulti aggregate-comp-stats
-  (fn dispatch-fn [& args] (-> args last first :type)))
-
-(defmethod aggregate-comp-stats :bolt
-  [& args]
-  (let [init-val {:type :bolt
-                  :cid+sid->input-stats {}
-                  :sid->output-stats {}
-                  :executor-stats []
-                  :window->emitted {}
-                  :window->transferred {}
-                  :window->exec-lat-wgt-avg {}
-                  :window->executed {}
-                  :window->proc-lat-wgt-avg {}
-                  :window->acked {}
-                  :window->failed {}}]
-    (apply aggregate-comp-stats* (concat args (list init-val)))))
-
-(defmethod aggregate-comp-stats :spout
-  [& args]
-  (let [init-val {:type :spout
-                  :sid->output-stats {}
-                  :executor-stats []
-                  :window->emitted {}
-                  :window->transferred {}
-                  :window->comp-lat-wgt-avg {}
-                  :window->acked {}
-                  :window->failed {}}]
-    (apply aggregate-comp-stats* (concat args (list init-val)))))
-
-(defmethod aggregate-comp-stats :default [& _] {})
-
-(defmulti post-aggregate-comp-stats
-  (fn [_ _ data] (:type data)))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defmethod post-aggregate-comp-stats :bolt
-  [task->component
-   exec->host+port
-   {{i-stats :cid+sid->input-stats
-     o-stats :sid->output-stats
-     num-tasks :num-tasks
-     num-executors :num-executors} :stats
-    comp-type :type :as acc-data}]
-  {:type comp-type
-   :num-tasks num-tasks
-   :num-executors num-executors
-   :cid+sid->input-stats
-   (->> i-stats
-        (map-val (fn [m]
-                     (let [executed (:executed m)
-                           lats (if (and executed (pos? executed))
-                                  {:execute-latency
-                                   (div (or (:executeLatencyTotal m) 0)
-                                        executed)
-                                   :process-latency
-                                   (div (or (:processLatencyTotal m) 0)
-                                        executed)}
-                                  {:execute-latency 0
-                                   :process-latency 0})]
-                       (-> m (merge lats) (dissoc :executeLatencyTotal
-                                                  :processLatencyTotal))))))
-   :sid->output-stats o-stats
-   :executor-stats (:executor-stats (:stats acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->emitted (map-key str (:window->emitted acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->transferred (map-key str (:window->transferred acc-data))
-   :window->execute-latency
-     (compute-weighted-averages-per-window acc-data
-                                           :window->exec-lat-wgt-avg
-                                           :window->executed)
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->executed (map-key str (:window->executed acc-data))
-   :window->process-latency
-     (compute-weighted-averages-per-window acc-data
-                                           :window->proc-lat-wgt-avg
-                                           :window->executed)
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->acked (map-key str (:window->acked acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->failed (map-key str (:window->failed acc-data))})
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defmethod post-aggregate-comp-stats :spout
-  [task->component
-   exec->host+port
-   {{o-stats :sid->output-stats
-     num-tasks :num-tasks
-     num-executors :num-executors} :stats
-    comp-type :type :as acc-data}]
-  {:type comp-type
-   :num-tasks num-tasks
-   :num-executors num-executors
-   :sid->output-stats
-   (->> o-stats
-        (map-val (fn [m]
-                     (let [acked (:acked m)
-                           lat (if (and acked (pos? acked))
-                                 {:complete-latency
-                                  (div (or (:completeLatencyTotal m) 0) acked)}
-                                 {:complete-latency 0})]
-                       (-> m (merge lat) (dissoc :completeLatencyTotal))))))
-   :executor-stats (:executor-stats (:stats acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->emitted (map-key str (:window->emitted acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->transferred (map-key str (:window->transferred acc-data))
-   :window->complete-latency
-     (compute-weighted-averages-per-window acc-data
-                                           :window->comp-lat-wgt-avg
-                                           :window->acked)
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->acked (map-key str (:window->acked acc-data))
-   ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-   :window->failed (map-key str (:window->failed acc-data))})
-
-(defmethod post-aggregate-comp-stats :default [& _] {})
-
-(defn thriftify-exec-agg-stats
-  [comp-id comp-type {:keys [executor-id host port uptime] :as stats}]
-  (doto (ExecutorAggregateStats.)
-    (.set_exec_summary (ExecutorSummary. (apply #(ExecutorInfo. %1 %2)
-                                                executor-id)
-                                         comp-id
-                                         host
-                                         port
-                                         (or uptime 0)))
-    (.set_stats ((condp = comp-type
-                   :bolt thriftify-bolt-agg-stats
-                   :spout thriftify-spout-agg-stats) stats))))
-
-(defn- thriftify-bolt-input-stats
-  [cid+sid->input-stats]
-  (into {} (for [[cid+sid input-stats] cid+sid->input-stats]
-             [(to-global-stream-id cid+sid)
-              (thriftify-bolt-agg-stats input-stats)])))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn- thriftify-bolt-output-stats
-  [sid->output-stats]
-  (map-val thriftify-bolt-agg-stats sid->output-stats))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn- thriftify-spout-output-stats
-  [sid->output-stats]
-  (map-val thriftify-spout-agg-stats sid->output-stats))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn thriftify-comp-page-data
-  [topo-id topology comp-id data]
-  (let [w->stats (swap-map-order
-                   (merge
-                     {:emitted (:window->emitted data)
-                      :transferred (:window->transferred data)
-                      :acked (:window->acked data)
-                      :failed (:window->failed data)}
-                     (condp = (:type data)
-                       :bolt {:execute-latency (:window->execute-latency data)
-                              :process-latency (:window->process-latency data)
-                              :executed (:window->executed data)}
-                       :spout {:complete-latency
-                               (:window->complete-latency data)}
-                       {}))) ; default
-        [compType exec-stats w->stats gsid->input-stats sid->output-stats]
-          (condp = (component-type topology comp-id)
-            :bolt [ComponentType/BOLT
-                   (->
-                     (partial thriftify-exec-agg-stats comp-id :bolt)
-                     (map (:executor-stats data)))
-                   (map-val thriftify-bolt-agg-stats w->stats)
-                   (thriftify-bolt-input-stats (:cid+sid->input-stats data))
-                   (thriftify-bolt-output-stats (:sid->output-stats data))]
-            :spout [ComponentType/SPOUT
-                    (->
-                      (partial thriftify-exec-agg-stats comp-id :spout)
-                      (map (:executor-stats data)))
-                    (map-val thriftify-spout-agg-stats w->stats)
-                    nil ;; spouts do not have input stats
-                    (thriftify-spout-output-stats (:sid->output-stats data))]),
-        num-executors (:num-executors data)
-        num-tasks (:num-tasks data)
-        ret (doto (ComponentPageInfo. comp-id compType)
-              (.set_topology_id topo-id)
-              (.set_topology_name nil)
-              (.set_window_to_stats w->stats)
-              (.set_sid_to_output_stats sid->output-stats)
-              (.set_exec_stats exec-stats))]
-    (and num-executors (.set_num_executors ret num-executors))
-    (and num-tasks (.set_num_tasks ret num-tasks))
-    (and gsid->input-stats
-         (.set_gsid_to_input_stats ret gsid->input-stats))
-    ret))
-
-(defn agg-comp-execs-stats
-  "Aggregate various executor statistics for a component from the given
-  heartbeats."
-  [exec->host+port
-   task->component
-   beats
-   window
-   include-sys?
-   topology-id
-   topology
-   component-id]
-  (->> ;; This iterates over each executor one time, because of lazy evaluation.
-    (extract-data-from-hb exec->host+port
-                          task->component
-                          beats
-                          include-sys?
-                          topology
-                          component-id)
-    (aggregate-comp-stats window include-sys?)
-    (post-aggregate-comp-stats task->component exec->host+port)
-    (thriftify-comp-page-data topology-id topology component-id)))
-
-(defn expand-averages
-  [avg counts]
-  (let [avg (clojurify-structure avg)
-        counts (clojurify-structure counts)]
-    (into {}
-          (for [[slice streams] counts]
-            [slice
-             (into {}
-                   (for [[stream c] streams]
-                     [stream
-                      [(* c (get-in avg [slice stream]))
-                       c]]
-                     ))]))))
-
-(defn expand-averages-seq
-  [average-seq counts-seq]
-  (->> (map vector average-seq counts-seq)
-       (map #(apply expand-averages %))
-       (apply merge-with (fn [s1 s2] (merge-with add-pairs s1 s2)))))
-
-(defn- val-avg
-  [[t c]]
-  (if (= c 0) 0
-    (double (/ t c))))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn aggregate-averages
-  [average-seq counts-seq]
-  (->> (expand-averages-seq average-seq counts-seq)
-       (map-val
-         (fn [s]
-           (map-val val-avg s)))))
-
-;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
-(defn aggregate-avg-streams
-  [avg counts]
-  (let [expanded (expand-averages avg counts)]
-    (->> expanded
-         (map-val #(reduce add-pairs (vals %)))
-         (map-val val-avg))))
-
-;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
-(defn pre-process
-  [stream-summary include-sys?]
-  (let [filter-fn (mk-include-sys-fn include-sys?)
-        emitted (:emitted stream-summary)
-        emitted (into {} (for [[window stat] emitted]
-                           {window (filter-key filter-fn stat)}))
-        transferred (:transferred stream-summary)
-        transferred (into {} (for [[window stat] transferred]
-                               {window (filter-key filter-fn stat)}))
-        stream-summary (-> stream-summary (dissoc :emitted) (assoc :emitted emitted))
-        stream-summary (-> stream-summary (dissoc :transferred) (assoc :transferred transferred))]
-    stream-summary))
-
-(defn aggregate-counts
-  [counts-seq]
-  (->> counts-seq
-       (map clojurify-structure)
-       (apply merge-with
-              (fn [s1 s2]
-                (merge-with + s1 s2)))))
-
-(defn aggregate-common-stats
-  [stats-seq]
-  {:emitted (aggregate-counts (map #(.get_emitted ^ExecutorStats %) stats-seq))
-   :transferred (aggregate-counts (map #(.get_transferred ^ExecutorStats %) stats-seq))})
-
-(defn- collectify
-  [obj]
-  (if (or (sequential? obj) (instance? Collection obj))
-    obj
-    [obj]))
-
-(defn aggregate-bolt-stats
-  [stats-seq include-sys?]
-  (let [stats-seq (collectify stats-seq)]
-    (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
-           {:acked
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_acked)
-                                   stats-seq))
-            :failed
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_failed)
-                                   stats-seq))
-            :executed
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_executed)
-                                   stats-seq))
-            :process-latencies
-            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_process_ms_avg)
-                                     stats-seq)
-                                (map #(.. ^ExecutorStats % get_specific get_bolt get_acked)
-                                     stats-seq))
-            :execute-latencies
-            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_execute_ms_avg)
-                                     stats-seq)
-                                (map #(.. ^ExecutorStats % get_specific get_bolt get_executed)
-                                     stats-seq))})))
-
-(defn aggregate-spout-stats
-  [stats-seq include-sys?]
-  (let [stats-seq (collectify stats-seq)]
-    (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
-           {:acked
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_acked)
-                                   stats-seq))
-            :failed
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_failed)
-                                   stats-seq))
-            :complete-latencies
-            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_spout get_complete_ms_avg)
-                                     stats-seq)
-                                (map #(.. ^ExecutorStats % get_specific get_spout get_acked)
-                                     stats-seq))})))
-
-(defn get-filled-stats
-  [summs]
-  (->> summs
-       (map #(.get_stats ^ExecutorSummary %))
-       (filter not-nil?)))
-
-(defn aggregate-spout-streams
-  [stats]
-  {:acked (aggregate-count-streams (:acked stats))
-   :failed (aggregate-count-streams (:failed stats))
-   :emitted (aggregate-count-streams (:emitted stats))
-   :transferred (aggregate-count-streams (:transferred stats))
-   :complete-latencies (aggregate-avg-streams (:complete-latencies stats)
-                                              (:acked stats))})
-
-(defn spout-streams-stats
-  [summs include-sys?]
-  (let [stats-seq (get-filled-stats summs)]
-    (aggregate-spout-streams
-      (aggregate-spout-stats
-        stats-seq include-sys?))))
-
-(defn aggregate-bolt-streams
-  [stats]
-  {:acked (aggregate-count-streams (:acked stats))
-   :failed (aggregate-count-streams (:failed stats))
-   :emitted (aggregate-count-streams (:emitted stats))
-   :transferred (aggregate-count-streams (:transferred stats))
-   :process-latencies (aggregate-avg-streams (:process-latencies stats)
-                                             (:acked stats))
-   :executed (aggregate-count-streams (:executed stats))
-   :execute-latencies (aggregate-avg-streams (:execute-latencies stats)
-                                             (:executed stats))})
-
-(defn compute-executor-capacity
-  [^ExecutorSummary e]
-  (let [stats (.get_stats e)
-        stats (if stats
-                (-> stats
-                    (aggregate-bolt-stats true)
-                    (aggregate-bolt-streams)
-                    swap-map-order
-                    (get (str TEN-MIN-IN-SECONDS))))
-        uptime (Utils/nullToZero (.get_uptime_secs e))
-        window (if (< uptime TEN-MIN-IN-SECONDS) uptime TEN-MIN-IN-SECONDS)
-        executed (-> stats :executed Utils/nullToZero)
-        latency (-> stats :execute-latencies Utils/nullToZero)]
-    (if (> window 0)
-      (div (* executed latency) (* 1000 window)))))
-
-(defn bolt-streams-stats
-  [summs include-sys?]
-  (let [stats-seq (get-filled-stats summs)]
-    (aggregate-bolt-streams
-      (aggregate-bolt-stats
-        stats-seq include-sys?))))
-
-(defn total-aggregate-stats
-  [spout-summs bolt-summs include-sys?]
-  (let [spout-stats (get-filled-stats spout-summs)
-        bolt-stats (get-filled-stats bolt-summs)
-        agg-spout-stats (-> spout-stats
-                            (aggregate-spout-stats include-sys?)
-                            aggregate-spout-streams)
-        agg-bolt-stats (-> bolt-stats
-                           (aggregate-bolt-stats include-sys?)
-                           aggregate-bolt-streams)]
-    (merge-with
-      (fn [s1 s2]
-        (merge-with + s1 s2))
-      (select-keys
-        agg-bolt-stats
-        ;; Include only keys that will be used.  We want to count acked and
-        ;; failed only for the "tuple trees," so we do not include those keys
-        ;; from the bolt executors.
-        [:emitted :transferred])
-      agg-spout-stats)))
-
-(defn error-subset
-  [error-str]
-  (apply str (take 200 error-str)))
-
-(defn most-recent-error
-  [errors-list]
-  (let [error (->> errors-list
-                   (sort-by #(.get_error_time_secs ^ErrorInfo %))
-                   reverse
-                   first)]
-    (if error
-      (error-subset (.get_error ^ErrorInfo error))
-      "")))
-
-(defn float-str [n]
-  (if n
-    (format "%.3f" (float n))
-    "0"))
-
-(defn compute-bolt-capacity
-  [executors]
-  (->> executors
-       (map compute-executor-capacity)
-       (map #(Utils/nullToZero %))
-       (apply max)))


[13/30] storm git commit: changed according to comments

Posted by bo...@apache.org.
changed according to comments


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

Branch: refs/heads/master
Commit: abe9b676c0f15fa47809ae4a094001e345521de6
Parents: bfbd375
Author: 卫乐 <we...@taobao.com>
Authored: Mon Feb 29 11:49:26 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Mon Feb 29 11:49:26 2016 +0800

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    |  8 +--
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |  4 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |  4 +-
 .../apache/storm/stats/BoltExecutorStats.java   | 57 ++++++++++++--------
 .../jvm/org/apache/storm/stats/CommonStats.java | 31 ++++++-----
 .../apache/storm/stats/SpoutExecutorStats.java  | 33 +++++++-----
 .../jvm/org/apache/storm/stats/StatsUtil.java   |  7 ++-
 .../jvm/org/apache/storm/utils/ConfigUtils.java |  8 ++-
 .../test/clj/org/apache/storm/nimbus_test.clj   |  2 +-
 .../clj/org/apache/storm/supervisor_test.clj    | 11 ++--
 10 files changed, 99 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 3b4e330..4bbce10 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -257,8 +257,8 @@
      :batch-transfer-queue batch-transfer->worker
      :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
      :suicide-fn (:suicide-fn worker)
-      :storm-cluster-state (ClusterUtils/mkStormClusterState (:state-store worker) (Utils/getWorkerACL storm-conf)
-                             (ClusterStateContext. DaemonType/WORKER))
+     :storm-cluster-state (ClusterUtils/mkStormClusterState (:state-store worker) (Utils/getWorkerACL storm-conf)
+                            (ClusterStateContext. DaemonType/WORKER))
      :type executor-type
      ;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)
      :stats (mk-executor-stats <> (ConfigUtils/samplingRate storm-conf))
@@ -861,7 +861,7 @@
 
 ;; TODO: refactor this to be part of an executor-specific map
 (defmethod mk-executor-stats :spout [_ rate]
-  (SpoutExecutorStats/mkSpoutStats rate))
+  (SpoutExecutorStats. rate))
 
 (defmethod mk-executor-stats :bolt [_ rate]
-  (BoltExecutorStats/mkBoltStats rate))
+  (BoltExecutorStats. rate))

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index f36cf7d..83f73d5 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -916,7 +916,7 @@
         storm-cluster-state (:storm-cluster-state nimbus)
         ^INimbus inimbus (:inimbus nimbus)
         ;; read all the topologies
-          topology-ids (.activeStorms storm-cluster-state)
+        topology-ids (.activeStorms storm-cluster-state)
         topologies (into {} (for [tid topology-ids]
                               {tid (read-topology-details nimbus tid)}))
         topologies (Topologies. topologies)
@@ -1800,7 +1800,7 @@
               storm-name (topology-conf TOPOLOGY-NAME)
               _ (check-authorization! nimbus storm-name topology-conf "getLogConfig")
              storm-cluster-state (:storm-cluster-state nimbus)
-              log-config (.topologyLogConfig storm-cluster-state id nil)]
+             log-config (.topologyLogConfig storm-cluster-state id nil)]
            (if log-config log-config (LogConfig.))))
 
       (^String getTopologyConf [this ^String id]

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index aad0e38..b9cf2d7 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -1222,7 +1222,7 @@
            (json-response {"status" "ok"
                            "id" host-port}
                           (m "callback")))))
-       
+
   (GET "/api/v1/topology/:id/profiling/dumpheap/:host-port"
        [:as {:keys [servlet-request]} id host-port & m]
        (populate-context! servlet-request)
@@ -1238,7 +1238,7 @@
            (json-response {"status" "ok"
                            "id" host-port}
                           (m "callback")))))
-  
+
   (GET "/" [:as {cookies :cookies}]
     (mark! ui:num-main-page-http-requests)
     (resp/redirect "/index.html"))

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
index d694bc3..f6dad09 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
@@ -17,9 +17,14 @@
  */
 package org.apache.storm.stats;
 
-import clojure.lang.PersistentVector;
+import com.google.common.collect.Lists;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import org.apache.storm.generated.BoltStats;
+import org.apache.storm.generated.ExecutorSpecificStats;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 
@@ -34,14 +39,14 @@ public class BoltExecutorStats extends CommonStats {
 
     public static final String[] BOLT_FIELDS = {ACKED, FAILED, EXECUTED, PROCESS_LATENCIES, EXECUTE_LATENCIES};
 
-    public BoltExecutorStats() {
-        super();
+    public BoltExecutorStats(int rate) {
+        super(rate);
 
-        put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
-        put(FAILED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
-        put(EXECUTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
-        put(PROCESS_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
-        put(EXECUTE_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(FAILED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(EXECUTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(PROCESS_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(EXECUTE_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
     }
 
     public MultiCountStatAndMetric getAcked() {
@@ -65,19 +70,19 @@ public class BoltExecutorStats extends CommonStats {
     }
 
     public void boltExecuteTuple(String component, String stream, long latencyMs) {
-        Object key = PersistentVector.create(component, stream);
+        List key = Lists.newArrayList(component, stream);
         this.getExecuted().incBy(key, this.rate);
         this.getExecuteLatencies().record(key, latencyMs);
     }
 
     public void boltAckedTuple(String component, String stream, long latencyMs) {
-        Object key = PersistentVector.create(component, stream);
+        List key = Lists.newArrayList(component, stream);
         this.getAcked().incBy(key, this.rate);
         this.getProcessLatencies().record(key, latencyMs);
     }
 
     public void boltFailedTuple(String component, String stream, long latencyMs) {
-        Object key = PersistentVector.create(component, stream);
+        List key = Lists.newArrayList(component, stream);
         this.getFailed().incBy(key, this.rate);
 
     }
@@ -92,16 +97,22 @@ public class BoltExecutorStats extends CommonStats {
         return ret;
     }
 
-    public void cleanupStats() {
-        super.cleanupStats();
-        for (String field : BOLT_FIELDS) {
-            cleanupStat(this.get(field));
-        }
-    }
-
-    public static BoltExecutorStats mkBoltStats(int rate) {
-        BoltExecutorStats stats = new BoltExecutorStats();
-        stats.setRate(rate);
-        return stats;
-    }
+//    public ExecutorStats renderStats() {
+//        cleanupStats();
+//
+//        ExecutorStats ret = new ExecutorStats();
+//        ret.set_emitted(valueStat(EMITTED));
+//        ret.set_transferred(valueStat(TRANSFERRED));
+//        ret.set_rate(this.rate);
+//
+//        BoltStats boltStats = new BoltStats(
+//                StatsUtil.windowSetConverter(valueStat(ACKED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+//                StatsUtil.windowSetConverter(valueStat(FAILED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+//                StatsUtil.windowSetConverter(valueStat(PROCESS_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+//                StatsUtil.windowSetConverter(valueStat(EXECUTED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+//                StatsUtil.windowSetConverter(valueStat(EXECUTE_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY));
+//        ret.set_specific(ExecutorSpecificStats.bolt(boltStats));
+//
+//        return ret;
+//    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java b/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
index 93d42a4..e386413 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/CommonStats.java
@@ -33,22 +33,19 @@ public class CommonStats {
     public static final String TRANSFERRED = "transferred";
     public static final String[] COMMON_FIELDS = {EMITTED, TRANSFERRED};
 
-    protected int rate;
+    protected final int rate;
     protected final Map metricMap = new HashMap();
 
-    public CommonStats() {
-        put(EMITTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
-        put(TRANSFERRED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+    public CommonStats(int rate) {
+        this.rate = rate;
+        this.put(EMITTED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
+        this.put(TRANSFERRED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
     }
 
     public int getRate() {
         return this.rate;
     }
 
-    public void setRate(int rate) {
-        this.rate = rate;
-    }
-
     public MultiCountStatAndMetric getEmitted() {
         return (MultiCountStatAndMetric) get(EMITTED);
     }
@@ -73,13 +70,13 @@ public class CommonStats {
         this.getTransferred().incBy(stream, this.rate * amount);
     }
 
-    protected void cleanupStats() {
-        for (String field : COMMON_FIELDS) {
-            cleanupStat(this.get(field));
+    public void cleanupStats() {
+        for (Object imetric : this.metricMap.values()) {
+            cleanupStat((IMetric) imetric);
         }
     }
 
-    protected void cleanupStat(IMetric metric) {
+    private void cleanupStat(IMetric metric) {
         if (metric instanceof MultiCountStatAndMetric) {
             ((MultiCountStatAndMetric) metric).close();
         } else if (metric instanceof MultiLatencyStatAndMetric) {
@@ -102,4 +99,14 @@ public class CommonStats {
         return ret;
     }
 
+    protected Map valueStat(String field) {
+        IMetric metric = this.get(field);
+        if (metric instanceof MultiCountStatAndMetric) {
+            return ((MultiCountStatAndMetric) metric).getTimeCounts();
+        } else if (metric instanceof MultiLatencyStatAndMetric) {
+            return ((MultiLatencyStatAndMetric) metric).getTimeLatAvg();
+        }
+        return null;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
index d6d9162..918ae06 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
@@ -19,6 +19,9 @@ package org.apache.storm.stats;
 
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.storm.generated.ExecutorSpecificStats;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 
@@ -31,8 +34,8 @@ public class SpoutExecutorStats extends CommonStats {
 
     public static final String[] SPOUT_FIELDS = {ACKED, FAILED, COMPLETE_LATENCIES};
 
-    public SpoutExecutorStats() {
-        super();
+    public SpoutExecutorStats(int rate) {
+        super(rate);
         this.put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
         this.put(FAILED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
         this.put(COMPLETE_LATENCIES, new MultiLatencyStatAndMetric(NUM_STAT_BUCKETS));
@@ -69,16 +72,18 @@ public class SpoutExecutorStats extends CommonStats {
         return ret;
     }
 
-    public void cleanupStats() {
-        super.cleanupStats();
-        for (String field : SpoutExecutorStats.SPOUT_FIELDS) {
-            cleanupStat(this.get(field));
-        }
-    }
-
-    public static SpoutExecutorStats mkSpoutStats(int rate) {
-        SpoutExecutorStats stats = new SpoutExecutorStats();
-        stats.setRate(rate);
-        return stats;
-    }
+//    public ExecutorStats renderStats() {
+//        cleanupStats();
+//
+//        ExecutorStats ret = new ExecutorStats();
+//        ret.set_emitted(valueStat(EMITTED));
+//        ret.set_transferred(valueStat(TRANSFERRED));
+//        ret.set_rate(this.rate);
+//
+//        SpoutStats spoutStats = new SpoutStats(
+//                valueStat(ACKED), valueStat(FAILED), valueStat(COMPLETE_LATENCIES));
+//        ret.set_specific(ExecutorSpecificStats.spout(spoutStats));
+//
+//        return ret;
+//    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index 75ec292..efdf8e0 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -113,10 +113,10 @@ public class StatsUtil {
     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 IdentityTransformer IDENTITY = new IdentityTransformer();
+    public static final IdentityTransformer IDENTITY = new IdentityTransformer();
     private static final ToStringTransformer TO_STRING = new ToStringTransformer();
     private static final FromGlobalStreamIdTransformer FROM_GSID = new FromGlobalStreamIdTransformer();
-    private static final ToGlobalStreamIdTransformer TO_GSID = new ToGlobalStreamIdTransformer();
+    public static final ToGlobalStreamIdTransformer TO_GSID = new ToGlobalStreamIdTransformer();
 
 
     // =====================================================================================
@@ -1659,6 +1659,9 @@ public class StatsUtil {
             Map executorStat = (Map) stat.get(1);
             ExecutorInfo executorInfo = new ExecutorInfo(start, end);
             ret.put(executorInfo, thriftifyExecutorStats(executorStat));
+//            ExecutorStats executorStat = (ExecutorStats) stat.get(1);
+//            ExecutorInfo executorInfo = new ExecutorInfo(start, end);
+//            ret.put(executorInfo, executorStat);
         }
         return ret;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java b/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java
index 1ac0249..36d4352 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java
@@ -44,7 +44,7 @@ public class ConfigUtils {
 
     // A singleton instance allows us to mock delegated static methods in our
     // tests by subclassing.
-    private static ConfigUtils _instance = new ConfigUtils();;
+    private static ConfigUtils _instance = new ConfigUtils();
 
     /**
      * Provide an instance of this class for delegates to use.  To mock out
@@ -66,7 +66,11 @@ public class ConfigUtils {
             dir = System.getProperty("storm.log.dir");
         } else if ((conf = readStormConfig()).get("storm.log.dir") != null) {
             dir = String.valueOf(conf.get("storm.log.dir"));
-        } else  {
+        } else if (System.getProperty("storm.local.dir") != null) {
+            dir = System.getProperty("storm.local.dir") + FILE_SEPARATOR + "logs";
+        } else if (conf.get("storm.local.dir") != null) {
+            dir = conf.get("storm.local.dir") + FILE_SEPARATOR + "logs";
+        } else {
             dir = concatIfNotNull(System.getProperty("storm.home")) + FILE_SEPARATOR + "logs";
         }
         try {

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/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 8c383e5..fe804d7 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -145,7 +145,7 @@
         stats (:executor-stats curr-beat)]
     (.workerHeartbeat state storm-id node port
       (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10
-                               :executor-stats (merge stats {executor (clojurify-structure (.renderStats (BoltExecutorStats/mkBoltStats 20)))})})
+                               :executor-stats (merge stats {executor (clojurify-structure (.renderStats (BoltExecutorStats. 20)))})})
       )))
 
 (defn slot-assignments [cluster storm-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/abe9b676/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
index cdd66e4..415a56d 100644
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@ -297,6 +297,7 @@
     (let [mock-port "42"
           mock-storm-id "fake-storm-id"
           mock-worker-id "fake-worker-id"
+          storm-log-dir (ConfigUtils/getLogDir)
           mock-cp (str Utils/FILE_PATH_SEPARATOR "base" Utils/CLASS_PATH_SEPARATOR Utils/FILE_PATH_SEPARATOR "stormjar.jar")
           mock-sensitivity "S3"
           mock-cp "/base:/stormjar.jar"
@@ -308,7 +309,7 @@
                                (str "-Dstorm.id=" mock-storm-id)
                                (str "-Dworker.id=" mock-worker-id)
                                (str "-Dworker.port=" mock-port)
-                               "-Dstorm.log.dir=/logs"
+                               (str "-Dstorm.log.dir=" storm-log-dir)
                                "-Dlog4j.configurationFile=/log4j2/worker.xml"
                                "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector"
                                "org.apache.storm.LogWriter"]
@@ -321,7 +322,7 @@
                                 "-Dworkers.artifacts=/tmp/workers-artifacts"
                                 "-Dstorm.conf.file="
                                 "-Dstorm.options="
-                                (str "-Dstorm.log.dir=" Utils/FILE_PATH_SEPARATOR "logs")
+                                (str "-Dstorm.log.dir=" storm-log-dir)
                                 (str "-Dlogging.sensitivity=" mock-sensitivity)
                                 (str "-Dlog4j.configurationFile=" Utils/FILE_PATH_SEPARATOR "log4j2" Utils/FILE_PATH_SEPARATOR "worker.xml")
                                 "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector"
@@ -484,6 +485,7 @@
           mock-cp "mock-classpath'quote-on-purpose"
           attrs (make-array FileAttribute 0)
           storm-local (.getCanonicalPath (.toFile (Files/createTempDirectory "storm-local" attrs)))
+          storm-log-dir (ConfigUtils/getLogDir)
           worker-script (str storm-local "/workers/" mock-worker-id "/storm-worker-script.sh")
           exp-launch ["/bin/worker-launcher"
                       "me"
@@ -499,7 +501,7 @@
                                " '-Dstorm.id=" mock-storm-id "'"
                                " '-Dworker.id=" mock-worker-id "'"
                                " '-Dworker.port=" mock-port "'"
-                               " '-Dstorm.log.dir=/logs'"
+                               " '-Dstorm.log.dir=" storm-log-dir "'"
                                " '-Dlog4j.configurationFile=/log4j2/worker.xml'"
                                " '-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector'"
                                " 'org.apache.storm.LogWriter'"
@@ -512,7 +514,7 @@
                                " '-Dworkers.artifacts=" (str storm-local "/workers-artifacts'")
                                " '-Dstorm.conf.file='"
                                " '-Dstorm.options='"
-                               " '-Dstorm.log.dir=/logs'"
+                               " '-Dstorm.log.dir=" storm-log-dir "'"
                                " '-Dlogging.sensitivity=" mock-sensitivity "'"
                                " '-Dlog4j.configurationFile=/log4j2/worker.xml'"
                                " '-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector'"
@@ -836,3 +838,4 @@
           {"sup1" [3 4]}
           (get-storm-id (:storm-cluster-state cluster) "topology2"))
         )))
+


[29/30] storm git commit: Merge branch 'master' of https://github.com/unsleepy22/storm into STORM-1252

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/unsleepy22/storm into STORM-1252

STORM-1252: port backtype.storm.stats to java


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

Branch: refs/heads/master
Commit: cf7ef7f771ab263bea976f51187323e54f75aaf9
Parents: afcb2a0 fb19f0d
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Tue Mar 15 12:33:47 2016 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Tue Mar 15 12:33:47 2016 -0500

----------------------------------------------------------------------
 .../clj/org/apache/storm/command/heartbeats.clj |    5 +-
 .../src/clj/org/apache/storm/converter.clj      |   46 +-
 .../org/apache/storm/daemon/builtin_metrics.clj |   33 +-
 .../clj/org/apache/storm/daemon/executor.clj    |   22 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |   91 +-
 .../src/clj/org/apache/storm/daemon/task.clj    |   10 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |   18 +-
 storm-core/src/clj/org/apache/storm/stats.clj   | 1568 -----------
 storm-core/src/clj/org/apache/storm/testing.clj |    8 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |   71 +-
 .../apache/storm/stats/BoltExecutorStats.java   |  105 +
 .../jvm/org/apache/storm/stats/CommonStats.java |  112 +
 .../apache/storm/stats/SpoutExecutorStats.java  |   79 +
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 2441 ++++++++++++++++++
 .../test/clj/org/apache/storm/nimbus_test.clj   |   19 +-
 15 files changed, 2865 insertions(+), 1763 deletions(-)
----------------------------------------------------------------------



[12/30] storm git commit: Merge https://github.com/apache/storm

Posted by bo...@apache.org.
Merge https://github.com/apache/storm


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

Branch: refs/heads/master
Commit: bfbd3753d62ade9675bba3e0676417f673cde3a3
Parents: e0e9de7 73312ad
Author: 卫乐 <we...@taobao.com>
Authored: Fri Feb 26 20:34:09 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Fri Feb 26 20:34:09 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                          |  1 +
 storm-core/src/clj/org/apache/storm/ui/core.clj       | 14 ++++----------
 storm-core/src/ui/public/component.html               |  2 +-
 .../ui/public/templates/topology-page-template.html   |  6 +++++-
 storm-core/src/ui/public/topology.html                |  2 +-
 5 files changed, 12 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/bfbd3753/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------


[24/30] storm git commit: 1. changed heartbeat structure to java HashMap 2. use HashMaps in StatsUtil instead of clojure map 3. changed tests accordingly

Posted by bo...@apache.org.
1. changed heartbeat structure to java HashMap
2. use HashMaps in StatsUtil instead of clojure map
3. changed tests accordingly


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

Branch: refs/heads/master
Commit: 4c246d1c5582396debfad2a3687a243303e9a0e5
Parents: 9002528
Author: 卫乐 <we...@taobao.com>
Authored: Tue Mar 8 20:28:14 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Tue Mar 8 20:28:14 2016 +0800

----------------------------------------------------------------------
 .../clj/org/apache/storm/command/heartbeats.clj |    5 +-
 .../src/clj/org/apache/storm/converter.clj      |   25 -
 .../clj/org/apache/storm/daemon/executor.clj    |    2 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |   56 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |   18 +-
 storm-core/src/clj/org/apache/storm/testing.clj |    8 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |   16 +-
 .../apache/storm/stats/BoltExecutorStats.java   |   47 +-
 .../apache/storm/stats/SpoutExecutorStats.java  |   37 +-
 .../jvm/org/apache/storm/stats/StatsUtil.java   | 1281 +++++++++++-------
 .../test/clj/org/apache/storm/nimbus_test.clj   |   17 +-
 11 files changed, 835 insertions(+), 677 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
index c4413f0..625cff7 100644
--- a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
+++ b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
@@ -22,7 +22,8 @@
             [clojure.string :as string])
   (:import [org.apache.storm.generated ClusterWorkerHeartbeat]
            [org.apache.storm.utils Utils ConfigUtils]
-           [org.apache.storm.cluster ZKStateStorage ClusterStateContext ClusterUtils])
+           [org.apache.storm.cluster ZKStateStorage ClusterStateContext ClusterUtils]
+           [org.apache.storm.stats StatsUtil])
   (:gen-class))
 
 (defn -main [command path & args]
@@ -37,7 +38,7 @@
       "get"
       (log-message 
        (if-let [hb (.get_worker_hb cluster path false)]
-         (clojurify-zk-worker-hb
+         (StatsUtil/convertZkWorkerHb
           (Utils/deserialize
            hb
            ClusterWorkerHeartbeat))

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index 495fe7f..6bd7e72 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -215,31 +215,6 @@
       (convert-to-symbol-from-status (.get_prev_status storm-base))
       (map-val clojurify-debugoptions (.get_component_debug storm-base)))))
 
-(defn clojurify-zk-worker-hb [^ClusterWorkerHeartbeat worker-hb]
-  (if worker-hb
-    {:storm-id (.get_storm_id worker-hb)
-     :executor-stats (clojurify-structure (StatsUtil/clojurifyStats (into {} (.get_executor_stats worker-hb))))
-     :uptime (.get_uptime_secs worker-hb)
-     :time-secs (.get_time_secs worker-hb)
-     }
-    {}))
-
-(defn clojurify-zk-executor-hb [^ExecutorBeat executor-hb]
-  (if executor-hb
-    {:stats (StatsUtil/clojurifyExecutorStats (.getStats executor-hb))
-     :uptime (.getUptime executor-hb)
-     :time-secs (.getTimeSecs executor-hb)
-     }
-    {}))
-
-(defn thriftify-zk-worker-hb [worker-hb]
-  (if (not-empty (filter second (:executor-stats worker-hb)))
-    (doto (ClusterWorkerHeartbeat.)
-      (.set_uptime_secs (:uptime worker-hb))
-      (.set_storm_id (:storm-id worker-hb))
-      (.set_executor_stats (StatsUtil/thriftifyStats (filter second (:executor-stats worker-hb))))
-      (.set_time_secs (:time-secs worker-hb)))))
-
 (defn thriftify-error [error]
   (doto (ErrorInfo. (:error error) (:time-secs error))
     (.set_host (:host error))

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 4bbce10..becd8f3 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -406,7 +406,7 @@
     (reify
       RunningExecutor
       (render-stats [this]
-        (clojurify-structure (.renderStats (:stats executor-data))))
+        (.renderStats (:stats executor-data)))
       (get-executor-id [this]
         executor-id)
       (credentials-changed [this creds]

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 83f73d5..997f92c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -559,48 +559,17 @@
                       executor->component
                       (:launch-time-secs storm-base))))
 
-;; Does not assume that clocks are synchronized. Executor heartbeat is only used so that
-;; nimbus knows when it's received a new heartbeat. All timing is done by nimbus and
-;; tracked through heartbeat-cache
-(defn- update-executor-cache [curr hb timeout]
-  (let [reported-time (:time-secs hb)
-        {last-nimbus-time :nimbus-time
-         last-reported-time :executor-reported-time} curr
-        reported-time (cond reported-time reported-time
-                            last-reported-time last-reported-time
-                            :else 0)
-        nimbus-time (if (or (not last-nimbus-time)
-                        (not= last-reported-time reported-time))
-                      (Time/currentTimeSecs)
-                      last-nimbus-time
-                      )]
-      {:is-timed-out (and
-                       nimbus-time
-                       (>= (Time/deltaSecs nimbus-time) timeout))
-       :nimbus-time nimbus-time
-       :executor-reported-time reported-time
-       :heartbeat hb}))
-
-(defn update-heartbeat-cache [cache executor-beats all-executors timeout]
-  (let [cache (select-keys cache all-executors)]
-    (into {}
-      (for [executor all-executors :let [curr (cache executor)]]
-        [executor
-         (update-executor-cache curr (get executor-beats executor) timeout)]
-         ))))
 
 (defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
   (log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
   (let [storm-cluster-state (:storm-cluster-state nimbus)
-        executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))
-                             executor-stats-clojurify (clojurify-structure executor-stats-java-map)]
-                         (->> (dofor [[^ExecutorInfo executor-info  ^ExecutorBeat executor-heartbeat] executor-stats-clojurify]
-                             {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-executor-hb executor-heartbeat)})
-                           (apply merge)))
-        cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
+        executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id
+                                                       (.get_executor_node_port (thriftify-assignment existing-assignment)))]
+                         (StatsUtil/convertExecutorBeats executor-stats-java-map))
+        cache (StatsUtil/updateHeartbeatCache (@(:heartbeats-cache nimbus) storm-id)
                                       executor-beats
-                                      all-executors
-                                      ((:conf nimbus) NIMBUS-TASK-TIMEOUT-SECS))]
+                                      (StatsUtil/convertExecutors all-executors)
+                                      (int ((:conf nimbus) NIMBUS-TASK-TIMEOUT-SECS)))]
       (swap! (:heartbeats-cache nimbus) assoc storm-id cache)))
 
 (defn- update-all-heartbeats! [nimbus existing-assignments topology->executors]
@@ -625,7 +594,7 @@
     (->> all-executors
         (filter (fn [executor]
           (let [start-time (get executor-start-times executor)
-                is-timed-out (-> heartbeats-cache (get executor) :is-timed-out)]
+                is-timed-out (.get (.get heartbeats-cache (StatsUtil/convertExecutor executor)) "is-timed-out")]
             (if (and start-time
                    (or
                     (< (Time/deltaSecs start-time)
@@ -1415,8 +1384,7 @@
                                      (throw
                                        (NotAliveException. (str storm-id))))
                   assignment (clojurify-assignment (.assignmentInfo storm-cluster-state storm-id nil))
-                  beats (map-val :heartbeat (get @(:heartbeats-cache nimbus)
-                                                 storm-id))
+                  beats (get @(:heartbeats-cache nimbus) storm-id)
                   all-components (set (vals task->component))]
               {:storm-name storm-name
                :storm-cluster-state storm-cluster-state
@@ -1919,9 +1887,9 @@
                           (map (fn [c] [c (errors-fn storm-cluster-state storm-id c)]))
                           (into {}))
               executor-summaries (dofor [[executor [node port]] (:executor->node+port assignment)]
-                                        (let [host (-> assignment :node->host (get node))
-                                              heartbeat (get beats executor)
-                                              excutorstats (:stats heartbeat)
+                                     (let [host (-> assignment :node->host (get node))
+                                              heartbeat (.get beats (StatsUtil/convertExecutor executor))
+                                              excutorstats (.get (.get heartbeat "heartbeat") "stats")
                                               excutorstats (if excutorstats
                                                       (StatsUtil/thriftifyExecutorStats excutorstats))]
                                               
@@ -1930,7 +1898,7 @@
                                                                 (-> executor first task->component)
                                                                 host
                                                                 port
-                                                                (Utils/nullToZero (:uptime heartbeat)))
+                                                                (Utils/nullToZero (.get heartbeat "uptime")))
                                             (.set_stats excutorstats))
                                           ))
               topo-info  (TopologyInfo. storm-id

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 92ba807..10a1e47 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -21,7 +21,8 @@
   (:require [org.apache.storm.daemon [executor :as executor]])
 
   (:require [clojure.set :as set])
-  (:import [java.io File])
+  (:import [java.io File]
+           [org.apache.storm.stats StatsUtil])
   (:import [java.util.concurrent Executors]
            [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
            [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
@@ -66,18 +67,15 @@
 (defnk do-executor-heartbeats [worker :executors nil]
   ;; stats is how we know what executors are assigned to this worker 
   (let [stats (if-not executors
-                  (into {} (map (fn [e] {e nil}) (:executors worker)))
-                  (->> executors
+                  (StatsUtil/mkEmptyExecutorZkHbs (:executors worker))
+                  (StatsUtil/convertExecutorZkHbs (->> executors
                     (map (fn [e] {(executor/get-executor-id e) (executor/render-stats e)}))
-                    (apply merge)))
-        zk-hb {:storm-id (:storm-id worker)
-               :executor-stats stats
-               :uptime (. (:uptime worker) upTime)
-               :time-secs (Time/currentTimeSecs)
-               }]
+                    (apply merge))))
+        zk-hb (StatsUtil/mkZkWorkerHb (:storm-id worker) stats (. (:uptime worker) upTime))]
     ;; do the zookeeper heartbeat
     (try
-      (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (long (:port worker)) (thriftify-zk-worker-hb zk-hb))
+      (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (long (:port worker))
+        (StatsUtil/thriftifyZkWorkerHb zk-hb))
       (catch Exception exc
         (log-error exc "Worker failed to write heatbeats to ZK or Pacemaker...will retry")))))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index 66fc051..419cf2b 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -452,7 +452,7 @@
         assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
         taskbeats (.taskbeats state storm-id (:task->node+port assignment))
         heartbeats (dofor [id task-ids] (get taskbeats id))
-        stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
+        stats (dofor [hb heartbeats] (if hb (.get (.get hb "stats") stat-key) 0))]
     (reduce + stats)))
 
 (defn emitted-spout-tuples
@@ -460,16 +460,16 @@
   (aggregated-stat
     cluster-map
     storm-name
-    :emitted
+    "emitted"
     :component-ids (keys (.get_spouts topology))))
 
 (defn transferred-tuples
   [cluster-map storm-name]
-  (aggregated-stat cluster-map storm-name :transferred))
+  (aggregated-stat cluster-map storm-name "transferred"))
 
 (defn acked-tuples
   [cluster-map storm-name]
-  (aggregated-stat cluster-map storm-name :acked))
+  (aggregated-stat cluster-map storm-name "acked"))
 
 (defn simulate-wait
   [cluster-map]

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index b9cf2d7..0730d96 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -124,11 +124,11 @@
 
 (defn spout-summary?
   [topology s]
-  (= :spout (executor-summary-type topology s)))
+  (= "spout" (executor-summary-type topology s)))
 
 (defn bolt-summary?
   [topology s]
-  (= :bolt (executor-summary-type topology s)))
+  (= "bolt" (executor-summary-type topology s)))
 
 (defn group-by-comp
   [summs]
@@ -230,8 +230,8 @@
   (let [components (for [[id spec] spout-bolt]
             [id
              (let [inputs (.get_inputs (.get_common spec))
-                   bolt-summs (get bolt-comp-summs id)
-                   spout-summs (get spout-comp-summs id)
+                   bolt-summs (.get bolt-comp-summs id)
+                   spout-summs (.get spout-comp-summs id)
                    bolt-cap (if bolt-summs
                               (StatsUtil/computeBoltCapacity bolt-summs)
                               0)]
@@ -240,17 +240,17 @@
                 :latency (if bolt-summs
                            (get-in
                              (clojurify-structure (StatsUtil/boltStreamsStats bolt-summs true))
-                             [:process-latencies window])
+                             ["process-latencies" window])
                            (get-in
                              (clojurify-structure (StatsUtil/spoutStreamsStats spout-summs true))
-                             [:complete-latencies window]))
+                             ["complete-latencies" window]))
                 :transferred (or
                                (get-in
                                  (clojurify-structure (StatsUtil/spoutStreamsStats spout-summs true))
-                                 [:transferred window])
+                                 ["transferred" window])
                                (get-in
                                  (clojurify-structure (StatsUtil/boltStreamsStats bolt-summs true))
-                                 [:transferred window]))
+                                 ["transferred" window]))
                 :stats (let [mapfn (fn [dat]
                                      (map (fn [^ExecutorSummary summ]
                                             {:host (.get_host summ)

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
index d8c7f06..e26e56b 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/BoltExecutorStats.java
@@ -18,9 +18,10 @@
 package org.apache.storm.stats;
 
 import com.google.common.collect.Lists;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
+import org.apache.storm.generated.BoltStats;
+import org.apache.storm.generated.ExecutorSpecificStats;
+import org.apache.storm.generated.ExecutorStats;
 import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 
@@ -33,8 +34,6 @@ public class BoltExecutorStats extends CommonStats {
     public static final String PROCESS_LATENCIES = "process-latencies";
     public static final String EXECUTE_LATENCIES = "execute-latencies";
 
-    public static final String[] BOLT_FIELDS = {ACKED, FAILED, EXECUTED, PROCESS_LATENCIES, EXECUTE_LATENCIES};
-
     public BoltExecutorStats(int rate) {
         super(rate);
 
@@ -83,32 +82,24 @@ public class BoltExecutorStats extends CommonStats {
 
     }
 
-    public Map renderStats() {
+    public ExecutorStats renderStats() {
         cleanupStats();
-        Map ret = new HashMap();
-        ret.putAll(valueStats(CommonStats.COMMON_FIELDS));
-        ret.putAll(valueStats(BoltExecutorStats.BOLT_FIELDS));
-        StatsUtil.putKV(ret, StatsUtil.TYPE, StatsUtil.KW_BOLT);
+
+        ExecutorStats ret = new ExecutorStats();
+        // common stats
+        ret.set_emitted(valueStat(EMITTED));
+        ret.set_transferred(valueStat(TRANSFERRED));
+        ret.set_rate(this.rate);
+
+        // bolt stats
+        BoltStats boltStats = new BoltStats(
+                StatsUtil.windowSetConverter(valueStat(ACKED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+                StatsUtil.windowSetConverter(valueStat(FAILED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+                StatsUtil.windowSetConverter(valueStat(PROCESS_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+                StatsUtil.windowSetConverter(valueStat(EXECUTED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
+                StatsUtil.windowSetConverter(valueStat(EXECUTE_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY));
+        ret.set_specific(ExecutorSpecificStats.bolt(boltStats));
 
         return ret;
     }
-
-//    public ExecutorStats renderStats() {
-//        cleanupStats();
-//
-//        ExecutorStats ret = new ExecutorStats();
-//        ret.set_emitted(valueStat(EMITTED));
-//        ret.set_transferred(valueStat(TRANSFERRED));
-//        ret.set_rate(this.rate);
-//
-//        BoltStats boltStats = new BoltStats(
-//                StatsUtil.windowSetConverter(valueStat(ACKED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-//                StatsUtil.windowSetConverter(valueStat(FAILED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-//                StatsUtil.windowSetConverter(valueStat(PROCESS_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-//                StatsUtil.windowSetConverter(valueStat(EXECUTED), StatsUtil.TO_GSID, StatsUtil.IDENTITY),
-//                StatsUtil.windowSetConverter(valueStat(EXECUTE_LATENCIES), StatsUtil.TO_GSID, StatsUtil.IDENTITY));
-//        ret.set_specific(ExecutorSpecificStats.bolt(boltStats));
-//
-//        return ret;
-//    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/4c246d1c/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
index 27c626e..3c09a38 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/SpoutExecutorStats.java
@@ -19,6 +19,9 @@ package org.apache.storm.stats;
 
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.storm.generated.ExecutorSpecificStats;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.SpoutStats;
 import org.apache.storm.metric.internal.MultiCountStatAndMetric;
 import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
 
@@ -29,8 +32,6 @@ public class SpoutExecutorStats extends CommonStats {
     public static final String FAILED = "failed";
     public static final String COMPLETE_LATENCIES = "complete-latencies";
 
-    public static final String[] SPOUT_FIELDS = {ACKED, FAILED, COMPLETE_LATENCIES};
-
     public SpoutExecutorStats(int rate) {
         super(rate);
         this.put(ACKED, new MultiCountStatAndMetric(NUM_STAT_BUCKETS));
@@ -59,28 +60,20 @@ public class SpoutExecutorStats extends CommonStats {
         this.getFailed().incBy(stream, this.rate);
     }
 
-    public Map renderStats() {
+    public ExecutorStats renderStats() {
         cleanupStats();
-        Map ret = new HashMap();
-        ret.putAll(valueStats(CommonStats.COMMON_FIELDS));
-        ret.putAll(valueStats(SpoutExecutorStats.SPOUT_FIELDS));
-        StatsUtil.putKV(ret, StatsUtil.TYPE, StatsUtil.KW_SPOUT);
+
+        ExecutorStats ret = new ExecutorStats();
+        // common fields
+        ret.set_emitted(valueStat(EMITTED));
+        ret.set_transferred(valueStat(TRANSFERRED));
+        ret.set_rate(this.rate);
+
+        // spout stats
+        SpoutStats spoutStats = new SpoutStats(
+                valueStat(ACKED), valueStat(FAILED), valueStat(COMPLETE_LATENCIES));
+        ret.set_specific(ExecutorSpecificStats.spout(spoutStats));
 
         return ret;
     }
-
-//    public ExecutorStats renderStats() {
-//        cleanupStats();
-//
-//        ExecutorStats ret = new ExecutorStats();
-//        ret.set_emitted(valueStat(EMITTED));
-//        ret.set_transferred(valueStat(TRANSFERRED));
-//        ret.set_rate(this.rate);
-//
-//        SpoutStats spoutStats = new SpoutStats(
-//                valueStat(ACKED), valueStat(FAILED), valueStat(COMPLETE_LATENCIES));
-//        ret.set_specific(ExecutorSpecificStats.spout(spoutStats));
-//
-//        return ret;
-//    }
 }


[03/30] storm git commit: add translated java files

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/52d3b587/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
new file mode 100644
index 0000000..144872f
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -0,0 +1,2178 @@
+/**
+ * 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 clojure.lang.Keyword;
+import clojure.lang.PersistentVector;
+import clojure.lang.RT;
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.BoltAggregateStats;
+import org.apache.storm.generated.BoltStats;
+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.TopologyPageInfo;
+import org.apache.storm.generated.TopologyStats;
+import org.apache.storm.metric.api.IMetric;
+import org.apache.storm.metric.internal.MultiCountStatAndMetric;
+import org.apache.storm.metric.internal.MultiLatencyStatAndMetric;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("unchecked, unused")
+public class StatsUtil {
+    private static final Logger logger = LoggerFactory.getLogger(StatsUtil.class);
+
+    private static final String TYPE = "type";
+    private static final String SPOUT = "spout";
+    private static final String BOLT = "bolt";
+
+    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 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 Keyword KW_SPOUT = keyword(SPOUT);
+    private static final Keyword KW_BOLT = keyword(BOLT);
+
+    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 IdentityTransformer IDENTITY = new IdentityTransformer();
+    private static final ToStringTransformer TO_STRING = new ToStringTransformer();
+    private static final FromGlobalStreamIdTransformer FROM_GSID = new FromGlobalStreamIdTransformer();
+    private static final ToGlobalStreamIdTransformer TO_GSID = new ToGlobalStreamIdTransformer();
+
+
+    // =====================================================================================
+    // update stats methods
+    // =====================================================================================
+
+    public static BoltExecutorStats mkBoltStats(int rate) {
+        BoltExecutorStats stats = new BoltExecutorStats();
+        stats.setRate(rate);
+        return stats;
+    }
+
+    public static SpoutExecutorStats mkSpoutStats(int rate) {
+        SpoutExecutorStats stats = new SpoutExecutorStats();
+        stats.setRate(rate);
+        return stats;
+    }
+
+    public static void emittedTuple(CommonStats stats, String stream) {
+        stats.getEmitted().incBy(stream, stats.rate);
+    }
+
+    public static void transferredTuples(CommonStats stats, String stream, int amount) {
+        stats.getTransferred().incBy(stream, stats.rate * amount);
+    }
+
+    public static void boltExecuteTuple(BoltExecutorStats stats, String component, String stream, long latencyMs) {
+        Object key = PersistentVector.create(component, stream);
+        stats.getExecuted().incBy(key, stats.rate);
+        stats.getExecuteLatencies().record(key, latencyMs);
+    }
+
+    public static void boltAckedTuple(BoltExecutorStats stats, String component, String stream, long latencyMs) {
+        Object key = PersistentVector.create(component, stream);
+        stats.getAcked().incBy(key, stats.rate);
+        stats.getProcessLatencies().record(key, latencyMs);
+    }
+
+    public static void boltFailedTuple(BoltExecutorStats stats, String component, String stream, long latencyMs) {
+        Object key = PersistentVector.create(component, stream);
+        stats.getFailed().incBy(key, stats.rate);
+
+    }
+
+    public static void spoutAckedTuple(SpoutExecutorStats stats, String stream, long latencyMs) {
+        stats.getAcked().incBy(stream, stats.rate);
+        stats.getCompleteLatencies().record(stream, latencyMs);
+    }
+
+    public static void spoutFailedTuple(SpoutExecutorStats stats, String stream, long latencyMs) {
+        stats.getFailed().incBy(stream, stats.rate);
+    }
+
+    private static void cleanupStat(IMetric metric) {
+        if (metric instanceof MultiCountStatAndMetric) {
+            ((MultiCountStatAndMetric) metric).close();
+        } else if (metric instanceof MultiLatencyStatAndMetric) {
+            ((MultiLatencyStatAndMetric) metric).close();
+        }
+    }
+
+    public static Map renderStats(SpoutExecutorStats stats) {
+        cleanupSpoutStats(stats);
+        Map ret = new HashMap();
+        ret.putAll(valueStats(stats, CommonStats.COMMON_FIELDS));
+        ret.putAll(valueStats(stats, SpoutExecutorStats.SPOUT_FIELDS));
+        putRawKV(ret, TYPE, KW_SPOUT);
+
+        return ret;
+    }
+
+    public static Map renderStats(BoltExecutorStats stats) {
+        cleanupBoltStats(stats);
+        Map ret = new HashMap();
+        ret.putAll(valueStats(stats, CommonStats.COMMON_FIELDS));
+        ret.putAll(valueStats(stats, BoltExecutorStats.BOLT_FIELDS));
+        putRawKV(ret, TYPE, KW_BOLT);
+
+        return ret;
+    }
+
+    public static void cleanupSpoutStats(SpoutExecutorStats stats) {
+        cleanupCommonStats(stats);
+        for (String field : SpoutExecutorStats.SPOUT_FIELDS) {
+            cleanupStat(stats.get(field));
+        }
+    }
+
+    public static void cleanupBoltStats(BoltExecutorStats stats) {
+        cleanupCommonStats(stats);
+        for (String field : BoltExecutorStats.BOLT_FIELDS) {
+            cleanupStat(stats.get(field));
+        }
+    }
+
+    public static void cleanupCommonStats(CommonStats stats) {
+        for (String field : CommonStats.COMMON_FIELDS) {
+            cleanupStat(stats.get(field));
+        }
+    }
+
+    private static Map valueStats(CommonStats stats, String[] fields) {
+        Map ret = new HashMap();
+        for (String field : fields) {
+            IMetric metric = stats.get(field);
+            if (metric instanceof MultiCountStatAndMetric) {
+                putRawKV(ret, field, ((MultiCountStatAndMetric) metric).getTimeCounts());
+            } else if (metric instanceof MultiLatencyStatAndMetric) {
+                putRawKV(ret, field, ((MultiLatencyStatAndMetric) metric).getTimeLatAvg());
+            }
+        }
+        putRawKV(ret, CommonStats.RATE, stats.getRate());
+
+        return ret;
+    }
+
+    // =====================================================================================
+    // 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 aggBoltLatAndCount(Map id2execAvg, Map id2procAvg, Map id2numExec) {
+        Map ret = new HashMap();
+        putRawKV(ret, EXEC_LAT_TOTAL, weightAvgAndSum(id2execAvg, id2numExec));
+        putRawKV(ret, PROC_LAT_TOTAL, weightAvgAndSum(id2procAvg, id2numExec));
+        putRawKV(ret, EXECUTED, sumValues(id2numExec));
+
+        return ret;
+    }
+
+    /**
+     * Aggregates number acked and complete latencies across all streams.
+     */
+    public static Map aggSpoutLatAndCount(Map id2compAvg, Map id2numAcked) {
+        Map ret = new HashMap();
+        putRawKV(ret, COMP_LAT_TOTAL, weightAvgAndSum(id2compAvg, id2numAcked));
+        putRawKV(ret, ACKED, sumValues(id2numAcked));
+
+        return ret;
+    }
+
+    /**
+     * Aggregates number executed and process & execute latencies.
+     */
+    public static Map aggBoltStreamsLatAndCount(Map id2execAvg, Map id2procAvg, Map id2numExec) {
+        Map ret = new HashMap();
+        if (id2execAvg == null || id2procAvg == null || id2numExec == null) {
+            return ret;
+        }
+        for (Object k : id2execAvg.keySet()) {
+            Map subMap = new HashMap();
+            putRawKV(subMap, EXEC_LAT_TOTAL, weightAvg(id2execAvg, id2numExec, k));
+            putRawKV(subMap, PROC_LAT_TOTAL, weightAvg(id2procAvg, id2numExec, k));
+            putRawKV(subMap, EXECUTED, id2numExec.get(k));
+            ret.put(k, subMap);
+        }
+        return ret;
+    }
+
+    /**
+     * Aggregates number acked and complete latencies.
+     */
+    public static Map aggSpoutStreamsLatAndCount(Map id2compAvg, Map id2acked) {
+        Map ret = new HashMap();
+        if (id2compAvg == null || id2acked == null) {
+            return ret;
+        }
+        for (Object k : id2compAvg.keySet()) {
+            Map subMap = new HashMap();
+            putRawKV(subMap, COMP_LAT_TOTAL, weightAvg(id2compAvg, id2acked, k));
+            putRawKV(subMap, ACKED, id2acked.get(k));
+            ret.put(k, subMap);
+        }
+        return ret;
+    }
+
+    public static Map aggPreMergeCompPageBolt(Map m, String window, boolean includeSys) {
+        Map ret = new HashMap();
+        putRawKV(ret, EXECUTOR_ID, getByKeyword(m, "exec-id"));
+        putRawKV(ret, HOST, getByKeyword(m, HOST));
+        putRawKV(ret, PORT, getByKeyword(m, PORT));
+        putRawKV(ret, UPTIME, getByKeyword(m, UPTIME));
+        putRawKV(ret, NUM_EXECUTORS, 1);
+        putRawKV(ret, NUM_TASKS, getByKeyword(m, NUM_TASKS));
+
+        Map stat2win2sid2num = getMapByKeyword(m, STATS);
+        putRawKV(ret, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
+
+        // calc cid+sid->input_stats
+        Map inputStats = new HashMap();
+        Map sid2acked = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, ACKED), TO_STRING).get(window);
+        Map sid2failed = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, FAILED), TO_STRING).get(window);
+        putRawKV(inputStats, ACKED, sid2acked != null ? sid2acked : new HashMap());
+        putRawKV(inputStats, FAILED, sid2failed != null ? sid2failed : new HashMap());
+
+        inputStats = swapMapOrder(inputStats);
+
+        Map sid2execLat = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, EXEC_LATENCIES), TO_STRING).get(window);
+        Map sid2procLat = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, PROC_LATENCIES), TO_STRING).get(window);
+        Map sid2exec = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, EXECUTED), TO_STRING).get(window);
+        mergeMaps(inputStats, aggBoltStreamsLatAndCount(sid2execLat, sid2procLat, sid2exec));
+        putRawKV(ret, CID_SID_TO_IN_STATS, inputStats);
+
+        // calc sid->output_stats
+        Map outputStats = new HashMap();
+        Map sid2emitted = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, EMITTED), TO_STRING).get(window);
+        Map sid2transferred = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, TRANSFERRED), TO_STRING).get(window);
+        if (sid2emitted != null) {
+            putRawKV(outputStats, EMITTED, filterSysStreams(sid2emitted, includeSys));
+        } else {
+            putRawKV(outputStats, EMITTED, new HashMap());
+        }
+        if (sid2transferred != null) {
+            putRawKV(outputStats, TRANSFERRED, filterSysStreams(sid2transferred, includeSys));
+        } else {
+            putRawKV(outputStats, TRANSFERRED, new HashMap());
+        }
+        outputStats = swapMapOrder(outputStats);
+        putRawKV(ret, SID_TO_OUT_STATS, outputStats);
+
+        return ret;
+    }
+
+    public static Map aggPreMergeCompPageSpout(Map m, String window, boolean includeSys) {
+        Map ret = new HashMap();
+        putRawKV(ret, EXECUTOR_ID, getByKeyword(m, "exec-id"));
+        putRawKV(ret, HOST, getByKeyword(m, HOST));
+        putRawKV(ret, PORT, getByKeyword(m, PORT));
+        putRawKV(ret, UPTIME, getByKeyword(m, UPTIME));
+        putRawKV(ret, NUM_EXECUTORS, 1);
+        putRawKV(ret, NUM_TASKS, getByKeyword(m, NUM_TASKS));
+
+        Map stat2win2sid2num = getMapByKeyword(m, STATS);
+
+        // calc sid->output-stats
+        Map outputStats = new HashMap();
+        Map win2sid2acked = windowSetConverter(getMapByKeyword(stat2win2sid2num, ACKED), TO_STRING);
+        Map win2sid2failed = windowSetConverter(getMapByKeyword(stat2win2sid2num, FAILED), TO_STRING);
+        Map win2sid2emitted = windowSetConverter(getMapByKeyword(stat2win2sid2num, EMITTED), TO_STRING);
+        Map win2sid2transferred = windowSetConverter(getMapByKeyword(stat2win2sid2num, TRANSFERRED), TO_STRING);
+        Map win2sid2compLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
+
+        putRawKV(outputStats, ACKED, win2sid2acked.get(window));
+        putRawKV(outputStats, FAILED, win2sid2failed.get(window));
+        putRawKV(outputStats, EMITTED, filterSysStreams((Map) win2sid2emitted.get(window), includeSys));
+        putRawKV(outputStats, TRANSFERRED, filterSysStreams((Map) win2sid2transferred.get(window), includeSys));
+        outputStats = swapMapOrder(outputStats);
+
+        Map sid2compLat = (Map) win2sid2compLat.get(window);
+        Map sid2acked = (Map) win2sid2acked.get(window);
+        mergeMaps(outputStats, aggSpoutStreamsLatAndCount(sid2compLat, sid2acked));
+        putRawKV(ret, SID_TO_OUT_STATS, outputStats);
+
+        return ret;
+    }
+
+    public static Map aggPreMergeTopoPageBolt(Map m, String window, boolean includeSys) {
+        Map ret = new HashMap();
+
+        Map subRet = new HashMap();
+        putRawKV(subRet, NUM_EXECUTORS, 1);
+        putRawKV(subRet, NUM_TASKS, getByKeyword(m, NUM_TASKS));
+
+        Map stat2win2sid2num = getMapByKeyword(m, STATS);
+        putRawKV(subRet, CAPACITY, computeAggCapacity(stat2win2sid2num, getByKeywordOr0(m, UPTIME).intValue()));
+
+        for (String key : new String[]{EMITTED, TRANSFERRED, ACKED, FAILED}) {
+            Map stat = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, key), TO_STRING).get(window);
+            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
+                stat = filterSysStreams(stat, includeSys);
+            }
+            long sum = 0;
+            if (stat != null) {
+                for (Object o : stat.values()) {
+                    sum += ((Number) o).longValue();
+                }
+            }
+            putRawKV(subRet, key, sum);
+        }
+
+        Map win2sid2execLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, EXEC_LATENCIES), TO_STRING);
+        Map win2sid2procLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, PROC_LATENCIES), TO_STRING);
+        Map win2sid2exec = windowSetConverter(getMapByKeyword(stat2win2sid2num, EXECUTED), TO_STRING);
+        subRet.putAll(aggBoltLatAndCount(
+                (Map) win2sid2execLat.get(window), (Map) win2sid2procLat.get(window), (Map) win2sid2exec.get(window)));
+
+        ret.put(getByKeyword(m, "comp-id"), subRet);
+        return ret;
+    }
+
+    public static Map aggPreMergeTopoPageSpout(Map m, String window, boolean includeSys) {
+        Map ret = new HashMap();
+
+        Map subRet = new HashMap();
+        putRawKV(subRet, NUM_EXECUTORS, 1);
+        putRawKV(subRet, NUM_TASKS, getByKeyword(m, NUM_TASKS));
+
+        // no capacity for spout
+        Map stat2win2sid2num = getMapByKeyword(m, STATS);
+        for (String key : new String[]{EMITTED, TRANSFERRED, FAILED}) {
+            Map stat = (Map) windowSetConverter(getMapByKeyword(stat2win2sid2num, key), TO_STRING).get(window);
+            if (EMITTED.equals(key) || TRANSFERRED.equals(key)) {
+                stat = filterSysStreams(stat, includeSys);
+            }
+            long sum = 0;
+            if (stat != null) {
+                for (Object o : stat.values()) {
+                    sum += ((Number) o).longValue();
+                }
+            }
+            putRawKV(subRet, key, sum);
+        }
+
+        Map win2sid2compLat = windowSetConverter(getMapByKeyword(stat2win2sid2num, COMP_LATENCIES), TO_STRING);
+        Map win2sid2acked = windowSetConverter(getMapByKeyword(stat2win2sid2num, ACKED), TO_STRING);
+        subRet.putAll(aggSpoutLatAndCount((Map) win2sid2compLat.get(window), (Map) win2sid2acked.get(window)));
+
+        ret.put(getByKeyword(m, "comp-id"), subRet);
+        return ret;
+    }
+
+    public static Map mergeAggCompStatsCompPageBolt(Map accBoltStats, Map boltStats) {
+        Map ret = new HashMap();
+
+        Map accIn = getMapByKeyword(accBoltStats, CID_SID_TO_IN_STATS);
+        Map accOut = getMapByKeyword(accBoltStats, SID_TO_OUT_STATS);
+        Map boltIn = getMapByKeyword(boltStats, CID_SID_TO_IN_STATS);
+        Map boltOut = getMapByKeyword(boltStats, SID_TO_OUT_STATS);
+
+        int numExecutors = getByKeywordOr0(accBoltStats, NUM_EXECUTORS).intValue();
+        putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putRawKV(ret, NUM_TASKS, sumOr0(
+                getByKeywordOr0(accBoltStats, NUM_TASKS), getByKeywordOr0(boltStats, NUM_TASKS)));
+
+        // (merge-with (partial merge-with sum-or-0) acc-out spout-out)
+        putRawKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, boltOut));
+        putRawKV(ret, CID_SID_TO_IN_STATS, fullMergeWithSum(accIn, boltIn));
+
+        long executed = sumStreamsLong(boltIn, EXECUTED);
+        putRawKV(ret, EXECUTED, executed);
+
+        Map executorStats = new HashMap();
+        putRawKV(executorStats, EXECUTOR_ID, getByKeyword(boltStats, EXECUTOR_ID));
+        putRawKV(executorStats, UPTIME, getByKeyword(boltStats, UPTIME));
+        putRawKV(executorStats, HOST, getByKeyword(boltStats, HOST));
+        putRawKV(executorStats, PORT, getByKeyword(boltStats, PORT));
+        putRawKV(executorStats, CAPACITY, getByKeyword(boltStats, CAPACITY));
+
+        putRawKV(executorStats, EMITTED, sumStreamsLong(boltOut, EMITTED));
+        putRawKV(executorStats, TRANSFERRED, sumStreamsLong(boltOut, TRANSFERRED));
+        putRawKV(executorStats, ACKED, sumStreamsLong(boltIn, ACKED));
+        putRawKV(executorStats, FAILED, sumStreamsLong(boltIn, FAILED));
+        putRawKV(executorStats, EXECUTED, executed);
+
+        if (executed > 0) {
+            putRawKV(executorStats, EXEC_LATENCY, sumStreamsDouble(boltIn, EXEC_LAT_TOTAL) / executed);
+            putRawKV(executorStats, PROC_LATENCY, sumStreamsDouble(boltIn, PROC_LAT_TOTAL) / executed);
+        } else {
+            putRawKV(executorStats, EXEC_LATENCY, null);
+            putRawKV(executorStats, PROC_LATENCY, null);
+        }
+        List executorStatsList = ((List) getByKeyword(accBoltStats, EXECUTOR_STATS));
+        executorStatsList.add(executorStats);
+        putRawKV(ret, EXECUTOR_STATS, executorStatsList);
+
+        return ret;
+    }
+
+    public static Map mergeAggCompStatsCompPageSpout(Map accSpoutStats, Map spoutStats) {
+        Map ret = new HashMap();
+
+        Map accOut = getMapByKeyword(accSpoutStats, SID_TO_OUT_STATS);
+        Map spoutOut = getMapByKeyword(spoutStats, SID_TO_OUT_STATS);
+
+        int numExecutors = getByKeywordOr0(accSpoutStats, NUM_EXECUTORS).intValue();
+        putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putRawKV(ret, NUM_TASKS, sumOr0(
+                getByKeywordOr0(accSpoutStats, NUM_TASKS), getByKeywordOr0(spoutStats, NUM_TASKS)));
+        putRawKV(ret, SID_TO_OUT_STATS, fullMergeWithSum(accOut, spoutOut));
+
+        Map executorStats = new HashMap();
+        putRawKV(executorStats, EXECUTOR_ID, getByKeyword(spoutStats, EXECUTOR_ID));
+        putRawKV(executorStats, UPTIME, getByKeyword(spoutStats, UPTIME));
+        putRawKV(executorStats, HOST, getByKeyword(spoutStats, HOST));
+        putRawKV(executorStats, PORT, getByKeyword(spoutStats, PORT));
+
+        putRawKV(executorStats, EMITTED, sumStreamsLong(spoutOut, EMITTED));
+        putRawKV(executorStats, TRANSFERRED, sumStreamsLong(spoutOut, TRANSFERRED));
+        putRawKV(executorStats, FAILED, sumStreamsLong(spoutOut, FAILED));
+        long acked = sumStreamsLong(spoutOut, ACKED);
+        putRawKV(executorStats, ACKED, acked);
+        if (acked > 0) {
+            putRawKV(executorStats, COMP_LATENCY, sumStreamsDouble(spoutOut, COMP_LAT_TOTAL) / acked);
+        } else {
+            putRawKV(executorStats, COMP_LATENCY, null);
+        }
+        List executorStatsList = ((List) getByKeyword(accSpoutStats, EXECUTOR_STATS));
+        executorStatsList.add(executorStats);
+        putRawKV(ret, EXECUTOR_STATS, executorStatsList);
+
+        return ret;
+    }
+
+    public static Map mergeAggCompStatsTopoPageBolt(Map accBoltStats, Map boltStats) {
+        Map ret = new HashMap();
+        Integer numExecutors = getByKeywordOr0(accBoltStats, NUM_EXECUTORS).intValue();
+        putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putRawKV(ret, NUM_TASKS, sumOr0(
+                getByKeywordOr0(accBoltStats, NUM_TASKS), getByKeywordOr0(boltStats, NUM_TASKS)));
+        putRawKV(ret, EMITTED, sumOr0(
+                getByKeywordOr0(accBoltStats, EMITTED), getByKeywordOr0(boltStats, EMITTED)));
+        putRawKV(ret, TRANSFERRED, sumOr0(
+                getByKeywordOr0(accBoltStats, TRANSFERRED), getByKeywordOr0(boltStats, TRANSFERRED)));
+        putRawKV(ret, EXEC_LAT_TOTAL, sumOr0(
+                getByKeywordOr0(accBoltStats, EXEC_LAT_TOTAL), getByKeywordOr0(boltStats, EXEC_LAT_TOTAL)));
+        putRawKV(ret, PROC_LAT_TOTAL, sumOr0(
+                getByKeywordOr0(accBoltStats, PROC_LAT_TOTAL), getByKeywordOr0(boltStats, PROC_LAT_TOTAL)));
+        putRawKV(ret, EXECUTED, sumOr0(
+                getByKeywordOr0(accBoltStats, EXECUTED), getByKeywordOr0(boltStats, EXECUTED)));
+        putRawKV(ret, ACKED, sumOr0(
+                getByKeywordOr0(accBoltStats, ACKED), getByKeywordOr0(boltStats, ACKED)));
+        putRawKV(ret, FAILED, sumOr0(
+                getByKeywordOr0(accBoltStats, FAILED), getByKeywordOr0(boltStats, FAILED)));
+        putRawKV(ret, CAPACITY, maxOr0(
+                getByKeywordOr0(accBoltStats, CAPACITY), getByKeywordOr0(boltStats, CAPACITY)));
+
+        return ret;
+    }
+
+    public static Map mergeAggCompStatsTopoPageSpout(Map accSpoutStats, Map spoutStats) {
+        Map ret = new HashMap();
+        Integer numExecutors = getByKeywordOr0(accSpoutStats, NUM_EXECUTORS).intValue();
+        putRawKV(ret, NUM_EXECUTORS, numExecutors + 1);
+        putRawKV(ret, NUM_TASKS, sumOr0(
+                getByKeywordOr0(accSpoutStats, NUM_TASKS), getByKeywordOr0(spoutStats, NUM_TASKS)));
+        putRawKV(ret, EMITTED, sumOr0(
+                getByKeywordOr0(accSpoutStats, EMITTED), getByKeywordOr0(spoutStats, EMITTED)));
+        putRawKV(ret, TRANSFERRED, sumOr0(
+                getByKeywordOr0(accSpoutStats, TRANSFERRED), getByKeywordOr0(spoutStats, TRANSFERRED)));
+        putRawKV(ret, COMP_LAT_TOTAL, sumOr0(
+                getByKeywordOr0(accSpoutStats, COMP_LAT_TOTAL), getByKeywordOr0(spoutStats, COMP_LAT_TOTAL)));
+        putRawKV(ret, ACKED, sumOr0(
+                getByKeywordOr0(accSpoutStats, ACKED), getByKeywordOr0(spoutStats, ACKED)));
+        putRawKV(ret, FAILED, sumOr0(
+                getByKeywordOr0(accSpoutStats, FAILED), getByKeywordOr0(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 aggTopoExecStats(String window, boolean includeSys, Map accStats, Map newData, String compType) {
+        Map ret = new HashMap();
+
+        Set workerSet = (Set) getByKeyword(accStats, WORKERS_SET);
+        Map bolt2stats = getMapByKeyword(accStats, BOLT_TO_STATS);
+        Map spout2stats = getMapByKeyword(accStats, SPOUT_TO_STATS);
+        Map win2emitted = getMapByKeyword(accStats, WIN_TO_EMITTED);
+        Map win2transferred = getMapByKeyword(accStats, WIN_TO_TRANSFERRED);
+        Map win2compLatWgtAvg = getMapByKeyword(accStats, WIN_TO_COMP_LAT_WGT_AVG);
+        Map win2acked = getMapByKeyword(accStats, WIN_TO_ACKED);
+        Map win2failed = getMapByKeyword(accStats, WIN_TO_FAILED);
+        Map stats = getMapByKeyword(newData, STATS);
+
+        boolean isSpout = compType.equals(SPOUT);
+        Map cid2stat2num;
+        if (isSpout) {
+            cid2stat2num = aggPreMergeTopoPageSpout(newData, window, includeSys);
+        } else {
+            cid2stat2num = aggPreMergeTopoPageBolt(newData, window, includeSys);
+        }
+
+        Map w2compLatWgtAvg, w2acked;
+        Map compLatStats = getMapByKeyword(stats, COMP_LATENCIES);
+        if (isSpout) { // agg spout stats
+            Map mm = new HashMap();
+
+            Map acked = getMapByKeyword(stats, ACKED);
+            for (Object win : acked.keySet()) {
+                mm.put(win, aggSpoutLatAndCount((Map) compLatStats.get(win), (Map) acked.get(win)));
+            }
+            mm = swapMapOrder(mm);
+            w2compLatWgtAvg = getMapByKeyword(mm, COMP_LAT_TOTAL);
+            w2acked = getMapByKeyword(mm, ACKED);
+        } else {
+            w2compLatWgtAvg = null;
+            w2acked = aggregateCountStreams(getMapByKeyword(stats, ACKED));
+        }
+
+        workerSet.add(Lists.newArrayList(getByKeyword(newData, HOST), getByKeyword(newData, PORT)));
+        putRawKV(ret, WORKERS_SET, workerSet);
+        putRawKV(ret, BOLT_TO_STATS, bolt2stats);
+        putRawKV(ret, SPOUT_TO_STATS, spout2stats);
+        putRawKV(ret, WIN_TO_EMITTED, mergeWithSum(win2emitted, aggregateCountStreams(
+                filterSysStreams(getMapByKeyword(stats, EMITTED), includeSys))));
+        putRawKV(ret, WIN_TO_TRANSFERRED, mergeWithSum(win2transferred, aggregateCountStreams(
+                filterSysStreams(getMapByKeyword(stats, TRANSFERRED), includeSys))));
+        putRawKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(win2compLatWgtAvg, w2compLatWgtAvg));
+
+        //boolean isSpoutStat = SPOUT.equals(((Keyword) getByKeyword(stats, TYPE)).getName());
+        putRawKV(ret, WIN_TO_ACKED, isSpout ? mergeWithSum(win2acked, w2acked) : win2acked);
+        putRawKV(ret, WIN_TO_FAILED, isSpout ?
+                mergeWithSum(aggregateCountStreams(getMapByKeyword(stats, FAILED)), win2failed) : win2failed);
+        putRawKV(ret, TYPE, getByKeyword(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) {
+            Set<Object> keySet = new HashSet<>();
+            keySet.addAll(spout2stats.keySet());
+            keySet.addAll(cid2stat2num.keySet());
+
+            Map mm = new HashMap();
+            for (Object k : keySet) {
+                mm.put(k, mergeAggCompStatsTopoPageSpout((Map) spout2stats.get(k), (Map) cid2stat2num.get(k)));
+            }
+            putRawKV(ret, SPOUT_TO_STATS, mm);
+        } else {
+            Set<Object> keySet = new HashSet<>();
+            keySet.addAll(bolt2stats.keySet());
+            keySet.addAll(cid2stat2num.keySet());
+
+            Map mm = new HashMap();
+            for (Object k : keySet) {
+                mm.put(k, mergeAggCompStatsTopoPageBolt((Map) bolt2stats.get(k), (Map) cid2stat2num.get(k)));
+            }
+            putRawKV(ret, BOLT_TO_STATS, mm);
+        }
+
+        return ret;
+    }
+
+    // TODO: add last-error-fn arg to get last error
+    public static TopologyPageInfo aggTopoExecsStats(
+            String topologyId, Map exec2nodePort, Map task2component,
+            Map beats, StormTopology topology, String window, boolean includeSys) {
+        List beatList = extractDataFromHb(exec2nodePort, task2component, beats, includeSys, topology);
+        Map topoStats = aggregateTopoStats(window, includeSys, beatList);
+        topoStats = postAggregateTopoStats(task2component, exec2nodePort, topoStats);
+
+        return thriftifyTopoPageData(topologyId, topoStats);
+    }
+
+    public static Map aggregateTopoStats(String win, boolean includeSys, List data) {
+        Map initVal = new HashMap();
+        putRawKV(initVal, WORKERS_SET, new HashSet());
+        putRawKV(initVal, BOLT_TO_STATS, new HashMap());
+        putRawKV(initVal, SPOUT_TO_STATS, new HashMap());
+        putRawKV(initVal, WIN_TO_EMITTED, new HashMap());
+        putRawKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
+        putRawKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
+        putRawKV(initVal, WIN_TO_ACKED, new HashMap());
+        putRawKV(initVal, WIN_TO_FAILED, new HashMap());
+
+        for (Object o : data) {
+            Map newData = (Map) o;
+            String compType = ((Keyword) getByKeyword(newData, TYPE)).getName();
+            initVal = aggTopoExecStats(win, includeSys, initVal, newData, compType);
+        }
+
+        return initVal;
+    }
+
+    public static Map postAggregateTopoStats(Map task2comp, Map exec2nodePort, Map accData) {
+        Map ret = new HashMap();
+        putRawKV(ret, NUM_TASKS, task2comp.size());
+        putRawKV(ret, NUM_WORKERS, ((Set) getByKeyword(accData, WORKERS_SET)).size());
+        putRawKV(ret, NUM_EXECUTORS, exec2nodePort.size());
+
+        Map bolt2stats = getMapByKeyword(accData, BOLT_TO_STATS);
+        Map 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 = getByKeywordOr0(m, EXECUTED).longValue();
+            if (executed > 0) {
+                double execLatencyTotal = getByKeywordOr0(m, EXEC_LAT_TOTAL).doubleValue();
+                putRawKV(m, EXEC_LATENCY, execLatencyTotal / executed);
+
+                double procLatencyTotal = getByKeywordOr0(m, PROC_LAT_TOTAL).doubleValue();
+                putRawKV(m, PROC_LATENCY, procLatencyTotal / executed);
+            }
+            removeByKeyword(m, EXEC_LAT_TOTAL);
+            removeByKeyword(m, PROC_LAT_TOTAL);
+            //TODO: get last error depends on cluster.clj
+            putRawKV(m, "last-error", null);
+
+            aggBolt2stats.put(id, m);
+        }
+        putRawKV(ret, BOLT_TO_STATS, aggBolt2stats);
+
+        Map spout2stats = getMapByKeyword(accData, SPOUT_TO_STATS);
+        Map spoutBolt2stats = 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 = getByKeywordOr0(m, ACKED).longValue();
+            if (acked > 0) {
+                double compLatencyTotal = getByKeywordOr0(m, COMP_LAT_TOTAL).doubleValue();
+                putRawKV(m, COMP_LATENCY, compLatencyTotal / acked);
+            }
+            removeByKeyword(m, COMP_LAT_TOTAL);
+            //TODO: get last error depends on cluster.clj
+            putRawKV(m, "last-error", null);
+
+            spoutBolt2stats.put(id, m);
+        }
+        putRawKV(ret, SPOUT_TO_STATS, spoutBolt2stats);
+
+        putRawKV(ret, WIN_TO_EMITTED, mapKeyStr(getMapByKeyword(accData, WIN_TO_EMITTED)));
+        putRawKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKeyword(accData, WIN_TO_TRANSFERRED)));
+        putRawKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKeyword(accData, WIN_TO_ACKED)));
+        putRawKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKeyword(accData, WIN_TO_FAILED)));
+        putRawKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
+                accData, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
+        return ret;
+    }
+
+    /**
+     * aggregate bolt stats
+     *
+     * @param statsSeq   a seq of ExecutorStats
+     * @param includeSys whether to include system streams
+     * @return aggregated bolt stats
+     */
+    public static Map aggregateBoltStats(List statsSeq, boolean includeSys) {
+        Map ret = new HashMap();
+
+        Map commonStats = preProcessStreamSummary(aggregateCommonStats(statsSeq), includeSys);
+        List acked = new ArrayList();
+        List failed = new ArrayList();
+        List executed = new ArrayList();
+        List processLatencies = new ArrayList();
+        List executeLatencies = new ArrayList();
+        for (Object o : statsSeq) {
+            ExecutorStats stat = (ExecutorStats) o;
+            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);
+        putRawKV(ret, ACKED, aggregateCounts(acked));
+        putRawKV(ret, FAILED, aggregateCounts(failed));
+        putRawKV(ret, EXECUTED, aggregateCounts(executed));
+        putRawKV(ret, PROC_LATENCIES, aggregateAverages(processLatencies, acked));
+        putRawKV(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
+     */
+    public static Map aggregateSpoutStats(List statsSeq, boolean includeSys) {
+        Map ret = new HashMap();
+
+        Map commonStats = preProcessStreamSummary(aggregateCommonStats(statsSeq), includeSys);
+        List acked = new ArrayList();
+        List failed = new ArrayList();
+        List completeLatencies = new ArrayList();
+        for (Object o : statsSeq) {
+            ExecutorStats stat = (ExecutorStats) o;
+            acked.add(stat.get_specific().get_spout().get_acked());
+            failed.add(stat.get_specific().get_spout().get_failed());
+            completeLatencies.add(stat.get_specific().get_spout().get_complete_ms_avg());
+        }
+        mergeMaps(ret, commonStats);
+        putRawKV(ret, ACKED, aggregateCounts(acked));
+        putRawKV(ret, FAILED, aggregateCounts(failed));
+        putRawKV(ret, COMP_LATENCIES, aggregateAverages(completeLatencies, acked));
+
+        return ret;
+    }
+
+    public static Map aggregateCommonStats(List statsSeq) {
+        Map ret = new HashMap();
+
+        List emitted = new ArrayList();
+        List transferred = new ArrayList();
+        for (Object o : statsSeq) {
+            ExecutorStats stat = (ExecutorStats) o;
+            emitted.add(stat.get_emitted());
+            transferred.add(stat.get_transferred());
+        }
+
+        putRawKV(ret, EMITTED, aggregateCounts(emitted));
+        putRawKV(ret, TRANSFERRED, aggregateCounts(transferred));
+        return ret;
+    }
+
+    public static Map preProcessStreamSummary(Map streamSummary, boolean includeSys) {
+        Map emitted = getMapByKeyword(streamSummary, EMITTED);
+        Map transferred = getMapByKeyword(streamSummary, TRANSFERRED);
+
+        putRawKV(streamSummary, EMITTED, filterSysStreams(emitted, includeSys));
+        putRawKV(streamSummary, TRANSFERRED, filterSysStreams(transferred, includeSys));
+
+        return streamSummary;
+    }
+
+    public static Map aggregateCountStreams(Map stats) {
+        Map ret = new HashMap();
+        for (Object o : stats.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            Map value = (Map) entry.getValue();
+            long sum = 0l;
+            for (Object num : value.values()) {
+                sum += ((Number) num).longValue();
+            }
+            ret.put(entry.getKey(), sum);
+        }
+        return ret;
+    }
+
+    public static Map aggregateAverages(List avgSeq, List countSeq) {
+        Map ret = new HashMap();
+
+        Map expands = expandAveragesSeq(avgSeq, countSeq);
+        for (Object o : expands.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            Object k = entry.getKey();
+
+            Map tmp = new HashMap();
+            Map inner = (Map) entry.getValue();
+            for (Object kk : inner.keySet()) {
+                List vv = (List) inner.get(kk);
+                tmp.put(kk, valAvg(((Number) vv.get(0)).doubleValue(), ((Number) vv.get(1)).longValue()));
+            }
+            ret.put(k, tmp);
+        }
+
+        return ret;
+    }
+
+    public static Map aggregateAvgStreams(Map avgs, Map counts) {
+        Map ret = new HashMap();
+
+        Map expands = expandAverages(avgs, counts);
+        for (Object o : expands.entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            Object win = e.getKey();
+
+            double avgTotal = 0.0;
+            long cntTotal = 0l;
+            Map inner = (Map) e.getValue();
+            for (Object kk : inner.keySet()) {
+                List vv = (List) inner.get(kk);
+                avgTotal += ((Number) vv.get(0)).doubleValue();
+                cntTotal += ((Number) vv.get(1)).longValue();
+            }
+            ret.put(win, valAvg(avgTotal, cntTotal));
+        }
+
+        return ret;
+    }
+
+    public static Map spoutStreamsStats(List summs, boolean includeSys) {
+        List statsSeq = getFilledStats(summs);
+        return aggregateSpoutStreams(aggregateSpoutStats(statsSeq, includeSys));
+    }
+
+    public static Map boltStreamsStats(List summs, boolean includeSys) {
+        List statsSeq = getFilledStats(summs);
+        return aggregateBoltStreams(aggregateBoltStats(statsSeq, includeSys));
+    }
+
+    public static Map aggregateSpoutStreams(Map stats) {
+        Map ret = new HashMap();
+        putRawKV(ret, ACKED, aggregateCountStreams(getMapByKeyword(stats, ACKED)));
+        putRawKV(ret, FAILED, aggregateCountStreams(getMapByKeyword(stats, FAILED)));
+        putRawKV(ret, EMITTED, aggregateCountStreams(getMapByKeyword(stats, EMITTED)));
+        putRawKV(ret, TRANSFERRED, aggregateCountStreams(getMapByKeyword(stats, TRANSFERRED)));
+        putRawKV(ret, COMP_LATENCIES, aggregateAvgStreams(
+                getMapByKeyword(stats, COMP_LATENCIES), getMapByKeyword(stats, ACKED)));
+        return ret;
+    }
+
+    public static Map aggregateBoltStreams(Map stats) {
+        Map ret = new HashMap();
+        putRawKV(ret, ACKED, aggregateCountStreams(getMapByKeyword(stats, ACKED)));
+        putRawKV(ret, FAILED, aggregateCountStreams(getMapByKeyword(stats, FAILED)));
+        putRawKV(ret, EMITTED, aggregateCountStreams(getMapByKeyword(stats, EMITTED)));
+        putRawKV(ret, TRANSFERRED, aggregateCountStreams(getMapByKeyword(stats, TRANSFERRED)));
+        putRawKV(ret, EXECUTED, aggregateCountStreams(getMapByKeyword(stats, EXECUTED)));
+        putRawKV(ret, PROC_LATENCIES, aggregateAvgStreams(
+                getMapByKeyword(stats, PROC_LATENCIES), getMapByKeyword(stats, ACKED)));
+        putRawKV(ret, EXEC_LATENCIES, aggregateAvgStreams(
+                getMapByKeyword(stats, EXEC_LATENCIES), getMapByKeyword(stats, EXECUTED)));
+        return ret;
+    }
+
+    /**
+     * A helper function that aggregates windowed stats from one spout executor.
+     */
+    public static Map aggBoltExecWinStats(Map accStats, Map newStats, boolean includeSys) {
+        Map ret = new HashMap();
+
+        Map m = new HashMap();
+        for (Object win : getMapByKeyword(newStats, EXECUTED).keySet()) {
+            m.put(win, aggBoltLatAndCount(
+                    (Map) (getMapByKeyword(newStats, EXEC_LATENCIES)).get(win),
+                    (Map) (getMapByKeyword(newStats, PROC_LATENCIES)).get(win),
+                    (Map) (getMapByKeyword(newStats, EXECUTED)).get(win)));
+        }
+        m = swapMapOrder(m);
+
+        Map win2execLatWgtAvg = getMapByKeyword(m, EXEC_LAT_TOTAL);
+        Map win2procLatWgtAvg = getMapByKeyword(m, PROC_LAT_TOTAL);
+        Map win2executed = getMapByKeyword(m, EXECUTED);
+
+        Map emitted = getMapByKeyword(newStats, EMITTED);
+        emitted = mergeWithSum(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
+                getMapByKeyword(accStats, WIN_TO_EMITTED));
+        putRawKV(ret, WIN_TO_EMITTED, emitted);
+
+        Map transferred = getMapByKeyword(newStats, TRANSFERRED);
+        transferred = mergeWithSum(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
+                getMapByKeyword(accStats, WIN_TO_TRANSFERRED));
+        putRawKV(ret, WIN_TO_TRANSFERRED, transferred);
+
+        putRawKV(ret, WIN_TO_EXEC_LAT_WGT_AVG, mergeWithSum(
+                getMapByKeyword(accStats, WIN_TO_EXEC_LAT_WGT_AVG), win2execLatWgtAvg));
+        putRawKV(ret, WIN_TO_PROC_LAT_WGT_AVG, mergeWithSum(
+                getMapByKeyword(accStats, WIN_TO_PROC_LAT_WGT_AVG), win2procLatWgtAvg));
+        putRawKV(ret, WIN_TO_EXECUTED, mergeWithSum(
+                getMapByKeyword(accStats, WIN_TO_EXECUTED), win2executed));
+        putRawKV(ret, WIN_TO_ACKED, mergeWithSum(
+                aggregateCountStreams(getMapByKeyword(newStats, ACKED)), getMapByKeyword(accStats, WIN_TO_ACKED)));
+        putRawKV(ret, WIN_TO_FAILED, mergeWithSum(
+                aggregateCountStreams(getMapByKeyword(newStats, FAILED)), getMapByKeyword(accStats, WIN_TO_FAILED)));
+
+        return ret;
+    }
+
+    /**
+     * A helper function that aggregates windowed stats from one spout executor.
+     */
+    public static Map aggSpoutExecWinStats(Map accStats, Map newStats, boolean includeSys) {
+        Map ret = new HashMap();
+
+        Map m = new HashMap();
+        for (Object win : getMapByKeyword(newStats, ACKED).keySet()) {
+            m.put(win, aggSpoutLatAndCount(
+                    (Map) (getMapByKeyword(newStats, COMP_LATENCIES)).get(win),
+                    (Map) (getMapByKeyword(newStats, ACKED)).get(win)));
+        }
+        m = swapMapOrder(m);
+
+        Map win2compLatWgtAvg = getMapByKeyword(m, COMP_LAT_TOTAL);
+        Map win2acked = getMapByKeyword(m, ACKED);
+
+        Map emitted = getMapByKeyword(newStats, EMITTED);
+        emitted = mergeWithSum(aggregateCountStreams(filterSysStreams(emitted, includeSys)),
+                getMapByKeyword(accStats, WIN_TO_EMITTED));
+        putRawKV(ret, WIN_TO_EMITTED, emitted);
+
+        Map transferred = getMapByKeyword(newStats, TRANSFERRED);
+        transferred = mergeWithSum(aggregateCountStreams(filterSysStreams(transferred, includeSys)),
+                getMapByKeyword(accStats, WIN_TO_TRANSFERRED));
+        putRawKV(ret, WIN_TO_TRANSFERRED, transferred);
+
+        putRawKV(ret, WIN_TO_COMP_LAT_WGT_AVG, mergeWithSum(
+                getMapByKeyword(accStats, WIN_TO_COMP_LAT_WGT_AVG), win2compLatWgtAvg));
+        putRawKV(ret, WIN_TO_ACKED, mergeWithSum(
+                getMapByKeyword(accStats, WIN_TO_ACKED), win2acked));
+        putRawKV(ret, WIN_TO_FAILED, mergeWithSum(
+                aggregateCountStreams(getMapByKeyword(newStats, FAILED)), getMapByKeyword(accStats, WIN_TO_FAILED)));
+
+        return ret;
+    }
+
+
+    /**
+     * aggregate counts
+     *
+     * @param countsSeq a seq of {win -> GlobalStreamId -> value}
+     */
+    public static Map aggregateCounts(List countsSeq) {
+        Map ret = new HashMap();
+        for (Object counts : countsSeq) {
+            for (Object o : ((Map) counts).entrySet()) {
+                Map.Entry e = (Map.Entry) o;
+                Object win = e.getKey();
+                Map stream2count = (Map) e.getValue();
+
+                if (!ret.containsKey(win)) {
+                    ret.put(win, stream2count);
+                } else {
+                    Map existing = (Map) ret.get(win);
+                    for (Object oo : stream2count.entrySet()) {
+                        Map.Entry ee = (Map.Entry) oo;
+                        Object stream = ee.getKey();
+                        if (!existing.containsKey(stream)) {
+                            existing.put(stream, ee.getValue());
+                        } else {
+                            existing.put(stream, (Long) ee.getValue() + (Long) existing.get(stream));
+                        }
+                    }
+                }
+            }
+        }
+        return ret;
+    }
+
+    public static Map aggregateCompStats(String window, boolean includeSys, List data, String compType) {
+        boolean isSpout = SPOUT.equals(compType);
+
+        Map initVal = new HashMap();
+        putRawKV(initVal, WIN_TO_ACKED, new HashMap());
+        putRawKV(initVal, WIN_TO_FAILED, new HashMap());
+        putRawKV(initVal, WIN_TO_EMITTED, new HashMap());
+        putRawKV(initVal, WIN_TO_TRANSFERRED, new HashMap());
+
+        Map stats = new HashMap();
+        putRawKV(stats, EXECUTOR_STATS, new ArrayList());
+        putRawKV(stats, SID_TO_OUT_STATS, new HashMap());
+        if (isSpout) {
+            putRawKV(initVal, TYPE, KW_SPOUT);
+            putRawKV(initVal, WIN_TO_COMP_LAT_WGT_AVG, new HashMap());
+        } else {
+            putRawKV(initVal, TYPE, KW_BOLT);
+            putRawKV(initVal, WIN_TO_EXECUTED, new HashMap());
+            putRawKV(stats, CID_SID_TO_IN_STATS, new HashMap());
+            putRawKV(initVal, WIN_TO_EXEC_LAT_WGT_AVG, new HashMap());
+            putRawKV(initVal, WIN_TO_PROC_LAT_WGT_AVG, new HashMap());
+        }
+        putRawKV(initVal, STATS, stats);
+
+        for (Object o : data) {
+            initVal = aggCompExecStats(window, includeSys, initVal, (Map) o, 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 aggCompExecStats(String window, boolean includeSys, Map accStats, Map newData, String compType) {
+        Map ret = new HashMap();
+        if (SPOUT.equals(compType)) {
+            ret.putAll(aggSpoutExecWinStats(accStats, getMapByKeyword(newData, STATS), includeSys));
+            putRawKV(ret, STATS, mergeAggCompStatsCompPageSpout(
+                    getMapByKeyword(accStats, STATS),
+                    aggPreMergeCompPageSpout(newData, window, includeSys)));
+        } else {
+            ret.putAll(aggBoltExecWinStats(accStats, getMapByKeyword(newData, STATS), includeSys));
+            putRawKV(ret, STATS, mergeAggCompStatsCompPageBolt(
+                    getMapByKeyword(accStats, STATS),
+                    aggPreMergeCompPageBolt(newData, window, includeSys)));
+        }
+        putRawKV(ret, TYPE, keyword(compType));
+
+        return ret;
+    }
+
+    public static Map postAggregateCompStats(Map task2component, Map exec2hostPort, Map accData) {
+        Map ret = new HashMap();
+
+        String compType = ((Keyword) getByKeyword(accData, TYPE)).getName();
+        Map stats = getMapByKeyword(accData, STATS);
+        Integer numTasks = getByKeywordOr0(stats, NUM_TASKS).intValue();
+        Integer numExecutors = getByKeywordOr0(stats, NUM_EXECUTORS).intValue();
+        Map outStats = getMapByKeyword(stats, SID_TO_OUT_STATS);
+
+        putRawKV(ret, TYPE, keyword(compType));
+        putRawKV(ret, NUM_TASKS, numTasks);
+        putRawKV(ret, NUM_EXECUTORS, numExecutors);
+        putRawKV(ret, EXECUTOR_STATS, getByKeyword(stats, EXECUTOR_STATS));
+        putRawKV(ret, WIN_TO_EMITTED, mapKeyStr(getMapByKeyword(accData, WIN_TO_EMITTED)));
+        putRawKV(ret, WIN_TO_TRANSFERRED, mapKeyStr(getMapByKeyword(accData, WIN_TO_TRANSFERRED)));
+        putRawKV(ret, WIN_TO_ACKED, mapKeyStr(getMapByKeyword(accData, WIN_TO_ACKED)));
+        putRawKV(ret, WIN_TO_FAILED, mapKeyStr(getMapByKeyword(accData, WIN_TO_FAILED)));
+
+        if (BOLT.equals(compType)) {
+            Map inStats = getMapByKeyword(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 = getByKeywordOr0(v, EXECUTED).longValue();
+                if (executed > 0) {
+                    double executeLatencyTotal = getByKeywordOr0(v, EXEC_LAT_TOTAL).doubleValue();
+                    double processLatencyTotal = getByKeywordOr0(v, PROC_LAT_TOTAL).doubleValue();
+                    putRawKV(v, EXEC_LATENCY, executeLatencyTotal / executed);
+                    putRawKV(v, PROC_LATENCY, processLatencyTotal / executed);
+                } else {
+                    putRawKV(v, EXEC_LATENCY, 0.0);
+                    putRawKV(v, PROC_LATENCY, 0.0);
+                }
+                removeByKeyword(v, EXEC_LAT_TOTAL);
+                removeByKeyword(v, PROC_LAT_TOTAL);
+                inStats2.put(k, v);
+            }
+            putRawKV(ret, CID_SID_TO_IN_STATS, inStats2);
+
+            putRawKV(ret, SID_TO_OUT_STATS, outStats);
+            putRawKV(ret, WIN_TO_EXECUTED, mapKeyStr(getMapByKeyword(accData, WIN_TO_EXECUTED)));
+            putRawKV(ret, WIN_TO_EXEC_LAT, computeWeightedAveragesPerWindow(
+                    accData, WIN_TO_EXEC_LAT_WGT_AVG, WIN_TO_EXECUTED));
+            putRawKV(ret, WIN_TO_PROC_LAT, computeWeightedAveragesPerWindow(
+                    accData, 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 = getByKeywordOr0(v, ACKED).longValue();
+                if (acked > 0) {
+                    double compLatencyTotal = getByKeywordOr0(v, COMP_LAT_TOTAL).doubleValue();
+                    putRawKV(v, COMP_LATENCY, compLatencyTotal / acked);
+                } else {
+                    putRawKV(v, COMP_LATENCY, 0.0);
+                }
+                removeByKeyword(v, COMP_LAT_TOTAL);
+                outStats2.put(k, v);
+            }
+            putRawKV(ret, SID_TO_OUT_STATS, outStats2);
+            putRawKV(ret, WIN_TO_COMP_LAT, computeWeightedAveragesPerWindow(
+                    accData, WIN_TO_COMP_LAT_WGT_AVG, WIN_TO_ACKED));
+        }
+
+        return ret;
+    }
+
+    /**
+     * called in nimbus.clj
+     */
+    public static ComponentPageInfo aggCompExecsStats(
+            Map exec2hostPort, Map task2component, Map beats, String window, boolean includeSys,
+            String topologyId, StormTopology topology, String componentId) {
+
+        List beatList = extractDataFromHb(exec2hostPort, task2component, beats, includeSys, topology, componentId);
+        Map compStats = aggregateCompStats(window, includeSys, beatList, componentType(topology, componentId).getName());
+        compStats = postAggregateCompStats(task2component, exec2hostPort, compStats);
+        return thriftifyCompPageData(topologyId, topology, componentId, compStats);
+    }
+
+
+    // =====================================================================================
+    // clojurify stats methods
+    // =====================================================================================
+
+    /**
+     * called in converter.clj
+     */
+    public static Map clojurifyStats(Map stats) {
+        Map ret = new HashMap();
+        for (Object o : stats.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            ExecutorInfo executorInfo = (ExecutorInfo) entry.getKey();
+            ExecutorStats executorStats = (ExecutorStats) entry.getValue();
+
+            ret.put(Lists.newArrayList(executorInfo.get_task_start(), executorInfo.get_task_end()),
+                    clojurifyExecutorStats(executorStats));
+        }
+        return ret;
+    }
+
+    public static Map clojurifyExecutorStats(ExecutorStats stats) {
+        Map ret = new HashMap();
+
+        putRawKV(ret, EMITTED, stats.get_emitted());
+        putRawKV(ret, TRANSFERRED, stats.get_transferred());
+        putRawKV(ret, "rate", stats.get_rate());
+
+        if (stats.get_specific().is_set_bolt()) {
+            mergeMaps(ret, clojurifySpecificStats(stats.get_specific().get_bolt()));
+            putRawKV(ret, TYPE, KW_BOLT);
+        } else {
+            mergeMaps(ret, clojurifySpecificStats(stats.get_specific().get_spout()));
+            putRawKV(ret, TYPE, KW_SPOUT);
+        }
+
+        return ret;
+    }
+
+    public static Map clojurifySpecificStats(SpoutStats stats) {
+        Map ret = new HashMap();
+        putRawKV(ret, ACKED, stats.get_acked());
+        putRawKV(ret, FAILED, stats.get_failed());
+        putRawKV(ret, COMP_LATENCIES, stats.get_complete_ms_avg());
+
+        return ret;
+    }
+
+    public static Map clojurifySpecificStats(BoltStats stats) {
+        Map 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);
+
+        putRawKV(ret, ACKED, acked);
+        putRawKV(ret, FAILED, failed);
+        putRawKV(ret, PROC_LATENCIES, processAvg);
+        putRawKV(ret, EXECUTED, executed);
+        putRawKV(ret, EXEC_LATENCIES, executeAvg);
+
+        return ret;
+    }
+
+    /**
+     * caller: nimbus.clj
+     */
+    public static List extractNodeInfosFromHbForComp(
+            Map exec2hostPort, Map task2component, boolean includeSys, String compId) {
+        List ret = new ArrayList();
+
+        Set<List> hostPorts = new HashSet<>();
+        for (Object o : exec2hostPort.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            List key = (List) entry.getKey();
+            List value = (List) entry.getValue();
+
+            Integer start = ((Number) key.get(0)).intValue();
+            String host = (String) value.get(0);
+            Integer port = (Integer) value.get(1);
+            String comp = (String) task2component.get(start);
+            if ((compId == null || compId.equals(comp)) && (includeSys || !Utils.isSystemId(comp))) {
+                hostPorts.add(Lists.newArrayList(host, port));
+            }
+        }
+
+        for (List hostPort : hostPorts) {
+            Map m = new HashMap();
+            putRawKV(m, HOST, hostPort.get(0));
+            putRawKV(m, PORT, hostPort.get(1));
+            ret.add(m);
+        }
+
+        return ret;
+    }
+
+    public static List extractDataFromHb(Map executor2hostPort, Map task2component, Map beats,
+                                         boolean includeSys, StormTopology topology) {
+        return extractDataFromHb(executor2hostPort, task2component, beats, includeSys, topology, null);
+    }
+
+    public static List extractDataFromHb(Map executor2hostPort, Map task2component, Map beats,
+                                         boolean includeSys, StormTopology topology, String compId) {
+        List ret = new ArrayList();
+        for (Object o : executor2hostPort.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            List key = (List) entry.getKey();
+            List value = (List) entry.getValue();
+
+            Integer start = ((Number) key.get(0)).intValue();
+            Integer end = ((Number) key.get(1)).intValue();
+
+            String host = (String) value.get(0);
+            Integer port = ((Number) value.get(1)).intValue();
+
+            Map beat = (Map) beats.get(key);
+            if (beat == null) {
+                continue;
+            }
+            String id = (String) task2component.get(start);
+
+            Map m = new HashMap();
+            if ((compId == null || compId.equals(id)) && (includeSys || !Utils.isSystemId(id))) {
+                putRawKV(m, "exec-id", entry.getKey());
+                putRawKV(m, "comp-id", id);
+                putRawKV(m, NUM_TASKS, end - start + 1);
+                putRawKV(m, HOST, host);
+                putRawKV(m, PORT, port);
+                putRawKV(m, UPTIME, beat.get(keyword(UPTIME)));
+                putRawKV(m, STATS, beat.get(keyword(STATS)));
+
+                Keyword type = componentType(topology, compId);
+                if (type != null) {
+                    putRawKV(m, TYPE, type);
+                } else {
+                    putRawKV(m, TYPE, getByKeyword(getMapByKeyword(beat, STATS), TYPE));
+                }
+                ret.add(m);
+            }
+        }
+        return ret;
+    }
+
+    private static Map computeWeightedAveragesPerWindow(Map accData, String wgtAvgKey, String divisorKey) {
+        Map ret = new HashMap();
+        for (Object o : getMapByKeyword(accData, wgtAvgKey).entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            Object window = e.getKey();
+            double wgtAvg = ((Number) e.getValue()).doubleValue();
+            long divisor = ((Number) getMapByKeyword(accData, divisorKey).get(window)).longValue();
+            if (divisor > 0) {
+                ret.put(window.toString(), wgtAvg / divisor);
+            }
+        }
+        return ret;
+    }
+
+
+    /**
+     * caller: core.clj
+     *
+     * @param executorSumms a list of ExecutorSummary
+     * @return max bolt capacity
+     */
+    public static double computeBoltCapacity(List executorSumms) {
+        double max = 0.0;
+        for (Object o : executorSumms) {
+            ExecutorSummary summary = (ExecutorSummary) o;
+            double capacity = computeExecutorCapacity(summary);
+            if (capacity > max) {
+                max = capacity;
+            }
+        }
+        return max;
+    }
+
+    public static double computeExecutorCapacity(ExecutorSummary summ) {
+        ExecutorStats stats = summ.get_stats();
+        if (stats == null) {
+            return 0.0;
+        } else {
+            Map m = aggregateBoltStats(Lists.newArrayList(stats), true);
+            m = swapMapOrder(aggregateBoltStreams(m));
+            Map data = getMapByKeyword(m, TEN_MIN_IN_SECONDS_STR);
+
+            int uptime = summ.get_uptime_secs();
+            int win = Math.min(uptime, TEN_MIN_IN_SECONDS);
+            long executed = getByKeywordOr0(data, EXECUTED).longValue();
+            double latency = getByKeywordOr0(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 getFilledStats(List summs) {
+        for (Iterator itr = summs.iterator(); itr.hasNext(); ) {
+            ExecutorSummary summ = (ExecutorSummary) itr.next();
+            if (summ.get_stats() == null) {
+                itr.remove();
+            }
+        }
+        return summs;
+    }
+
+    private static Map mapKeyStr(Map m) {
+        Map ret = new HashMap();
+        for (Object k : m.keySet()) {
+            ret.put(k.toString(), m.get(k));
+        }
+        return ret;
+    }
+
+    private static long sumStreamsLong(Map m, String key) {
+        long sum = 0;
+        if (m == null) {
+            return sum;
+        }
+        for (Object v : m.values()) {
+            Map sub = (Map) v;
+            for (Object o : sub.entrySet()) {
+                Map.Entry e = (Map.Entry) o;
+                if (((Keyword) e.getKey()).getName().equals(key)) {
+                    sum += ((Number) e.getValue()).longValue();
+                }
+            }
+        }
+        return sum;
+    }
+
+    private static double sumStreamsDouble(Map m, String key) {
+        double sum = 0;
+        if (m == null) {
+            return sum;
+        }
+        for (Object v : m.values()) {
+            Map sub = (Map) v;
+            for (Object o : sub.entrySet()) {
+                Map.Entry e = (Map.Entry) o;
+                if (((Keyword) e.getKey()).getName().equals(key)) {
+                    sum += ((Number) e.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 stats      { win -> stream id -> value }
+     * @param includeSys whether to filter system streams
+     * @return filtered stats
+     */
+    private static Map filterSysStreams(Map stats, boolean includeSys) {
+        if (!includeSys) {
+            for (Object win : stats.keySet()) {
+                Map stream2stat = (Map) stats.get(win);
+                for (Iterator itr = stream2stat.keySet().iterator(); itr.hasNext(); ) {
+                    Object key = itr.next();
+                    if (key instanceof String && Utils.isSystemId((String) key)) {
+                        itr.remove();
+                    }
+                }
+            }
+        }
+        return stats;
+    }
+
+    /**
+     * equals to clojure's: (merge-with (partial merge-with sum-or-0) acc-out spout-out)
+     */
+    private static Map fullMergeWithSum(Map m1, Map m2) {
+        Set<Object> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        Map ret = new HashMap();
+        for (Object k : allKeys) {
+            Map mm1 = null, mm2 = null;
+            if (m1 != null) {
+                mm1 = (Map) m1.get(k);
+            }
+            if (m2 != null) {
+                mm2 = (Map) m2.get(k);
+            }
+            ret.put(k, mergeWithSum(mm1, mm2));
+        }
+
+        return ret;
+    }
+
+    private static Map mergeWithSum(Map m1, Map m2) {
+        Map ret = new HashMap();
+
+        Set<Object> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        for (Object k : allKeys) {
+            Number n1 = getOr0(m1, k);
+            Number n2 = getOr0(m2, k);
+            ret.put(k, add(n1, n2));
+        }
+        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 Map mergeWithAddPair(Map m1, Map m2) {
+        Map ret = new HashMap();
+
+        Set<Object> allKeys = new HashSet<>();
+        if (m1 != null) {
+            allKeys.addAll(m1.keySet());
+        }
+        if (m2 != null) {
+            allKeys.addAll(m2.keySet());
+        }
+
+        for (Object k : allKeys) {
+            Map mm1 = (m1 != null) ? (Map) m1.get(k) : null;
+            Map mm2 = (m2 != null) ? (Map) 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 tmp = new HashMap();
+                for (Object kk : mm1.keySet()) {
+                    List seq1 = (List) mm1.get(kk);
+                    List seq2 = (List) mm2.get(kk);
+                    List sums = new ArrayList();
+                    for (int i = 0; i < seq1.size(); i++) {
+                        sums.add(add((Number) seq1.get(i), (Number) seq2.get(i)));
+                    }
+                    tmp.put(kk, sums);
+                }
+                ret.put(k, tmp);
+            }
+        }
+        return ret;
+    }
+
+    // =====================================================================================
+    // thriftify stats methods
+    // =====================================================================================
+
+    private static TopologyPageInfo thriftifyTopoPageData(String topologyId, Map data) {
+        TopologyPageInfo ret = new TopologyPageInfo(topologyId);
+        Integer numTasks = getByKeywordOr0(data, NUM_TASKS).intValue();
+        Integer numWorkers = getByKeywordOr0(data, NUM_WORKERS).intValue();
+        Integer numExecutors = getByKeywordOr0(data, NUM_EXECUTORS).intValue();
+        Map spout2stats = getMapByKeyword(data, SPOUT_TO_STATS);
+        Map bolt2stats = getMapByKeyword(data, BOLT_TO_STATS);
+        Map win2emitted = getMapByKeyword(data, WIN_TO_EMITTED);
+        Map win2transferred = getMapByKeyword(data, WIN_TO_TRANSFERRED);
+        Map win2compLatency = getMapByKeyword(data, WIN_TO_COMP_LAT);
+        Map win2acked = getMapByKeyword(data, WIN_TO_ACKED);
+        Map win2failed = getMapByKeyword(data, WIN_TO_FAILED);
+
+        Map<String, ComponentAggregateStats> spoutAggStats = new HashMap<>();
+        for (Object o : spout2stats.entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            String id = (String) e.getKey();
+            Map v = (Map) e.getValue();
+            putRawKV(v, TYPE, KW_SPOUT);
+
+            spoutAggStats.put(id, thriftifySpoutAggStats(v));
+        }
+
+        Map<String, ComponentAggregateStats> boltAggStats = new HashMap<>();
+        for (Object o : bolt2stats.entrySet()) {
+            Map.Entry e = (Map.Entry) o;
+            String id = (String) e.getKey();
+            Map v = (Map) e.getValue();
+            putRawKV(v, TYPE, KW_BOLT);
+
+            boltAggStats.put(id, thriftifyBoltAggStats(v));
+        }
+
+        TopologyStats topologyStats = new TopologyStats();
+        topologyStats.set_window_to_acked(win2acked);
+        topologyStats.set_window_to_emitted(win2emitted);
+        topologyStats.set_window_to_failed(win2failed);
+        topologyStats.set_window_to_transferred(win2transferred);
+        topologyStats.set_window_to_complete_latencies_ms(win2compLatency);
+
+        ret.set_num_tasks(numTasks);
+        ret.set_num_workers(numWorkers);
+        ret.set_num_executors(numExecutors);
+        ret.set_id_to_spout_agg_stats(spoutAggStats);
+        ret.set_id_to_bolt_agg_stats(boltAggStats);
+        ret.set_topology_stats(topologyStats);
+
+        return ret;
+    }
+
+    private static ComponentAggregateStats thriftifySpoutAggStats(Map m) {
+        ComponentAggregateStats stats = new ComponentAggregateStats();
+        stats.set_type(ComponentType.SPOUT);
+        stats.set_last_error((ErrorInfo) getByKeyword(m, LAST_ERROR));
+        thriftifyCommonAggStats(stats, m);
+
+        SpoutAggregateStats spoutAggStats = new SpoutAggregateStats();
+        spoutAggStats.set_complete_latency_ms(getByKeywordOr0(m, COMP_LATENCY).doubleValue());
+        SpecificAggregateStats specificStats = SpecificAggregateStats.spout(spoutAggStats);
+
+        stats.set_specific_stats(specificStats);
+        return stats;
+    }
+
+    private static ComponentAggregateStats thriftifyBoltAggStats(Map m) {
+        ComponentAggregateStats stats = new ComponentAggregateStats();
+        stats.set_type(ComponentType.BOLT);
+        stats.set_last_error((ErrorInfo) getByKeyword(m, LAST_ERROR));
+        thriftifyCommonAggStats(stats, m);
+
+        BoltAggregateStats boltAggStats = new BoltAggregateStats();
+        boltAggStats.set_execute_latency_ms(getByKeywordOr0(m, EXEC_LATENCY).doubleValue());
+        boltAggStats.set_process_latency_ms(getByKeywordOr0(m, PROC_LATENCY).doubleValue());
+        boltAggStats.set_executed(getByKeywordOr0(m, EXECUTED).longValue());
+        boltAggStats.set_capacity(getByKeywordOr0(m, CAPACITY).doubleValue());
+        SpecificAggregateStats specificStats = SpecificAggregateStats.bolt(boltAggStats);
+
+        stats.set_specific_stats(specificStats);
+        return stats;
+    }
+
+    private static ExecutorAggregateStats thriftifyExecAggStats(String compId, Keyword compType, Map m) {
+        ExecutorAggregateStats stats = new ExecutorAggregateStats();
+
+        ExecutorSummary executorSummary = new ExecutorSummary();
+        List executor = (List) getByKeyword(m, EXECUTOR_ID);
+        executorSummary.set_executor_info(new ExecutorInfo(((Number) executor.get(0)).intValue(),
+                ((Number) executor.get(1)).intValue()));
+        executorSummary.set_component_id(compId);
+        executorSummary.set_host((String) getByKeyword(m, HOST));
+        executorSummary.set_port(getByKeywordOr0(m, PORT).intValue());
+        int uptime = getByKeywordOr0(m, UPTIME).intValue();
+        executorSummary.set_uptime_secs(uptime);
+        stats.set_exec_summary(executorSummary);
+
+        if (compType.getName().equals(SPOUT)) {
+            stats.set_stats(thriftifySpoutAggStats(m));
+        } else {
+            stats.set_stats(thriftifyBoltAggStats(m));
+        }
+
+        return stats;
+    }
+
+    private static Map thriftifyBoltOutputStats(Map id2outStats) {
+        Map ret = new HashMap();
+        for (Object k : id2outStats.keySet()) {
+            ret.put(k, thriftifyBoltAggStats((Map) id2outStats.get(k)));
+        }
+        return ret;
+    }
+
+    private static Map thriftifySpoutOutputStats(Map id2outStats) {
+        Map ret = new HashMap();
+        for (Object k : id2outStats.keySet()) {
+            ret.put(k, thriftifySpoutAggStats((Map) id2outStats.get(k)));
+        }
+        return ret;
+    }
+
+    private static Map thriftifyBoltInputStats(Map cidSid2inputStats) {
+        Map ret = new HashMap();
+        for (Object e : cidSid2inputStats.entrySet()) {
+            Map.Entry entry = (Map.Entry) e;
+            ret.put(toGlobalStreamId((List) entry.getKey()),
+                    thriftifyBoltAggStats((Map) entry.getValue()));
+        }
+        return ret;
+    }
+
+    private static ComponentAggregateStats thriftifyCommonAggStats(ComponentAggregateStats stats, Map m) {
+        CommonAggregateStats commonStats = new CommonAggregateStats();
+        commonStats.set_num_tasks(getByKeywordOr0(m, NUM_TASKS).intValue());
+        commonStats.set_num_executors(getByKeywordOr0(m, NUM_EXECUTORS).intValue());
+        commonStats.set_emitted(getByKeywordOr0(m, EMITTED).longValue());
+        commonStats.set_transferred(getByKeywordOr0(m, TRANSFERRED).longValue());
+        commonStats.set_acked(getByKeywordOr0(m, ACKED).longValue());
+        commonStats.set_failed(getByKeywordOr0(m, FAILED).longValue());
+
+        stats.set_common_stats(commonStats);
+        return stats;
+    }
+
+    private static ComponentPageInfo thriftifyCompPageData(
+            String topologyId, StormTopology topology, String compId, Map data) {
+        ComponentPageInfo ret = new ComponentPageInfo();
+        ret.set_component_id(compId);
+
+        Map win2stats = new HashMap();
+        putRawKV(win2stats, EMITTED, getMapByKeyword(data, WIN_TO_EMITTED));
+        putRawKV(win2stats, TRANSFERRED, getMapByKeyword(data, WIN_TO_TRANSFERRED));
+        putRawKV(win2stats, ACKED, getMapByKeyword(data, WIN_TO_ACKED));
+        putRawKV(win2stats, FAILED, getMapByKeyword(data, WIN_TO_FAILED));
+
+        Keyword type = (Keyword) getByKeyword(data, TYPE);
+        String compType = type.getName();
+        if (compType.equals(SPOUT)) {
+            ret.set_component_type(ComponentType.SPOUT);
+            putRawKV(win2stats, COMP_LATENCY, getMapByKeyword(data, WIN_TO_COMP_LAT));
+        } else {
+            ret.set_component_type(ComponentType.BOLT);
+            putRawKV(win2stats, EXEC_LATENCY, getMapByKeyword(data, WIN_TO_EXEC_LAT));
+            putRawKV(win2stats, PROC_LATENCY, getMapByKeyword(data, WIN_TO_PROC_LAT));
+            putRawKV(win2stats, EXECUTED, getMapByKeyword(data, WIN_TO_EXECUTED));
+        }
+        win2stats = swapMapOrder(win2stats);
+
+        List<ExecutorAggregateStats> execStats = new ArrayList<>();
+        List executorStats = (List) getByKeyword(data, EXECUTOR_STATS);
+        if (executorStats != null) {
+            for (Object o : executorStats) {
+                execStats.add(thriftifyExecAggStats(compId, type, (Map) o));
+            }
+        }
+
+        Map gsid2inputStats, sid2outputStats;
+        if (compType.equals(SPOUT)) {
+            Map tmp = new HashMap();
+            for (Object k : win2stats.keySet()) {
+                tmp.put(k, thriftifySpoutAggStats((Map) win2stats.get(k)));
+            }
+            win2stats = tmp;
+            gsid2inputStats = null;
+            sid2outputStats = thriftifySpoutOutputStats(getMapByKeyword(data, SID_TO_OUT_STATS));
+        } else {
+            Map tmp = new HashMap();
+            for (Object k : win2stats.keySet()) {
+                tmp.put(k, thriftifyBoltAggStats((Map) win2stats.get(k)));
+            }
+            win2stats = tmp;
+            gsid2inputStats = thriftifyBoltInputStats(getMapByKeyword(data, CID_SID_TO_IN_STATS));
+            sid2outputStats = thriftifyBoltOutputStats(getMapByKeyword(data, SID_TO_OUT_STATS));
+        }
+        ret.set_num_executors(getByKeywordOr0(data, NUM_EXECUTORS).intValue());
+        ret.set_num_tasks(getByKeywordOr0(data, NUM_TASKS).intValue());
+        ret.set_topology_id(topologyId);
+        ret.set_topology_name(null);
+        ret.set_window_to_stats(win2stats);
+        ret.set_sid_to_output_stats(sid2outputStats);
+        ret.set_exec_stats(execStats);
+        ret.set_gsid_to_input_stats(gsid2inputStats);
+
+        return ret;
+    }
+
+    /**
+     * called in converter.clj
+     */
+    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;
+    }
+
+    /**
+     * called in nimbus.clj
+     */
+    public static ExecutorStats thriftifyExecutorStats(Map stats) {
+        ExecutorStats ret = new ExecutorStats();
+        ExecutorSpecificStats specificStats = thriftifySpecificStats(stats);
+        ret.set_specific(specificStats);
+
+        ret.set_emitted(windowSetConverter(getMapByKeyword(stats, EMITTED), TO_STRING, TO_STRING));
+        ret.set_transferred(windowSetConverter(getMapByKeyword(stats, TRANSFERRED), TO_STRING, TO_STRING));
+        ret.set_rate(((Number) getByKeyword(stats, "rate")).doubleValue());
+
+        return ret;
+    }
+
+    private static ExecutorSpecificStats thriftifySpecificStats(Map stats) {
+        ExecutorSpecificStats specificStats = new ExecutorSpecificStats();
+
+        String compType = ((Keyword) getByKeyword(stats, TYPE)).getName();
+        if (BOLT.equals(compType)) {
+            BoltStats boltStats = new BoltStats();
+            boltStats.set_acked(windowSetConverter(getMapByKeyword(stats, ACKED), TO_GSID, TO_STRING));
+            boltStats.set_executed(windowSetConverter(getMapByKeyword(stats, EXECUTED), TO_GSID, TO_STRING));
+            boltStats.set_execute_ms_avg(windowSetConverter(getMapByKeyword(stats, EXEC_LATENCIES), TO_GSID, TO_STRING));
+            boltStats.set_failed(windowSetConverter(getMapByKeyword(stats, FAILED), TO_GSID, TO_STRING));
+            boltStats.set_process_ms_avg(windowSetConverter(getMapByKeyword(stats, PROC_LATENCIES), TO_GSID, TO_STRING));
+            specificStats.set_bolt(boltStats);
+        } else {
+            SpoutStats spoutStats = new SpoutStats();
+            spoutStats.set_acked(windowSetConverter(getMapByKeyword(stats, ACKED), TO_STRING, TO_STRING));
+            spoutStats.set_failed(windowSetConverter(getMapByKeyword(stats, FAILED), TO_STRING, TO_STRING));
+            spoutStats.set_complete_ms_avg(windowSetConverter(getMapByKeyword(stats, COMP_LATENCIES), TO_STRING, TO_STRING));
+            specificStats.set_spout(spoutStats);
+        }
+        return specificStats;
+    }
+
+
+    // =====================================================================================
+    // helper methods
+    // =====================================================================================
+
+    private static GlobalStreamId toGlobalStreamId(List list) {
+        return new GlobalStreamId((String) list.get(0), (String) list.get(1));
+    }
+
+    /**
+     * 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());
+    }
+
+    /**
+     * the value of m is as follows:
+     * <pre>
+     * #org.apache.storm.stats.CommonStats {
+     *  :executed {
+     *      ":all-time" {["split" "default"] 18727460},
+     *      "600" {["split" "default"] 11554},
+     *      "10800" {["split" "default"] 207269},
+     *      "86400" {["split" "default"] 1659614}},
+     *  :execute-latencies {
+     *      ":all-time" {["split" "default"] 0.5874528633354443},
+     *      "600" {["split" "default"] 0.6140350877192983},
+     *      "10800" {["split" "default"] 0.5864434687156971},
+     *      "86400" {["split" "default"] 0.5815376460556336}}
+     * }
+     * </pre>
+     */
+    private static double computeAggCapacity(Map m, Integer uptime) {
+        if (uptime != null) {
+            Map execAvg = (Map) ((Map) getByKeyword(m, EXEC_LATENCIES)).get(TEN_MIN_IN_SECONDS_STR);
+            Map exec = (Map) ((Map) getByKeyword(m, EXECUTED)).get(TEN_MIN_IN_SECONDS_STR);
+
+            Set<Object> allKeys = new HashSet<>();
+            if (execAvg != null) {
+                allKeys.addAll(execAvg.keySet());
+            }
+            if (exec != null) {
+                allKeys.addAll(exec.keySet());
+            }
+
+            double totalAvg = 0;
+            for (Object k : allKeys) {
+                double avg = getOr0(execAvg, k).doubleValue();
+                long cnt = getOr0(exec, k).longValue();
+                totalAvg += avg * cnt;
+            }
+            return totalAvg / (Math.min(uptime, TEN_MIN_IN_SECONDS) * 1000);
+        }
+        return 0.0;
+    }
+
+    private static Number getOr0(Map m, Object k) {
+        if (m == null) {
+            return 0;
+        }
+
+        Number n = (Number) m.get(k);
+        if (n == null) {
+            return 0;
+        }
+        return n;
+    }
+
+    private static Number getByKeywordOr0(Map m, String k) {
+        if (m == null) {
+            return 0;
+        }
+
+        Number n = (Number) m.get(keyword(k));
+        if (n == null) {
+            return 0;
+        }
+        return n;
+    }
+
+    private static Double weightAvgAndSum(Map id2Avg, Map id2num) {
+        double ret = 0;
+        if (id2Avg == null || id2num == null) {
+            return ret;
+        }
+
+        for (Object o : id2Avg.entrySet()) {
+            Map.Entry entry = (Map.Entry) o;
+            Object k = entry.getKey();
+            double v = ((Number) entry.getValue()).doubleValue();
+            long n = ((Number) id2num.get(k)).longValue();
+            ret += productOr0(v, n);
+        }
+        return ret;
+    }
+
+    private static double weightAvg(Map id2Avg, Map id2num, Object key) {
+        if (id2Avg == null || id2num == null) {
+            return 0.0;
+        }
+        return productOr0(id2Avg.get(key), id2num.get(key));
+    }
+
+    public static Keyword componentType(StormTopology topology, String compId) {
+        if (compId == null) {
+            return null;
+        }
+
+        Map<String, Bolt> bolts = topology.get_bolts();
+        if (Utils.isSystemId(compId) || bolts.containsKey(compId)) {
+            return KW_BOLT;
+        }
+        return KW_SPOUT;
+    }
+
+    public static void putRawKV(Map map, String k, Object v) {
+        map.put(keyword(k), v);
+    }
+
+    private static void removeByKeyword(Map map, String k) {
+        map.remove(keyword(k));
+    }
+
+    public static Object getByKeyword(Map map, String key) {
+        return map.get(keyword(key));
+    }
+
+    public static Map getMapByKeyword(Map map, String key) {
+        if (map == null) {
+            return null;
+        }
+        return (Map) map.get(keyword(key));
+    }
+
+    private static Number add(Number n1, Number n2) {
+        if (n1 instanceof Long || n1 instanceof Integer) {
+            return n1.longValue() + n2.longValue();
+        }
+        return n1.doubleValue() + n2.doubleValue();
+    }
+
+    private static long sumValues(Map m) {
+        long ret = 0L;
+        if (m == null) {
+            return ret;
+        }
+
+        for (Object o : m.values()) {
+            ret += ((Number) o).longValue();
+        }
+        return ret;
+    }
+
+    private static Number sumOr0(Object a, Object b) {
+        if (isValidNumber(a) && isValidNumber(b)) {
+            if (a instanceof Long || a instanceof Integer) {
+                return ((Number) a).longValue() + ((Number) b).longValue();
+            } else {
+                return ((Number) a).doubleValue() + ((Number) b).doubleValue();
+            }
+        }
+        return 0;
+    }
+
+    private static double productOr0(Object a, Object b) {
+        if (isValidNumber(a) && isValidNumber(b)) {
+            return ((Number) a).doubleValue() * ((Number) b).doubleValue();
+        }
+        return 0;
+    }
+
+    private static double maxOr0(Object a, Object b) {
+        if (isValidNumber(a) && isValidNumber(b)) {
+            return Math.max(((Number) a).doubleValue(), ((Number) b).doubleValue());
+        }
+        return 0;
+    }
+
+    /**
+     * For a nested map, rearrange data such that the top-level keys become the
+     * nested map's keys and vice versa.
+     * Example:
+     * {:a {:X :banana, :Y :pear}, :b {:X :apple, :Y :orange}}
+     * -> {:Y {:a :pear, :b :orange}, :X {:a :banana, :b :apple}}"
+     */
+    private static Map swapMapOrder(Map m) {
+        if (m.size() == 0) {
+            return m;
+        }
+
+        Map ret = new HashMap();
+        for (Object k1 : m.keySet()) {
+            Map v = (Map) m.get(k1);
+            if (v != null) {
+                for (Object k2 : v.keySet()) {
+                    Map subRet = (Map) ret.get(k2);
+                    if (subRet == null) {
+                        subRet = new HashMap();
+                        ret.put(k2, subRet);
+                    }
+                    subRet.put(k1, v.get(k2));
+                }
+            }
+        }
+        return ret;
+    }
+
+    /**
+     * @param avgs   a PersistentHashMap of values: { win -> GlobalStreamId -> value }
+     * @param counts a PersistentHashMap of values: { win -> GlobalStreamId -> value }
+     * @return a PersistentHashMap of values: {win -> GlobalStreamId -> [cnt*avg, cnt]}
+     */
+    private static Map expandAverages(Map avgs, Map counts) {
+        Map ret = new HashMap();
+
+        for (Object win : counts.keySet()) {
+            Map inner = new HashMap();
+
+            Map stream2cnt = (Map) counts.get(win);
+            for (Object stream : stream2cnt.keySet()) {
+                Long cnt = (Long) stream2cnt.get(stream);
+                Double avg = (Double) ((Map) avgs.get(win)).get(stream);
+                if (avg == null) {
+                    avg = 0.0;
+                }
+                inner.put(stream, Lists.newArrayList(cnt * avg, cnt));
+            }
+            ret.put(win, inner);
+        }
+
+        return ret;
+    }
+
+    /**
+     * first zip the two seqs, then do expand-average, then merge with sum
+     *
+     * @param avgSeq   list of avgs like: [{win -> Glob

<TRUNCATED>

[28/30] storm git commit: upmerge from master

Posted by bo...@apache.org.
upmerge from master


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

Branch: refs/heads/master
Commit: fb19f0d95cbecfd86cc534677b53302a58046bf6
Parents: 7b35428 6390d18
Author: 卫乐 <we...@taobao.com>
Authored: Fri Mar 11 10:15:53 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Fri Mar 11 10:15:53 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   3 +
 conf/log4j2.xml                                 |   2 +-
 .../org/apache/storm/starter/ManualDRPC.java    |  53 +-
 .../src/clj/org/apache/storm/LocalDRPC.clj      |  56 --
 .../src/clj/org/apache/storm/converter.clj      |  15 +
 .../src/clj/org/apache/storm/daemon/common.clj  | 350 +-----------
 .../src/clj/org/apache/storm/daemon/drpc.clj    | 215 +-------
 .../clj/org/apache/storm/daemon/executor.clj    |  24 +-
 .../clj/org/apache/storm/daemon/logviewer.clj   |   2 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |  63 +--
 .../clj/org/apache/storm/daemon/supervisor.clj  |   7 +-
 .../src/clj/org/apache/storm/daemon/task.clj    |   5 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  24 +-
 storm-core/src/clj/org/apache/storm/testing.clj | 100 ++--
 .../clj/org/apache/storm/trident/testing.clj    |   2 -
 storm-core/src/clj/org/apache/storm/ui/core.clj |  15 +-
 .../src/jvm/org/apache/storm/LocalDRPC.java     |  72 +++
 .../org/apache/storm/daemon/DaemonCommon.java   |  22 +
 .../jvm/org/apache/storm/daemon/DrpcServer.java | 357 ++++++++++++
 .../org/apache/storm/daemon/StormCommon.java    | 537 +++++++++++++++++++
 .../storm/utils/StormCommonInstaller.java       |  43 ++
 .../src/jvm/org/apache/storm/utils/Utils.java   |  16 +
 .../org/apache/storm/integration_test.clj       |   6 +-
 .../test/clj/org/apache/storm/drpc_test.clj     |  27 +-
 .../test/clj/org/apache/storm/nimbus_test.clj   | 121 +++--
 .../apache/storm/security/auth/auth_test.clj    |   3 +-
 .../storm/security/auth/drpc_auth_test.clj      |   5 +-
 .../clj/org/apache/storm/supervisor_test.clj    |  11 +-
 28 files changed, 1322 insertions(+), 834 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/task.clj
index 707cdda,f6c536d..80255ad
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@@ -27,7 -27,10 +27,8 @@@
    (:import [org.apache.storm.generated ShellComponent JavaObject])
    (:import [org.apache.storm.spout ShellSpout])
    (:import [java.util Collection List ArrayList])
-   (:import [org.apache.storm Thrift])
+   (:import [org.apache.storm Thrift]
+            (org.apache.storm.daemon StormCommon))
 -  (:require [org.apache.storm
 -             [stats :as stats]])
    (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics]))
  
  (defn mk-topology-context-builder [worker executor-data topology]

http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/ui/core.clj
index a538876,c1ea340..885d754
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@@ -21,13 -21,10 +21,11 @@@
          ring.middleware.multipart-params)
    (:use [ring.middleware.json :only [wrap-json-params]])
    (:use [hiccup core page-helpers])
 -  (:use [org.apache.storm config util log stats converter])
 +  (:use [org.apache.storm config util log converter])
    (:use [org.apache.storm.ui helpers])
-   (:use [org.apache.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
-                                                 ACKER-FAIL-STREAM-ID mk-authorization-handler]]])
    (:import [org.apache.storm.utils Time]
             [org.apache.storm.generated NimbusSummary]
 +           [org.apache.storm.stats StatsUtil]
             [org.apache.storm.ui UIHelpers IConfigurator FilterConfiguration]
             [org.apache.storm.metric StormMetricsRegistry])
    (:use [clojure.string :only [blank? lower-case trim split]])

http://git-wip-us.apache.org/repos/asf/storm/blob/fb19f0d9/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 1f708cb,fa475e7..e15c7a6
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -22,11 -22,10 +22,11 @@@
              TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
             [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
             [org.apache.storm.generated GlobalStreamId]
 -           [org.apache.storm Thrift MockAutoCred])
 +           [org.apache.storm Thrift MockAutoCred]
 +           [org.apache.storm.stats BoltExecutorStats StatsUtil])
    (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
    (:import [org.apache.storm.scheduler INimbus])
-   (:import [org.mockito Mockito])
+   (:import [org.mockito Mockito Matchers])
    (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
    (:import [org.apache.storm.testing.staticmocking MockedCluster])


[18/30] storm git commit: upmerge from master

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/39ea23cd/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index fe804d7,fb000da..53a3678
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -22,8 -22,7 +22,8 @@@
              TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
             [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
             [org.apache.storm.generated GlobalStreamId]
-            [org.apache.storm Thrift]
 -           [org.apache.storm Thrift MockAutoCred])
++           [org.apache.storm ThriftMockAutoCred]
 +           [org.apache.storm.stats BoltExecutorStats])
    (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
    (:import [org.apache.storm.scheduler INimbus])
    (:import [org.mockito Mockito])


[21/30] storm git commit: upmerge from master

Posted by bo...@apache.org.
upmerge from master


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

Branch: refs/heads/master
Commit: 9002528531693d793299458fd6f01b3e83ca1528
Parents: 39ea23c
Author: 卫乐 <we...@taobao.com>
Authored: Sat Mar 5 20:03:01 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Sat Mar 5 20:03:01 2016 +0800

----------------------------------------------------------------------
 storm-core/test/clj/org/apache/storm/nimbus_test.clj | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/90025285/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 53a3678..904d0db 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -22,7 +22,7 @@
             TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
            [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
            [org.apache.storm.generated GlobalStreamId]
-           [org.apache.storm ThriftMockAutoCred]
+           [org.apache.storm Thrift MockAutoCred]
            [org.apache.storm.stats BoltExecutorStats])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
   (:import [org.apache.storm.scheduler INimbus])


[09/30] storm git commit: fixed a potential NPE

Posted by bo...@apache.org.
fixed a potential NPE


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

Branch: refs/heads/master
Commit: 3fc80c4b0bfc83d2534fab160c72894af044dbc3
Parents: e5564c0
Author: 卫乐 <we...@taobao.com>
Authored: Thu Feb 25 15:25:58 2016 +0800
Committer: 卫乐 <we...@taobao.com>
Committed: Thu Feb 25 15:25:58 2016 +0800

----------------------------------------------------------------------
 storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3fc80c4b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
index c06d7db..75ec292 100644
--- a/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
+++ b/storm-core/src/jvm/org/apache/storm/stats/StatsUtil.java
@@ -1156,6 +1156,9 @@ public class StatsUtil {
     public static List extractDataFromHb(Map executor2hostPort, Map task2component, Map beats,
                                          boolean includeSys, StormTopology topology, String compId) {
         List ret = new ArrayList();
+        if (executor2hostPort == null) {
+            return ret;
+        }
         for (Object o : executor2hostPort.entrySet()) {
             Map.Entry entry = (Map.Entry) o;
             List key = (List) entry.getKey();