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();