diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md index 215511398c5..62167fbf319 100644 --- a/STORM-UI-REST-API.md +++ b/STORM-UI-REST-API.md @@ -442,7 +442,7 @@ Response fields: |windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"| |executors| Integer |Number of executor tasks in the component| |componentErrors| Array of Errors | List of component errors| -|componentErrors.time| Long | Timestamp when the exception occurred | +|componentErrors.errorTime| Long | Timestamp when the exception occurred (Prior to 0.11.0, this field was named 'time'.)| |componentErrors.errorHost| String | host name for the error| |componentErrors.errorPort| String | port for the error| |componentErrors.error| String |Shows the error happened in a component| @@ -483,7 +483,7 @@ Sample response: "componentType": "spout", "windowHint": "10m 0s", "executors": 5, - "componentErrors":[{"time": 1406006074000, + "componentErrors":[{"errorTime": 1406006074000, "errorHost": "10.11.1.70", "errorPort": 6701, "errorWorkerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log", diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj index 9da0e4f498f..a30850d7639 100644 --- a/storm-core/src/clj/backtype/storm/daemon/common.clj +++ b/storm-core/src/clj/backtype/storm/daemon/common.clj @@ -310,7 +310,7 @@ (doseq [[_ component] (all-components ret) :let [common (.get_common component)]] (.put_to_streams common EVENTLOGGER-STREAM-ID (thrift/output-fields (eventlogger-bolt-fields)))) - (.put_to_bolts ret "__eventlogger" eventlogger-bolt) + (.put_to_bolts ret EVENTLOGGER-COMPONENT-ID eventlogger-bolt) )) (defn add-metric-components! [storm-conf ^StormTopology topology] diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj index 19870f1e8e2..70bd197813a 100644 --- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj +++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj @@ -36,10 +36,13 @@ ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice - LogConfig LogLevel LogLevelAction]) + ComponentPageInfo TopologyPageInfo LogConfig LogLevel LogLevelAction]) (:import [backtype.storm.daemon Shutdownable]) (:use [backtype.storm util config log timer zookeeper]) - (:require [backtype.storm [cluster :as cluster] [stats :as stats] [converter :as converter]]) + (:require [backtype.storm [cluster :as cluster] + [converter :as converter] + [stats :as stats] + [tuple :as tuple]]) (:require [clojure.set :as set]) (:import [backtype.storm.daemon.common StormBase Assignment]) (:use [backtype.storm.daemon common]) @@ -978,13 +981,6 @@ (.set_host (:host %)) (.set_port (:port %)))))) -(defn- get-last-error - [storm-cluster-state storm-id component-id] - (if-let [e (.last-error storm-cluster-state storm-id component-id)] - (doto (ErrorInfo. (:error e) (:time-secs e)) - (.set_host (:host e)) - (.set_port (:port e))))) - (defn- thriftify-executor-id [[first-task-id last-task-id]] (ExecutorInfo. (int first-task-id) (int last-task-id))) @@ -1082,7 +1078,42 @@ (.prepare inimbus conf (master-inimbus-dir conf)) (log-message "Starting Nimbus with conf " conf) (let [nimbus (nimbus-data conf inimbus) - principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)] + principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf) + get-common-topo-info + (fn [^String storm-id operation] + (let [storm-cluster-state (:storm-cluster-state nimbus) + topology-conf (try-read-storm-conf conf storm-id) + storm-name (topology-conf TOPOLOGY-NAME) + _ (check-authorization! nimbus + storm-name + topology-conf + operation) + topology (try-read-storm-topology conf storm-id) + task->component (storm-task-info topology topology-conf) + base (.storm-base storm-cluster-state storm-id nil) + launch-time-secs (if base (:launch-time-secs base) + (throw + (NotAliveException. (str storm-id)))) + assignment (.assignment-info storm-cluster-state storm-id nil) + beats (map-val :heartbeat (get @(:heartbeats-cache nimbus) + storm-id)) + all-components (set (vals task->component))] + {:storm-name storm-name + :storm-cluster-state storm-cluster-state + :all-components all-components + :launch-time-secs launch-time-secs + :assignment assignment + :beats beats + :topology topology + :task->component task->component + :base base})) + get-last-error (fn [storm-cluster-state storm-id component-id] + (if-let [e (.last-error storm-cluster-state + storm-id + component-id)] + (doto (ErrorInfo. (:error e) (:time-secs e)) + (.set_host (:host e)) + (.set_port (:port e)))))] (.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf) ;add to nimbuses @@ -1453,16 +1484,14 @@ )) (^TopologyInfo getTopologyInfoWithOpts [this ^String storm-id ^GetInfoOptions options] - (let [storm-cluster-state (:storm-cluster-state nimbus) - topology-conf (try-read-storm-conf conf storm-id) - storm-name (topology-conf TOPOLOGY-NAME) - _ (check-authorization! nimbus storm-name topology-conf "getTopologyInfo") - task->component (storm-task-info (try-read-storm-topology conf storm-id) topology-conf) - base (.storm-base storm-cluster-state storm-id nil) - launch-time-secs (if base (:launch-time-secs base) (throw (NotAliveException. (str storm-id)))) - assignment (.assignment-info storm-cluster-state storm-id nil) - beats (map-val :heartbeat (get @(:heartbeats-cache nimbus) storm-id)) - all-components (-> task->component reverse-map keys) + (let [{:keys [storm-name + storm-cluster-state + all-components + launch-time-secs + assignment + beats + task->component + base]} (get-common-topo-info storm-id "getTopologyInfo") num-err-choice (or (.get_num_err_choice options) NumErrorsChoice/ALL) errors-fn (condp = num-err-choice @@ -1509,11 +1538,94 @@ topo-info )) - (^TopologyInfo getTopologyInfo [this ^String storm-id] + (^TopologyInfo getTopologyInfo [this ^String topology-id] (.getTopologyInfoWithOpts this - storm-id + topology-id (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL)))) + (^TopologyPageInfo getTopologyPageInfo + [this ^String topo-id ^String window ^boolean include-sys?] + (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)] + (when-let [owner (:owner (:base info))] + (.set_owner topo-page-info owner)) + (when-let [sched-status (.get @(:id->sched-status nimbus) topo-id)] + (.set_sched_status topo-page-info sched-status)) + (doto topo-page-info + (.set_name (:storm-name info)) + (.set_status (extract-status-str (:base info))) + (.set_uptime_secs (time-delta (:launch-time-secs info))) + (.set_topology_conf (to-json (try-read-storm-conf conf + topo-id))) + (.set_replication_count + (.getReplicationCount (:code-distributor nimbus) topo-id))) + (when-let [debug-options + (get-in info [:base :component->debug topo-id])] + (.set_debug_options + topo-page-info + (converter/thriftify-debugoptions debug-options))) + topo-page-info)) + + (^ComponentPageInfo getComponentPageInfo + [this + ^String topo-id + ^String component-id + ^String window + ^boolean include-sys?] + (let [info (get-common-topo-info topo-id "getComponentPageInfo") + {:keys [executor->node+port node->host]} (:assignment info) + 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 + (:task->component info) + (:beats info) + window + include-sys? + topo-id + (:topology info) + component-id)] + (doto comp-page-info + (.set_topology_name (:storm-name info)) + (.set_errors (get-errors (:storm-cluster-state info) + topo-id + component-id)) + (.set_topology_status (extract-status-str (:base info)))) + (when-let [debug-options + (get-in info [:base :component->debug component-id])] + (.set_debug_options + comp-page-info + (converter/thriftify-debugoptions debug-options))) + ;; Add the event logger details. + (let [component->tasks (reverse-map (:task->component info)) + eventlogger-tasks (sort (get component->tasks + EVENTLOGGER-COMPONENT-ID)) + ;; Find the task the events from this component route to. + task-index (mod (tuple/list-hash-code [component-id]) + (count eventlogger-tasks)) + task-id (nth eventlogger-tasks task-index) + eventlogger-exec (first (filter (fn [[start stop]] + (between? task-id start stop)) + (keys executor->host+port))) + [host port] (get executor->host+port eventlogger-exec)] + (if (and host port) + (doto comp-page-info + (.set_eventlog_host host) + (.set_eventlog_port port)))) + comp-page-info)) + Shutdownable (shutdown [this] (log-message "Shutting down master") diff --git a/storm-core/src/clj/backtype/storm/log.clj b/storm-core/src/clj/backtype/storm/log.clj index 55b44fec23a..abe9b3263ab 100644 --- a/storm-core/src/clj/backtype/storm/log.clj +++ b/storm-core/src/clj/backtype/storm/log.clj @@ -15,7 +15,9 @@ ;; limitations under the License. (ns backtype.storm.log - (:require [clojure.tools.logging :as log])) + (:require [clojure.tools.logging :as log]) + (:use [clojure pprint]) + (:import [java.io StringWriter])) (defmacro log-message [& args] @@ -44,3 +46,11 @@ (defn log-stream [& args] (apply log/log-stream args)) + +(defmacro log-pprint + [& args] + `(let [^StringWriter writer# (StringWriter.)] + (doall + (for [object# [~@args]] + (pprint object# writer#))) + (log-message "\n" writer#))) diff --git a/storm-core/src/clj/backtype/storm/stats.clj b/storm-core/src/clj/backtype/storm/stats.clj index db8930cc2f0..3f55fc18e19 100644 --- a/storm-core/src/clj/backtype/storm/stats.clj +++ b/storm-core/src/clj/backtype/storm/stats.clj @@ -17,13 +17,20 @@ (ns backtype.storm.stats (:import [backtype.storm.generated Nimbus Nimbus$Processor Nimbus$Iface StormTopology ShellComponent NotAliveException AlreadyAliveException InvalidTopologyException GlobalStreamId - ClusterSummary TopologyInfo TopologySummary ExecutorSummary ExecutorStats ExecutorSpecificStats - SpoutStats BoltStats ErrorInfo SupervisorSummary]) - (:use [backtype.storm util log]) - (:use [clojure.math.numeric-tower :only [ceil]])) + ClusterSummary TopologyInfo TopologySummary ExecutorInfo ExecutorSummary ExecutorStats + ExecutorSpecificStats SpoutStats BoltStats ErrorInfo + SupervisorSummary CommonAggregateStats ComponentAggregateStats + ComponentPageInfo ComponentType BoltAggregateStats + ExecutorAggregateStats SpecificAggregateStats + SpoutAggregateStats TopologyPageInfo TopologyStats]) + (:use [backtype.storm log util]) + (:use [clojure.math.numeric-tower :only [ceil]]) + (:use [backtype.storm.daemon [common :only [system-id?]]])) ;;TODO: consider replacing this with some sort of RRD +(def TEN-MIN-IN-SECONDS (* 10 60)) + (defn curr-time-bucket [^Integer time-secs ^Integer bucket-size-secs] (* bucket-size-secs (unchecked-divide-int time-secs bucket-size-secs))) @@ -141,26 +148,6 @@ [num-buckets & bucket-sizes] (apply rolling-window-set update-keyed-avg merge-keyed-avg extract-keyed-avg num-buckets bucket-sizes)) -;; (defn choose-bucket [val buckets] -;; (let [ret (find-first #(<= val %) buckets)] -;; (if ret -;; ret -;; (* 10 (first buckets))) -;; )) - -;; ;; buckets must be between 1 and 9 -;; (defn to-proportional-bucket -;; "Maps to a bucket in the values order of magnitude. So if buckets are [1 2 5], -;; 3 -> 5 -;; 7 -> 10 -;; 1234 -> 2000 -;; etc." -;; [val buckets] -;; (cond (= 0 val) 0 -;; (between? val 1 9) (choose-bucket val buckets) -;; :else (* 10 (to-proportional-bucket (ceil (/ val 10)) -;; buckets)))) - (def COMMON-FIELDS [:emitted :transferred]) (defrecord CommonStats [emitted transferred rate]) @@ -322,16 +309,16 @@ [(.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 symbol) - (window-set-converter (.get_failed stats) from-global-stream-id symbol) - (window-set-converter (.get_process_ms_avg stats) from-global-stream-id symbol) - (window-set-converter (.get_executed stats) from-global-stream-id symbol) - (window-set-converter (.get_execute_ms_avg stats) from-global-stream-id symbol)]) + [(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] - [(window-set-converter (.get_acked stats) symbol) - (window-set-converter (.get_failed stats) symbol) - (window-set-converter (.get_complete_ms_avg stats) symbol)]) + [(.get_acked stats) + (.get_failed stats) + (.get_complete_ms_avg stats)]) (defn clojurify-executor-stats @@ -340,7 +327,9 @@ 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. (window-set-converter (.get_emitted stats) symbol) (window-set-converter (.get_transferred stats) symbol) (.get_rate 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 @@ -373,4 +362,1229 @@ (ExecutorStats. (window-set-converter (:emitted stats) str) (window-set-converter (:transferred stats) str) specific-stats - rate))) \ No newline at end of file + rate))) + +(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] + {:pre (apply = (map #(set (keys %)) + [idk->exec-avg + idk->proc-avg + idk->num-executed]))} + (letfn [(weight-avg [[id avg]] (let [num-e (get idk->num-executed id)] + (if (and avg num-e) + (* avg num-e) + 0)))] + {:executeLatencyTotal (sum (map weight-avg idk->exec-avg)) + :processLatencyTotal (sum (map weight-avg idk->proc-avg)) + :executed (sum (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] + {:pre (apply = (map #(set (keys %)) + [sid->comp-avg + sid->num-acked]))} + (letfn [(weight-avg [[id avg]] (* avg (get sid->num-acked id)))] + {:completeLatencyTotal (sum (map weight-avg sid->comp-avg)) + :acked (sum (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 (system-id? stream)))))) + +(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] + {:pre (apply = (map #(set (keys %)) + [idk->exec-avg + idk->proc-avg + idk->executed]))} + (letfn [(weight-avg [id avg] (let [num-e (idk->executed id)] + (if (and avg num-e) + (* avg num-e) + 0)))] + (into {} + (for [k (keys idk->exec-avg)] + [k {:executeLatencyTotal (weight-avg k (idk->exec-avg k)) + :processLatencyTotal (weight-avg k (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] + {:pre (apply = (map #(set (keys %)) + [idk->comp-avg + idk->acked]))} + (letfn [(weight-avg [id avg] (let [num-e (get idk->acked id)] + (if (and avg num-e) + (* avg num-e) + 0)))] + (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?] + (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?] + (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?] + (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 + sum) + :transferred (-> statk->w->sid->num + :transferred + str-key + (get window) + handle-sys-components-fn + vals + sum) + :capacity (compute-agg-capacity statk->w->sid->num uptime) + :acked (-> statk->w->sid->num + :acked + str-key + (get window) + vals + sum) + :failed (-> statk->w->sid->num + :failed + str-key + (get window) + vals + sum)})})) + +(defn agg-pre-merge-topo-page-spout + [{comp-id :comp-id + num-tasks :num-tasks + statk->w->sid->num :stats} + window + include-sys?] + (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 + sum) + :transferred (-> statk->w->sid->num + :transferred + str-key + (get window) + handle-sys-components-fn + vals + sum) + :failed (-> statk->w->sid->num + :failed + str-key + (get window) + vals + sum)})})) + +(defn apply-default + [f defaulting-fn & args] + (apply f (map defaulting-fn args))) + +(defn apply-or-0 + [f & args] + (apply apply-default f #(or % 0) args)) + +(defn sum-or-0 + [& args] + (apply apply-or-0 + args)) + +(defn max-or-0 + [& args] + (apply apply-or-0 max args)) + +(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))}) + +(defn aggregate-count-streams + [stats] + (->> stats + (map-val #(reduce + (vals %))))) + +(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 + (.containsKey bolts id) :bolt + (.containsKey spouts id) :spout))) + +(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 (system-id? 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)))])) + :window->emitted (map-key str (:window->emitted acc-data)) + :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) + :window->acked (map-key str (:window->acked acc-data)) + :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))) + +(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)))})) + +(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))) + +(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)) + :window->emitted (map-key str (:window->emitted acc-data)) + :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) + :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) + :window->acked (map-key str (:window->acked acc-data)) + :window->failed (map-key str (:window->failed acc-data))}) + +(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)) + :window->emitted (map-key str (:window->emitted acc-data)) + :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) + :window->acked (map-key str (:window->acked acc-data)) + :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)]))) + +(defn- thriftify-bolt-output-stats + [sid->output-stats] + (map-val thriftify-bolt-agg-stats sid->output-stats)) + +(defn- thriftify-spout-output-stats + [sid->output-stats] + (map-val thriftify-spout-agg-stats sid->output-stats)) + +(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 (= t 0) 0 + (double (/ t c)))) + +(defn aggregate-averages + [average-seq counts-seq] + (->> (expand-averages-seq average-seq counts-seq) + (map-val + (fn [s] + (map-val val-avg s))))) + +(defn aggregate-avg-streams + [avg counts] + (let [expanded (expand-averages avg counts)] + (->> expanded + (map-val #(reduce add-pairs (vals %))) + (map-val val-avg)))) + +(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 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 (nil-to-zero (.get_uptime_secs e)) + window (if (< uptime TEN-MIN-IN-SECONDS) uptime TEN-MIN-IN-SECONDS) + executed (-> stats :executed nil-to-zero) + latency (-> stats :execute-latencies nil-to-zero)] + (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 nil-to-zero) + (apply max))) diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj index 4cd9046af12..5fa487feb4d 100644 --- a/storm-core/src/clj/backtype/storm/ui/core.clj +++ b/storm-core/src/clj/backtype/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 [backtype.storm config util log tuple zookeeper]) + (:use [backtype.storm config util log stats tuple zookeeper]) (:use [backtype.storm.ui helpers]) (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID ACKER-FAIL-STREAM-ID system-id? mk-authorization-handler]]]) @@ -32,7 +32,10 @@ ExecutorStats ExecutorSummary ExecutorInfo TopologyInfo SpoutStats BoltStats ErrorInfo ClusterSummary SupervisorSummary TopologySummary Nimbus$Client StormTopology GlobalStreamId RebalanceOptions - KillOptions GetInfoOptions NumErrorsChoice DebugOptions + KillOptions GetInfoOptions NumErrorsChoice DebugOptions TopologyPageInfo + TopologyStats CommonAggregateStats ComponentAggregateStats + ComponentType BoltAggregateStats SpoutAggregateStats + ExecutorAggregateStats SpecificAggregateStats ComponentPageInfo LogConfig LogLevel LogLevelAction]) (:import [backtype.storm.security.auth AuthUtils ReqContext]) (:import [backtype.storm.generated AuthorizationException]) @@ -82,94 +85,10 @@ (throw (AuthorizationException. (str "UI request '" op "' for '" user "' user is not authorized"))))))))) -(defn get-filled-stats - [summs] - (->> summs - (map #(.get_stats ^ExecutorSummary %)) - (filter not-nil?))) - -(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 - (.containsKey bolts id) :bolt - (.containsKey spouts id) :spout))) - (defn executor-summary-type [topology ^ExecutorSummary s] (component-type topology (.get_component_id s))) -(defn add-pairs - ([] [0 0]) - ([[a1 a2] [b1 b2]] - [(+ a1 b1) (+ a2 b2)])) - -(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 (= t 0) 0 - (double (/ t c)))) - -(defn aggregate-averages - [average-seq counts-seq] - (->> (expand-averages-seq average-seq counts-seq) - (map-val - (fn [s] - (map-val val-avg s))))) - -(defn aggregate-counts - [counts-seq] - (->> counts-seq - (map clojurify-structure) - (apply merge-with - (fn [s1 s2] - (merge-with + s1 s2))))) - -(defn aggregate-avg-streams - [avg counts] - (let [expanded (expand-averages avg counts)] - (->> expanded - (map-val #(reduce add-pairs (vals %))) - (map-val val-avg)))) - -(defn aggregate-count-streams - [stats] - (->> stats - (map-val #(reduce + (vals %))))) - -(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 mk-include-sys-fn - [include-sys?] - (if include-sys? - (fn [_] true) - (fn [stream] (and (string? stream) (not (system-id? stream)))))) - (defn is-ack-stream [stream] (let [acker-streams @@ -178,80 +97,6 @@ ACKER-FAIL-STREAM-ID]] (every? #(not= %1 stream) acker-streams))) -(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-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 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 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-summary? [topology s] (= :spout (executor-summary-type topology s))) @@ -265,29 +110,6 @@ (let [ret (group-by #(.get_component_id ^ExecutorSummary %) summs)] (into (sorted-map) ret ))) -(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)] - error)) - -(defn component-task-summs - [^TopologyInfo summ topology id] - (let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ)) - bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ)) - spout-comp-summs (group-by-comp spout-summs) - bolt-comp-summs (group-by-comp bolt-summs) - ret (if (contains? spout-comp-summs id) - (spout-comp-summs id) - (bolt-comp-summs id))] - (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret))) - (defn logviewer-link [host fname secure?] (if (and secure? (*STORM-CONF* LOGVIEWER-HTTPS-PORT)) (url-format "https://%s:%s/log?file=%s" @@ -300,36 +122,9 @@ fname)) ) -(defn executor-has-task-id? [task-id executor-info] - (between? task-id (.get_task_start executor-info) (.get_task_end executor-info))) - -(defn get-host-port [task-id executor-summs] - (let [ex-sum (some #(if (executor-has-task-id? task-id (.get_executor_info %)) %) executor-summs)] - {:host (.get_host ex-sum) :port (.get_port ex-sum)})) - -(defn get-sorted-eventlogger-task-ids [executor-summs] - (let [executor-infos (map #(.get_executor_info %) executor-summs)] - (sort (flatten (map #(range (.get_task_start %) (inc (.get_task_end %))) executor-infos))))) - -(defn get-eventlogger-executor-summs [^TopologyInfo topology-info topology] - (let [bolt-summs (filter (partial bolt-summary? topology) (.get_executors topology-info))] - ((group-by-comp bolt-summs) "__eventlogger"))) - -; -; The eventlogger uses fields grouping on the component-id so that events from same component -; always goes to the same event logger task. Here we use the same fields grouping -; to find the correct eventlogger task. -(defn get-mapped-task-id [sorted-task-ids ^String component-id] - (nth sorted-task-ids (mod (list-hash-code [component-id]) (count sorted-task-ids)))) - (defn event-log-link - [topology-id ^TopologyInfo topology-info topology component-id secure?] - (let [executor-summs (get-eventlogger-executor-summs topology-info topology) - sorted-task-ids (get-sorted-eventlogger-task-ids executor-summs) - mapped-task-id (get-mapped-task-id sorted-task-ids component-id) - host-port (get-host-port mapped-task-id executor-summs) - fname (event-logs-filename topology-id (host-port :port))] - (logviewer-link (host-port :host) fname secure?))) + [topology-id component-id host port secure?] + (logviewer-link host (event-logs-filename topology-id port) secure?)) (defn worker-log-link [host port topology-id secure?] (let [fname (logs-filename topology-id port)] @@ -337,30 +132,6 @@ (defn nimbus-log-link [host port] (url-format "http://%s:%s/log?file=nimbus.log" host (*STORM-CONF* LOGVIEWER-PORT) port)) - -(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 "600"))) - uptime (nil-to-zero (.get_uptime_secs e)) - window (if (< uptime 600) uptime 600) - executed (-> stats :executed nil-to-zero) - latency (-> stats :execute-latencies nil-to-zero)] - (if (> window 0) - (div (* executed latency) (* 1000 window))))) - -(defn compute-bolt-capacity - [executors] - (->> executors - (map compute-executor-capacity) - (map nil-to-zero) - (apply max))) - (defn get-error-time [error] (if error @@ -373,7 +144,7 @@ "")) (defn get-error-port - [error error-host top-id] + [error] (if error (.get_port ^ErrorInfo error) "")) @@ -384,40 +155,11 @@ (.get_host ^ErrorInfo error) "")) -(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 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 get-error-time + [error] + (if error + (.get_error_time_secs ^ErrorInfo error) + "")) (defn stats-times [stats-map] @@ -433,16 +175,6 @@ "All time" (pretty-uptime-sec window))) -(defn topology-action-button - [id name action command is-wait default-wait enabled] - [:input {:type "button" - :value action - (if enabled :enabled :disabled) "" - :onclick (str "confirmAction('" - (StringEscapeUtils/escapeJavaScript id) "', '" - (StringEscapeUtils/escapeJavaScript name) "', '" - command "', " is-wait ", " default-wait ")")}]) - (defn sanitize-stream-name [name] (let [sym-regex #"(?![A-Za-z_\-:\.])."] @@ -536,9 +268,7 @@ spout-comp-summs (group-by-comp spout-summs) bolt-comp-summs (group-by-comp bolt-summs) bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) - bolt-comp-summs) - topology-conf (from-json - (.getTopologyConf ^Nimbus$Client nimbus id))] + bolt-comp-summs)] (visualization-data (merge (hashmap-to-persistent spouts) (hashmap-to-persistent bolts)) @@ -681,181 +411,150 @@ "replicationCount" (.get_replication_count t) "schedulerInfo" (.get_sched_status t)})})) -(defn topology-stats [id window stats] +(defn topology-stats [window stats] (let [times (stats-times (:emitted stats)) display-map (into {} (for [t times] [t pretty-uptime-sec])) display-map (assoc display-map ":all-time" (fn [_] "All time"))] - (for [k (concat times [":all-time"]) - :let [disp ((display-map k) k)]] + (for [w (concat times [":all-time"]) + :let [disp ((display-map w) w)]] {"windowPretty" disp - "window" k - "emitted" (get-in stats [:emitted k]) - "transferred" (get-in stats [:transferred k]) - "completeLatency" (float-str (get-in stats [:complete-latencies k])) - "acked" (get-in stats [:acked k]) - "failed" (get-in stats [:failed k])}))) - -(defn spout-comp [top-id summ-map errors window include-sys? secure?] - (for [[id summs] summ-map - :let [stats-seq (get-filled-stats summs) - stats (aggregate-spout-streams - (aggregate-spout-stats - stats-seq include-sys?)) - last-error (most-recent-error (get errors id)) - error-host (get-error-host last-error) - error-port (get-error-port last-error error-host top-id)]] - {"spoutId" id - "encodedSpoutId" (url-encode id) - "executors" (count summs) - "tasks" (sum-tasks summs) - "emitted" (get-in stats [:emitted window]) - "transferred" (get-in stats [:transferred window]) - "completeLatency" (float-str (get-in stats [:complete-latencies window])) - "acked" (get-in stats [:acked window]) - "failed" (get-in stats [:failed window]) - "errorHost" error-host - "errorPort" error-port - "errorWorkerLogLink" (worker-log-link error-host error-port top-id secure?) - "errorLapsedSecs" (get-error-time last-error) - "lastError" (get-error-data last-error) - "time" (if last-error (* 1000 (long (.get_error_time_secs ^ErrorInfo last-error))))})) - -(defn bolt-comp [top-id summ-map errors window include-sys? secure?] - (for [[id summs] summ-map - :let [stats-seq (get-filled-stats summs) - stats (aggregate-bolt-streams - (aggregate-bolt-stats - stats-seq include-sys?)) - last-error (most-recent-error (get errors id)) - error-host (get-error-host last-error) - error-port (get-error-port last-error error-host top-id)]] - {"boltId" id - "encodedBoltId" (url-encode id) - "executors" (count summs) - "tasks" (sum-tasks summs) - "emitted" (get-in stats [:emitted window]) - "transferred" (get-in stats [:transferred window]) - "capacity" (float-str (nil-to-zero (compute-bolt-capacity summs))) - "executeLatency" (float-str (get-in stats [:execute-latencies window])) - "executed" (get-in stats [:executed window]) - "processLatency" (float-str (get-in stats [:process-latencies window])) - "acked" (get-in stats [:acked window]) - "failed" (get-in stats [:failed window]) - "errorHost" error-host - "errorPort" error-port - "errorWorkerLogLink" (worker-log-link error-host error-port top-id secure?) - "errorLapsedSecs" (get-error-time last-error) - "lastError" (get-error-data last-error) - "time" (if last-error (* 1000 (long (.get_error_time_secs ^ErrorInfo last-error))))})) - -(defn topology-summary [^TopologyInfo summ] - (let [executors (.get_executors summ) - workers (set (for [^ExecutorSummary e executors] - [(.get_host e) (.get_port e)])) - topology-id (.get_id summ) - debug-options (get (.get_component_debug summ) topology-id)] - {"id" topology-id - "encodedId" (url-encode (.get_id summ)) - "owner" (.get_owner summ) - "name" (.get_name summ) - "status" (.get_status summ) - "uptime" (pretty-uptime-sec (.get_uptime_secs summ)) - "uptimeSeconds" (.get_uptime_secs summ) - "tasksTotal" (sum-tasks executors) - "workersTotal" (count workers) - "executorsTotal" (count executors) - "schedulerInfo" (.get_sched_status summ) - "debug" (if (not-nil? debug-options) (.is_enable debug-options) false) - "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 10) - "replicationCount" (.get_replication_count summ)})) - -(defn spout-summary-json [topology-id id stats window] - (let [times (stats-times (:emitted stats)) - display-map (into {} (for [t times] [t pretty-uptime-sec])) - display-map (assoc display-map ":all-time" (fn [_] "All time"))] - (for [k (concat times [":all-time"]) - :let [disp ((display-map k) k)]] - {"windowPretty" disp - "window" k - "emitted" (get-in stats [:emitted k]) - "transferred" (get-in stats [:transferred k]) - "completeLatency" (float-str (get-in stats [:complete-latencies k])) - "acked" (get-in stats [:acked k]) - "failed" (get-in stats [:failed k])}))) + "window" w + "emitted" (get-in stats [:emitted w]) + "transferred" (get-in stats [:transferred w]) + "completeLatency" (float-str (get-in stats [:complete-latencies w])) + "acked" (get-in stats [:acked w]) + "failed" (get-in stats [:failed w])}))) + +(defn build-visualization [id window include-sys? user] + (thrift/with-configured-nimbus-connection nimbus + (let [window (if window window ":all-time") + topology-info (->> (doto + (GetInfoOptions.) + (.set_num_err_choice NumErrorsChoice/ONE)) + (.getTopologyInfoWithOpts ^Nimbus$Client nimbus + id)) + storm-topology (.getTopology ^Nimbus$Client nimbus id) + spout-executor-summaries (filter (partial spout-summary? storm-topology) (.get_executors topology-info)) + bolt-executor-summaries (filter (partial bolt-summary? storm-topology) (.get_executors topology-info)) + spout-comp-id->executor-summaries (group-by-comp spout-executor-summaries) + bolt-comp-id->executor-summaries (group-by-comp bolt-executor-summaries) + bolt-comp-id->executor-summaries (filter-key (mk-include-sys-fn include-sys?) bolt-comp-id->executor-summaries) + id->spout-spec (.get_spouts storm-topology) + id->bolt (.get_bolts storm-topology) + visualizer-data (visualization-data (merge (hashmap-to-persistent id->spout-spec) + (hashmap-to-persistent id->bolt)) + spout-comp-id->executor-summaries + bolt-comp-id->executor-summaries + window + id)] + {"visualizationTable" (stream-boxes visualizer-data)}))) + +(defn- get-error-json + [topo-id error-info secure?] + (let [host (get-error-host error-info) + port (get-error-port error-info)] + {"lastError" (get-error-data error-info) + "errorTime" (get-error-time error-info) + "errorHost" host + "errorPort" port + "errorLapsedSecs" (get-error-time error-info) + "errorWorkerLogLink" (worker-log-link host port topo-id secure?)})) + +(defn- common-agg-stats-json + "Returns a JSON representation of a common aggregated statistics." + [^CommonAggregateStats common-stats] + {"executors" (.get_num_executors common-stats) + "tasks" (.get_num_tasks common-stats) + "emitted" (.get_emitted common-stats) + "transferred" (.get_transferred common-stats) + "acked" (.get_acked common-stats) + "failed" (.get_failed common-stats)}) + +(defmulti comp-agg-stats-json + "Returns a JSON representation of aggregated statistics." + (fn [_ _ [id ^ComponentAggregateStats s]] (.get_type s))) + +(defmethod comp-agg-stats-json ComponentType/SPOUT + [topo-id secure? [id ^ComponentAggregateStats s]] + (let [^SpoutAggregateStats ss (.. s get_specific_stats get_spout) + cs (.get_common_stats s)] + (merge + (common-agg-stats-json cs) + (get-error-json topo-id (.get_last_error s) secure?) + {"spoutId" id + "encodedSpoutId" (url-encode id) + "completeLatency" (float-str (.get_complete_latency_ms ss))}))) + +(defmethod comp-agg-stats-json ComponentType/BOLT + [topo-id secure? [id ^ComponentAggregateStats s]] + (let [^BoltAggregateStats ss (.. s get_specific_stats get_bolt) + cs (.get_common_stats s)] + (merge + (common-agg-stats-json cs) + (get-error-json topo-id (.get_last_error s) secure?) + {"boltId" id + "encodedBoltId" (url-encode id) + "capacity" (float-str (.get_capacity ss)) + "executeLatency" (float-str (.get_execute_latency_ms ss)) + "executed" (.get_executed ss) + "processLatency" (float-str (.get_process_latency_ms ss))}))) + +(defn- unpack-topology-page-info + "Unpacks the serialized object to data structures" + [^TopologyPageInfo topo-info window secure?] + (let [id (.get_id topo-info) + ^TopologyStats topo-stats (.get_topology_stats topo-info) + stat->window->number + {:emitted (.get_window_to_emitted topo-stats) + :transferred (.get_window_to_transferred topo-stats) + :complete-latencies (.get_window_to_complete_latencies_ms topo-stats) + :acked (.get_window_to_acked topo-stats) + :failed (.get_window_to_failed topo-stats)} + topo-stats (topology-stats window stat->window->number) + [debugEnabled + samplingPct] (if-let [debug-opts (.get_debug_options topo-info)] + [(.is_enable debug-opts) + (.get_samplingpct debug-opts)]) + uptime (.get_uptime_secs topo-info)] + {"id" id + "encodedId" (url-encode id) + "owner" (.get_owner topo-info) + "name" (.get_name topo-info) + "status" (.get_status topo-info) + "uptime" (pretty-uptime-sec uptime) + "uptimeSeconds" uptime + "tasksTotal" (.get_num_tasks topo-info) + "workersTotal" (.get_num_workers topo-info) + "executorsTotal" (.get_num_executors topo-info) + "schedulerInfo" (.get_sched_status topo-info) + "topologyStats" topo-stats + "spouts" (map (partial comp-agg-stats-json id secure?) + (.get_id_to_spout_agg_stats topo-info)) + "bolts" (map (partial comp-agg-stats-json id secure?) + (.get_id_to_bolt_agg_stats topo-info)) + "configuration" (.get_topology_conf topo-info) + "debug" (or debugEnabled false) + "samplingPct" (or samplingPct 10) + "replicationCount" (.get_replication_count topo-info)})) (defn topology-page [id window include-sys? user secure?] (thrift/with-configured-nimbus-connection nimbus (let [window (if window window ":all-time") window-hint (window-hint window) - summ (->> (doto - (GetInfoOptions.) - (.set_num_err_choice NumErrorsChoice/ONE)) - (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id)) - topology (.getTopology ^Nimbus$Client nimbus id) - topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id)) - spout-summs (filter (partial spout-summary? topology) (.get_executors summ)) - bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ)) - spout-comp-summs (group-by-comp spout-summs) - bolt-comp-summs (group-by-comp bolt-summs) - bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs) - name (.get_name summ) - status (.get_status summ) - msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS) - spouts (.get_spouts topology) - bolts (.get_bolts topology) - replication-count (.get_replication_count summ) - visualizer-data (visualization-data (merge (hashmap-to-persistent spouts) - (hashmap-to-persistent bolts)) - spout-comp-summs - bolt-comp-summs - window - id)] + topo-page-info (.getTopologyPageInfo ^Nimbus$Client nimbus + id + window + include-sys?) + topology-conf (from-json (.get_topology_conf topo-page-info)) + msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)] (merge - (topology-summary summ) + (unpack-topology-page-info topo-page-info window secure?) {"user" user "window" window "windowHint" window-hint "msgTimeout" msg-timeout - "topologyStats" (topology-stats id window (total-aggregate-stats spout-summs bolt-summs include-sys?)) - "spouts" (spout-comp id spout-comp-summs (.get_errors summ) window include-sys? secure?) - "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys? secure?) "configuration" topology-conf - "visualizationTable" (stream-boxes visualizer-data) - "replicationCount" replication-count})))) - -(defn spout-output-stats - [stream-summary window] - (let [stream-summary (map-val swap-map-order (swap-map-order stream-summary))] - (for [[s stats] (stream-summary window)] - {"stream" s - "emitted" (nil-to-zero (:emitted stats)) - "transferred" (nil-to-zero (:transferred stats)) - "completeLatency" (float-str (:complete-latencies stats)) - "acked" (nil-to-zero (:acked stats)) - "failed" (nil-to-zero (:failed stats))}))) - -(defn spout-executor-stats - [topology-id executors window include-sys? secure?] - (for [^ExecutorSummary e executors - :let [stats (.get_stats e) - stats (if stats - (-> stats - (aggregate-spout-stats include-sys?) - aggregate-spout-streams - swap-map-order - (get window)))]] - {"id" (pretty-executor-info (.get_executor_info e)) - "encodedId" (url-encode (pretty-executor-info (.get_executor_info e))) - "uptime" (pretty-uptime-sec (.get_uptime_secs e)) - "uptimeSeconds" (.get_uptime_secs e) - "host" (.get_host e) - "port" (.get_port e) - "emitted" (nil-to-zero (:emitted stats)) - "transferred" (nil-to-zero (:transferred stats)) - "completeLatency" (float-str (:complete-latencies stats)) - "acked" (nil-to-zero (:acked stats)) - "failed" (nil-to-zero (:failed stats)) - "workerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id secure?)})) + "visualizationTable" []})))) (defn component-errors [errors-list topology-id secure?] @@ -867,139 +566,204 @@ {"time" (* 1000 (long (.get_error_time_secs e))) "errorHost" (.get_host e) "errorPort" (.get_port e) - "errorWorkerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id secure?) + "errorWorkerLogLink" (worker-log-link (.get_host e) + (.get_port e) + topology-id + secure?) "errorLapsedSecs" (get-error-time e) "error" (.get_error e)})})) -(defn spout-stats - [window ^TopologyInfo topology-info component executors include-sys? secure?] - (let [window-hint (str " (" (window-hint window) ")") - stats (get-filled-stats executors) - stream-summary (-> stats (aggregate-spout-stats include-sys?)) - summary (-> stream-summary aggregate-spout-streams)] - {"spoutSummary" (spout-summary-json - (.get_id topology-info) component summary window) - "outputStats" (spout-output-stats stream-summary window) - "executorStats" (spout-executor-stats (.get_id topology-info) - executors window include-sys? secure?)})) - -(defn bolt-summary - [topology-id id stats window] - (let [times (stats-times (:emitted stats)) - display-map (into {} (for [t times] [t pretty-uptime-sec])) - display-map (assoc display-map ":all-time" (fn [_] "All time"))] - (for [k (concat times [":all-time"]) - :let [disp ((display-map k) k)]] - {"window" k - "windowPretty" disp - "emitted" (get-in stats [:emitted k]) - "transferred" (get-in stats [:transferred k]) - "executeLatency" (float-str (get-in stats [:execute-latencies k])) - "executed" (get-in stats [:executed k]) - "processLatency" (float-str (get-in stats [:process-latencies k])) - "acked" (get-in stats [:acked k]) - "failed" (get-in stats [:failed k])}))) - -(defn bolt-output-stats - [stream-summary window] - (let [stream-summary (-> stream-summary - swap-map-order - (get window) - (select-keys [:emitted :transferred]) - swap-map-order)] - (for [[s stats] stream-summary] - {"stream" s - "emitted" (nil-to-zero (:emitted stats)) - "transferred" (nil-to-zero (:transferred stats))}))) - -(defn bolt-input-stats - [stream-summary window] - (let [stream-summary - (-> stream-summary - swap-map-order - (get window) - (select-keys [:acked :failed :process-latencies - :executed :execute-latencies]) - swap-map-order)] - (for [[^GlobalStreamId s stats] stream-summary] - {"component" (.get_componentId s) - "encodedComponent" (url-encode (.get_componentId s)) - "stream" (.get_streamId s) - "executeLatency" (float-str (:execute-latencies stats)) - "processLatency" (float-str (:process-latencies stats)) - "executed" (nil-to-zero (:executed stats)) - "acked" (nil-to-zero (:acked stats)) - "failed" (nil-to-zero (:failed stats))}))) - -(defn bolt-executor-stats - [topology-id executors window include-sys? secure?] - (for [^ExecutorSummary e executors - :let [stats (.get_stats e) - stats (if stats - (-> stats - (aggregate-bolt-stats include-sys?) - (aggregate-bolt-streams) - swap-map-order - (get window)))]] - {"id" (pretty-executor-info (.get_executor_info e)) - "encodedId" (url-encode (pretty-executor-info (.get_executor_info e))) - "uptime" (pretty-uptime-sec (.get_uptime_secs e)) - "uptimeSeconds" (.get_uptime_secs e) - "host" (.get_host e) - "port" (.get_port e) - "emitted" (nil-to-zero (:emitted stats)) - "transferred" (nil-to-zero (:transferred stats)) - "capacity" (float-str (nil-to-zero (compute-executor-capacity e))) - "executeLatency" (float-str (:execute-latencies stats)) - "executed" (nil-to-zero (:executed stats)) - "processLatency" (float-str (:process-latencies stats)) - "acked" (nil-to-zero (:acked stats)) - "failed" (nil-to-zero (:failed stats)) - "workerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id secure?)})) - -(defn bolt-stats - [window ^TopologyInfo topology-info component executors include-sys? secure?] - (let [window-hint (str " (" (window-hint window) ")") - stats (get-filled-stats executors) - stream-summary (-> stats (aggregate-bolt-stats include-sys?)) - summary (-> stream-summary aggregate-bolt-streams)] - {"boltStats" (bolt-summary (.get_id topology-info) component summary window) - "inputStats" (bolt-input-stats stream-summary window) - "outputStats" (bolt-output-stats stream-summary window) - "executorStats" (bolt-executor-stats - (.get_id topology-info) executors window include-sys? secure?)})) +(defmulti unpack-comp-agg-stat + (fn [[_ ^ComponentAggregateStats s]] (.get_type s))) + +(defmethod unpack-comp-agg-stat ComponentType/BOLT + [[window ^ComponentAggregateStats s]] + (let [^CommonAggregateStats comm-s (.get_common_stats s) + ^SpecificAggregateStats spec-s (.get_specific_stats s) + ^BoltAggregateStats bolt-s (.get_bolt spec-s)] + {"window" window + "windowPretty" (window-hint window) + "emitted" (.get_emitted comm-s) + "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)) + "executed" (.get_executed bolt-s) + "capacity" (float-str (.get_capacity bolt-s))})) + +(defmethod unpack-comp-agg-stat ComponentType/SPOUT + [[window ^ComponentAggregateStats s]] + (let [^CommonAggregateStats comm-s (.get_common_stats s) + ^SpecificAggregateStats spec-s (.get_specific_stats s) + ^SpoutAggregateStats spout-s (.get_spout spec-s)] + {"window" window + "windowPretty" (window-hint window) + "emitted" (.get_emitted comm-s) + "transferred" (.get_transferred comm-s) + "acked" (.get_acked comm-s) + "failed" (.get_failed comm-s) + "completeLatency" (float-str (.get_complete_latency_ms spout-s))})) + +(defn- unpack-bolt-input-stat + [[^GlobalStreamId s ^ComponentAggregateStats stats]] + (let [^SpecificAggregateStats sas (.get_specific_stats stats) + ^BoltAggregateStats bas (.get_bolt sas) + ^CommonAggregateStats cas (.get_common_stats stats) + comp-id (.get_componentId s)] + {"component" comp-id + "encodedComponentId" (url-encode comp-id) + "stream" (.get_streamId s) + "executeLatency" (float-str (.get_execute_latency_ms bas)) + "processLatency" (float-str (.get_process_latency_ms bas)) + "executed" (nil-to-zero (.get_executed bas)) + "acked" (nil-to-zero (.get_acked cas)) + "failed" (nil-to-zero (.get_failed cas))})) + +(defmulti unpack-comp-output-stat + (fn [[_ ^ComponentAggregateStats s]] (.get_type s))) + +(defmethod unpack-comp-output-stat ComponentType/BOLT + [[stream-id ^ComponentAggregateStats stats]] + (let [^CommonAggregateStats cas (.get_common_stats stats)] + {"stream" stream-id + "emitted" (nil-to-zero (.get_emitted cas)) + "transferred" (nil-to-zero (.get_transferred cas))})) + +(defmethod unpack-comp-output-stat ComponentType/SPOUT + [[stream-id ^ComponentAggregateStats stats]] + (let [^CommonAggregateStats cas (.get_common_stats stats) + ^SpecificAggregateStats spec-s (.get_specific_stats stats) + ^SpoutAggregateStats spout-s (.get_spout spec-s)] + {"stream" stream-id + "emitted" (nil-to-zero (.get_emitted cas)) + "transferred" (nil-to-zero (.get_transferred cas)) + "completeLatency" (float-str (.get_complete_latency_ms spout-s)) + "acked" (nil-to-zero (.get_acked cas)) + "failed" (nil-to-zero (.get_failed cas))})) + +(defmulti unpack-comp-exec-stat + (fn [_ _ ^ComponentAggregateStats cas] (.get_type (.get_stats ^ExecutorAggregateStats cas)))) + +(defmethod unpack-comp-exec-stat ComponentType/BOLT + [topology-id secure? ^ExecutorAggregateStats eas] + (let [^ExecutorSummary summ (.get_exec_summary eas) + ^ExecutorInfo info (.get_executor_info summ) + ^ComponentAggregateStats stats (.get_stats eas) + ^SpecificAggregateStats ss (.get_specific_stats stats) + ^BoltAggregateStats bas (.get_bolt ss) + ^CommonAggregateStats cas (.get_common_stats stats) + host (.get_host summ) + port (.get_port summ) + exec-id (pretty-executor-info info) + uptime (.get_uptime_secs summ)] + {"id" exec-id + "encodedId" (url-encode exec-id) + "uptime" (pretty-uptime-sec uptime) + "uptimeSeconds" uptime + "host" host + "port" port + "emitted" (nil-to-zero (.get_emitted cas)) + "transferred" (nil-to-zero (.get_transferred cas)) + "capacity" (float-str (nil-to-zero (.get_capacity bas))) + "executeLatency" (float-str (.get_execute_latency_ms bas)) + "executed" (nil-to-zero (.get_executed bas)) + "processLatency" (float-str (.get_process_latency_ms bas)) + "acked" (nil-to-zero (.get_acked cas)) + "failed" (nil-to-zero (.get_failed cas)) + "workerLogLink" (worker-log-link host port topology-id secure?)})) + +(defmethod unpack-comp-exec-stat ComponentType/SPOUT + [topology-id secure? ^ExecutorAggregateStats eas] + (let [^ExecutorSummary summ (.get_exec_summary eas) + ^ExecutorInfo info (.get_executor_info summ) + ^ComponentAggregateStats stats (.get_stats eas) + ^SpecificAggregateStats ss (.get_specific_stats stats) + ^SpoutAggregateStats sas (.get_spout ss) + ^CommonAggregateStats cas (.get_common_stats stats) + host (.get_host summ) + port (.get_port summ) + exec-id (pretty-executor-info info) + uptime (.get_uptime_secs summ)] + {"id" exec-id + "encodedId" (url-encode exec-id) + "uptime" (pretty-uptime-sec uptime) + "uptimeSeconds" uptime + "host" host + "port" port + "emitted" (nil-to-zero (.get_emitted cas)) + "transferred" (nil-to-zero (.get_transferred cas)) + "completeLatency" (float-str (.get_complete_latency_ms sas)) + "acked" (nil-to-zero (.get_acked cas)) + "failed" (nil-to-zero (.get_failed cas)) + "workerLogLink" (worker-log-link host port topology-id secure?)})) + +(defmulti unpack-component-page-info + "Unpacks component-specific info to clojure data structures" + (fn [^ComponentPageInfo info & _] + (.get_component_type info))) + +(defmethod unpack-component-page-info ComponentType/BOLT + [^ComponentPageInfo info topology-id window include-sys? secure?] + (merge + {"boltStats" (map unpack-comp-agg-stat (.get_window_to_stats info)) + "inputStats" (map unpack-bolt-input-stat (.get_gsid_to_input_stats info)) + "outputStats" (map unpack-comp-output-stat (.get_sid_to_output_stats info)) + "executorStats" (map (partial unpack-comp-exec-stat topology-id secure?) + (.get_exec_stats info))} + (-> info .get_errors (component-errors topology-id secure?)))) + +(defmethod unpack-component-page-info ComponentType/SPOUT + [^ComponentPageInfo info topology-id window include-sys? secure?] + (merge + {"spoutSummary" (map unpack-comp-agg-stat (.get_window_to_stats info)) + "outputStats" (map unpack-comp-output-stat (.get_sid_to_output_stats info)) + "executorStats" (map (partial unpack-comp-exec-stat topology-id secure?) + (.get_exec_stats info))} + (-> info .get_errors (component-errors topology-id secure?)))) (defn component-page [topology-id component window include-sys? user secure?] (thrift/with-configured-nimbus-connection nimbus - (let [window (if window window ":all-time") - summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id) - topology (.getTopology ^Nimbus$Client nimbus topology-id) - topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id)) - type (component-type topology component) - summs (component-task-summs summ topology component) - spec (cond (= type :spout) (spout-stats window summ component summs include-sys? secure?) - (= type :bolt) (bolt-stats window summ component summs include-sys? secure?)) - errors (component-errors (get (.get_errors summ) component) topology-id secure?) - component->debug (.get_component_debug summ) - debug-options (get component->debug component (get component->debug topology-id))] - (merge - {"user" user - "id" component - "encodedId" (url-encode component) - "name" (.get_name summ) - "executors" (count summs) - "tasks" (sum-tasks summs) - "topologyId" topology-id - "topologyStatus" (.get_status summ) - "encodedTopologyId" (url-encode topology-id) - "window" window - "componentType" (name type) - "windowHint" (window-hint window) - "debug" (if (not-nil? debug-options) (.is_enable debug-options) false) - "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 10) - "eventLogLink" (event-log-link topology-id summ topology component secure?)} - spec errors)))) + (let [window (or window ":all-time") + window-hint (window-hint window) + comp-page-info (.getComponentPageInfo ^Nimbus$Client nimbus + topology-id + component + window + include-sys?) + topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus + topology-id)) + msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS) + [debugEnabled + samplingPct] (if-let [debug-opts (.get_debug_options comp-page-info)] + [(.is_enable debug-opts) + (.get_samplingpct debug-opts)])] + (assoc + (unpack-component-page-info comp-page-info + topology-id + window + include-sys? + secure?) + "user" user + "id" component + "encodedId" (url-encode component) + "name" (.get_topology_name comp-page-info) + "executors" (.get_num_executors comp-page-info) + "tasks" (.get_num_tasks comp-page-info) + "topologyId" topology-id + "topologyStatus" (.get_topology_status comp-page-info) + "encodedTopologyId" (url-encode topology-id) + "window" window + "componentType" (-> comp-page-info .get_component_type str lower-case) + "windowHint" window-hint + "debug" (or debugEnabled false) + "samplingPct" (or samplingPct 10) + "eventLogLink" (event-log-link topology-id + component + (.get_eventlog_host comp-page-info) + (.get_eventlog_port comp-page-info) + secure?))))) (defn- level-to-dict [level] (if level @@ -1023,7 +787,7 @@ (defn topology-config [topology-id] (thrift/with-configured-nimbus-connection nimbus - (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id)))) + (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id)))) (defn topology-op-response [topology-id op] {"topologyOperation" op, diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj index 47e846f7670..3be33b4e7ed 100644 --- a/storm-core/src/clj/backtype/storm/ui/helpers.clj +++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj @@ -96,40 +96,10 @@ )] ]) -(defn float-str [n] - (if n - (format "%.3f" (float n)) - "0" - )) - -(defn swap-map-order [m] - (->> m - (map (fn [[k v]] - (into - {} - (for [[k2 v2] v] - [k2 {k v2}] - )) - )) - (apply merge-with merge) - )) - (defn url-format [fmt & args] (String/format fmt (to-array (map #(url-encode (str %)) args)))) -(defn to-tasks [^ExecutorInfo e] - (let [start (.get_task_start e) - end (.get_task_end e)] - (range start (inc end)) - )) - -(defn sum-tasks [executors] - (reduce + (->> executors - (map #(.get_executor_info ^ExecutorSummary %)) - (map to-tasks) - (map count)))) - (defn pretty-executor-info [^ExecutorInfo e] (str "[" (.get_task_start e) "-" (.get_task_end e) "]")) diff --git a/storm-core/src/jvm/backtype/storm/generated/Assignment.java b/storm-core/src/jvm/backtype/storm/generated/Assignment.java index f576c84c7a3..7555b543a18 100644 --- a/storm-core/src/jvm/backtype/storm/generated/Assignment.java +++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class Assignment implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment"); @@ -678,15 +678,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 2: // NODE_HOST if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map388 = iprot.readMapBegin(); - struct.node_host = new HashMap(2*_map388.size); - String _key389; - String _val390; - for (int _i391 = 0; _i391 < _map388.size; ++_i391) + org.apache.thrift.protocol.TMap _map504 = iprot.readMapBegin(); + struct.node_host = new HashMap(2*_map504.size); + String _key505; + String _val506; + for (int _i507 = 0; _i507 < _map504.size; ++_i507) { - _key389 = iprot.readString(); - _val390 = iprot.readString(); - struct.node_host.put(_key389, _val390); + _key505 = iprot.readString(); + _val506 = iprot.readString(); + struct.node_host.put(_key505, _val506); } iprot.readMapEnd(); } @@ -698,26 +698,26 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 3: // EXECUTOR_NODE_PORT if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map392 = iprot.readMapBegin(); - struct.executor_node_port = new HashMap,NodeInfo>(2*_map392.size); - List _key393; - NodeInfo _val394; - for (int _i395 = 0; _i395 < _map392.size; ++_i395) + org.apache.thrift.protocol.TMap _map508 = iprot.readMapBegin(); + struct.executor_node_port = new HashMap,NodeInfo>(2*_map508.size); + List _key509; + NodeInfo _val510; + for (int _i511 = 0; _i511 < _map508.size; ++_i511) { { - org.apache.thrift.protocol.TList _list396 = iprot.readListBegin(); - _key393 = new ArrayList(_list396.size); - long _elem397; - for (int _i398 = 0; _i398 < _list396.size; ++_i398) + org.apache.thrift.protocol.TList _list512 = iprot.readListBegin(); + _key509 = new ArrayList(_list512.size); + long _elem513; + for (int _i514 = 0; _i514 < _list512.size; ++_i514) { - _elem397 = iprot.readI64(); - _key393.add(_elem397); + _elem513 = iprot.readI64(); + _key509.add(_elem513); } iprot.readListEnd(); } - _val394 = new NodeInfo(); - _val394.read(iprot); - struct.executor_node_port.put(_key393, _val394); + _val510 = new NodeInfo(); + _val510.read(iprot); + struct.executor_node_port.put(_key509, _val510); } iprot.readMapEnd(); } @@ -729,25 +729,25 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Assignment struct) case 4: // EXECUTOR_START_TIME_SECS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map399 = iprot.readMapBegin(); - struct.executor_start_time_secs = new HashMap,Long>(2*_map399.size); - List _key400; - long _val401; - for (int _i402 = 0; _i402 < _map399.size; ++_i402) + org.apache.thrift.protocol.TMap _map515 = iprot.readMapBegin(); + struct.executor_start_time_secs = new HashMap,Long>(2*_map515.size); + List _key516; + long _val517; + for (int _i518 = 0; _i518 < _map515.size; ++_i518) { { - org.apache.thrift.protocol.TList _list403 = iprot.readListBegin(); - _key400 = new ArrayList(_list403.size); - long _elem404; - for (int _i405 = 0; _i405 < _list403.size; ++_i405) + org.apache.thrift.protocol.TList _list519 = iprot.readListBegin(); + _key516 = new ArrayList(_list519.size); + long _elem520; + for (int _i521 = 0; _i521 < _list519.size; ++_i521) { - _elem404 = iprot.readI64(); - _key400.add(_elem404); + _elem520 = iprot.readI64(); + _key516.add(_elem520); } iprot.readListEnd(); } - _val401 = iprot.readI64(); - struct.executor_start_time_secs.put(_key400, _val401); + _val517 = iprot.readI64(); + struct.executor_start_time_secs.put(_key516, _val517); } iprot.readMapEnd(); } @@ -779,10 +779,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) oprot.writeFieldBegin(NODE_HOST_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.node_host.size())); - for (Map.Entry _iter406 : struct.node_host.entrySet()) + for (Map.Entry _iter522 : struct.node_host.entrySet()) { - oprot.writeString(_iter406.getKey()); - oprot.writeString(_iter406.getValue()); + oprot.writeString(_iter522.getKey()); + oprot.writeString(_iter522.getValue()); } oprot.writeMapEnd(); } @@ -794,17 +794,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, struct.executor_node_port.size())); - for (Map.Entry, NodeInfo> _iter407 : struct.executor_node_port.entrySet()) + for (Map.Entry, NodeInfo> _iter523 : struct.executor_node_port.entrySet()) { { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter407.getKey().size())); - for (long _iter408 : _iter407.getKey()) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter523.getKey().size())); + for (long _iter524 : _iter523.getKey()) { - oprot.writeI64(_iter408); + oprot.writeI64(_iter524); } oprot.writeListEnd(); } - _iter407.getValue().write(oprot); + _iter523.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -816,17 +816,17 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Assignment struct) oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, struct.executor_start_time_secs.size())); - for (Map.Entry, Long> _iter409 : struct.executor_start_time_secs.entrySet()) + for (Map.Entry, Long> _iter525 : struct.executor_start_time_secs.entrySet()) { { - oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter409.getKey().size())); - for (long _iter410 : _iter409.getKey()) + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter525.getKey().size())); + for (long _iter526 : _iter525.getKey()) { - oprot.writeI64(_iter410); + oprot.writeI64(_iter526); } oprot.writeListEnd(); } - oprot.writeI64(_iter409.getValue()); + oprot.writeI64(_iter525.getValue()); } oprot.writeMapEnd(); } @@ -865,42 +865,42 @@ public void write(org.apache.thrift.protocol.TProtocol prot, Assignment struct) if (struct.is_set_node_host()) { { oprot.writeI32(struct.node_host.size()); - for (Map.Entry _iter411 : struct.node_host.entrySet()) + for (Map.Entry _iter527 : struct.node_host.entrySet()) { - oprot.writeString(_iter411.getKey()); - oprot.writeString(_iter411.getValue()); + oprot.writeString(_iter527.getKey()); + oprot.writeString(_iter527.getValue()); } } } if (struct.is_set_executor_node_port()) { { oprot.writeI32(struct.executor_node_port.size()); - for (Map.Entry, NodeInfo> _iter412 : struct.executor_node_port.entrySet()) + for (Map.Entry, NodeInfo> _iter528 : struct.executor_node_port.entrySet()) { { - oprot.writeI32(_iter412.getKey().size()); - for (long _iter413 : _iter412.getKey()) + oprot.writeI32(_iter528.getKey().size()); + for (long _iter529 : _iter528.getKey()) { - oprot.writeI64(_iter413); + oprot.writeI64(_iter529); } } - _iter412.getValue().write(oprot); + _iter528.getValue().write(oprot); } } } if (struct.is_set_executor_start_time_secs()) { { oprot.writeI32(struct.executor_start_time_secs.size()); - for (Map.Entry, Long> _iter414 : struct.executor_start_time_secs.entrySet()) + for (Map.Entry, Long> _iter530 : struct.executor_start_time_secs.entrySet()) { { - oprot.writeI32(_iter414.getKey().size()); - for (long _iter415 : _iter414.getKey()) + oprot.writeI32(_iter530.getKey().size()); + for (long _iter531 : _iter530.getKey()) { - oprot.writeI64(_iter415); + oprot.writeI64(_iter531); } } - oprot.writeI64(_iter414.getValue()); + oprot.writeI64(_iter530.getValue()); } } } @@ -914,64 +914,64 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Assignment struct) t BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map416 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.node_host = new HashMap(2*_map416.size); - String _key417; - String _val418; - for (int _i419 = 0; _i419 < _map416.size; ++_i419) + org.apache.thrift.protocol.TMap _map532 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.node_host = new HashMap(2*_map532.size); + String _key533; + String _val534; + for (int _i535 = 0; _i535 < _map532.size; ++_i535) { - _key417 = iprot.readString(); - _val418 = iprot.readString(); - struct.node_host.put(_key417, _val418); + _key533 = iprot.readString(); + _val534 = iprot.readString(); + struct.node_host.put(_key533, _val534); } } struct.set_node_host_isSet(true); } if (incoming.get(1)) { { - org.apache.thrift.protocol.TMap _map420 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.executor_node_port = new HashMap,NodeInfo>(2*_map420.size); - List _key421; - NodeInfo _val422; - for (int _i423 = 0; _i423 < _map420.size; ++_i423) + org.apache.thrift.protocol.TMap _map536 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executor_node_port = new HashMap,NodeInfo>(2*_map536.size); + List _key537; + NodeInfo _val538; + for (int _i539 = 0; _i539 < _map536.size; ++_i539) { { - org.apache.thrift.protocol.TList _list424 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - _key421 = new ArrayList(_list424.size); - long _elem425; - for (int _i426 = 0; _i426 < _list424.size; ++_i426) + org.apache.thrift.protocol.TList _list540 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _key537 = new ArrayList(_list540.size); + long _elem541; + for (int _i542 = 0; _i542 < _list540.size; ++_i542) { - _elem425 = iprot.readI64(); - _key421.add(_elem425); + _elem541 = iprot.readI64(); + _key537.add(_elem541); } } - _val422 = new NodeInfo(); - _val422.read(iprot); - struct.executor_node_port.put(_key421, _val422); + _val538 = new NodeInfo(); + _val538.read(iprot); + struct.executor_node_port.put(_key537, _val538); } } struct.set_executor_node_port_isSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map427 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.executor_start_time_secs = new HashMap,Long>(2*_map427.size); - List _key428; - long _val429; - for (int _i430 = 0; _i430 < _map427.size; ++_i430) + org.apache.thrift.protocol.TMap _map543 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.executor_start_time_secs = new HashMap,Long>(2*_map543.size); + List _key544; + long _val545; + for (int _i546 = 0; _i546 < _map543.size; ++_i546) { { - org.apache.thrift.protocol.TList _list431 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - _key428 = new ArrayList(_list431.size); - long _elem432; - for (int _i433 = 0; _i433 < _list431.size; ++_i433) + org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + _key544 = new ArrayList(_list547.size); + long _elem548; + for (int _i549 = 0; _i549 < _list547.size; ++_i549) { - _elem432 = iprot.readI64(); - _key428.add(_elem432); + _elem548 = iprot.readI64(); + _key544.add(_elem548); } } - _val429 = iprot.readI64(); - struct.executor_start_time_secs.put(_key428, _val429); + _val545 = iprot.readI64(); + struct.executor_start_time_secs.put(_key544, _val545); } } struct.set_executor_start_time_secs_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java new file mode 100644 index 00000000000..3234d2bac88 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java @@ -0,0 +1,704 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22") +public class BoltAggregateStats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltAggregateStats"); + + private static final org.apache.thrift.protocol.TField EXECUTE_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("execute_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)1); + private static final org.apache.thrift.protocol.TField PROCESS_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("process_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)2); + private static final org.apache.thrift.protocol.TField EXECUTED_FIELD_DESC = new org.apache.thrift.protocol.TField("executed", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField CAPACITY_FIELD_DESC = new org.apache.thrift.protocol.TField("capacity", org.apache.thrift.protocol.TType.DOUBLE, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new BoltAggregateStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new BoltAggregateStatsTupleSchemeFactory()); + } + + private double execute_latency_ms; // optional + private double process_latency_ms; // optional + private long executed; // optional + private double capacity; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + EXECUTE_LATENCY_MS((short)1, "execute_latency_ms"), + PROCESS_LATENCY_MS((short)2, "process_latency_ms"), + EXECUTED((short)3, "executed"), + CAPACITY((short)4, "capacity"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // EXECUTE_LATENCY_MS + return EXECUTE_LATENCY_MS; + case 2: // PROCESS_LATENCY_MS + return PROCESS_LATENCY_MS; + case 3: // EXECUTED + return EXECUTED; + case 4: // CAPACITY + return CAPACITY; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __EXECUTE_LATENCY_MS_ISSET_ID = 0; + private static final int __PROCESS_LATENCY_MS_ISSET_ID = 1; + private static final int __EXECUTED_ISSET_ID = 2; + private static final int __CAPACITY_ISSET_ID = 3; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.EXECUTE_LATENCY_MS,_Fields.PROCESS_LATENCY_MS,_Fields.EXECUTED,_Fields.CAPACITY}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.EXECUTE_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("execute_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.PROCESS_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("process_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.EXECUTED, new org.apache.thrift.meta_data.FieldMetaData("executed", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.CAPACITY, new org.apache.thrift.meta_data.FieldMetaData("capacity", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(BoltAggregateStats.class, metaDataMap); + } + + public BoltAggregateStats() { + } + + /** + * Performs a deep copy on other. + */ + public BoltAggregateStats(BoltAggregateStats other) { + __isset_bitfield = other.__isset_bitfield; + this.execute_latency_ms = other.execute_latency_ms; + this.process_latency_ms = other.process_latency_ms; + this.executed = other.executed; + this.capacity = other.capacity; + } + + public BoltAggregateStats deepCopy() { + return new BoltAggregateStats(this); + } + + @Override + public void clear() { + set_execute_latency_ms_isSet(false); + this.execute_latency_ms = 0.0; + set_process_latency_ms_isSet(false); + this.process_latency_ms = 0.0; + set_executed_isSet(false); + this.executed = 0; + set_capacity_isSet(false); + this.capacity = 0.0; + } + + public double get_execute_latency_ms() { + return this.execute_latency_ms; + } + + public void set_execute_latency_ms(double execute_latency_ms) { + this.execute_latency_ms = execute_latency_ms; + set_execute_latency_ms_isSet(true); + } + + public void unset_execute_latency_ms() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID); + } + + /** Returns true if field execute_latency_ms is set (has been assigned a value) and false otherwise */ + public boolean is_set_execute_latency_ms() { + return EncodingUtils.testBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID); + } + + public void set_execute_latency_ms_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID, value); + } + + public double get_process_latency_ms() { + return this.process_latency_ms; + } + + public void set_process_latency_ms(double process_latency_ms) { + this.process_latency_ms = process_latency_ms; + set_process_latency_ms_isSet(true); + } + + public void unset_process_latency_ms() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID); + } + + /** Returns true if field process_latency_ms is set (has been assigned a value) and false otherwise */ + public boolean is_set_process_latency_ms() { + return EncodingUtils.testBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID); + } + + public void set_process_latency_ms_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID, value); + } + + public long get_executed() { + return this.executed; + } + + public void set_executed(long executed) { + this.executed = executed; + set_executed_isSet(true); + } + + public void unset_executed() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EXECUTED_ISSET_ID); + } + + /** Returns true if field executed is set (has been assigned a value) and false otherwise */ + public boolean is_set_executed() { + return EncodingUtils.testBit(__isset_bitfield, __EXECUTED_ISSET_ID); + } + + public void set_executed_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXECUTED_ISSET_ID, value); + } + + public double get_capacity() { + return this.capacity; + } + + public void set_capacity(double capacity) { + this.capacity = capacity; + set_capacity_isSet(true); + } + + public void unset_capacity() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CAPACITY_ISSET_ID); + } + + /** Returns true if field capacity is set (has been assigned a value) and false otherwise */ + public boolean is_set_capacity() { + return EncodingUtils.testBit(__isset_bitfield, __CAPACITY_ISSET_ID); + } + + public void set_capacity_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CAPACITY_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EXECUTE_LATENCY_MS: + if (value == null) { + unset_execute_latency_ms(); + } else { + set_execute_latency_ms((Double)value); + } + break; + + case PROCESS_LATENCY_MS: + if (value == null) { + unset_process_latency_ms(); + } else { + set_process_latency_ms((Double)value); + } + break; + + case EXECUTED: + if (value == null) { + unset_executed(); + } else { + set_executed((Long)value); + } + break; + + case CAPACITY: + if (value == null) { + unset_capacity(); + } else { + set_capacity((Double)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EXECUTE_LATENCY_MS: + return Double.valueOf(get_execute_latency_ms()); + + case PROCESS_LATENCY_MS: + return Double.valueOf(get_process_latency_ms()); + + case EXECUTED: + return Long.valueOf(get_executed()); + + case CAPACITY: + return Double.valueOf(get_capacity()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case EXECUTE_LATENCY_MS: + return is_set_execute_latency_ms(); + case PROCESS_LATENCY_MS: + return is_set_process_latency_ms(); + case EXECUTED: + return is_set_executed(); + case CAPACITY: + return is_set_capacity(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof BoltAggregateStats) + return this.equals((BoltAggregateStats)that); + return false; + } + + public boolean equals(BoltAggregateStats that) { + if (that == null) + return false; + + boolean this_present_execute_latency_ms = true && this.is_set_execute_latency_ms(); + boolean that_present_execute_latency_ms = true && that.is_set_execute_latency_ms(); + if (this_present_execute_latency_ms || that_present_execute_latency_ms) { + if (!(this_present_execute_latency_ms && that_present_execute_latency_ms)) + return false; + if (this.execute_latency_ms != that.execute_latency_ms) + return false; + } + + boolean this_present_process_latency_ms = true && this.is_set_process_latency_ms(); + boolean that_present_process_latency_ms = true && that.is_set_process_latency_ms(); + if (this_present_process_latency_ms || that_present_process_latency_ms) { + if (!(this_present_process_latency_ms && that_present_process_latency_ms)) + return false; + if (this.process_latency_ms != that.process_latency_ms) + return false; + } + + boolean this_present_executed = true && this.is_set_executed(); + boolean that_present_executed = true && that.is_set_executed(); + if (this_present_executed || that_present_executed) { + if (!(this_present_executed && that_present_executed)) + return false; + if (this.executed != that.executed) + return false; + } + + boolean this_present_capacity = true && this.is_set_capacity(); + boolean that_present_capacity = true && that.is_set_capacity(); + if (this_present_capacity || that_present_capacity) { + if (!(this_present_capacity && that_present_capacity)) + return false; + if (this.capacity != that.capacity) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_execute_latency_ms = true && (is_set_execute_latency_ms()); + list.add(present_execute_latency_ms); + if (present_execute_latency_ms) + list.add(execute_latency_ms); + + boolean present_process_latency_ms = true && (is_set_process_latency_ms()); + list.add(present_process_latency_ms); + if (present_process_latency_ms) + list.add(process_latency_ms); + + boolean present_executed = true && (is_set_executed()); + list.add(present_executed); + if (present_executed) + list.add(executed); + + boolean present_capacity = true && (is_set_capacity()); + list.add(present_capacity); + if (present_capacity) + list.add(capacity); + + return list.hashCode(); + } + + @Override + public int compareTo(BoltAggregateStats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_execute_latency_ms()).compareTo(other.is_set_execute_latency_ms()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_execute_latency_ms()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.execute_latency_ms, other.execute_latency_ms); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_process_latency_ms()).compareTo(other.is_set_process_latency_ms()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_process_latency_ms()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.process_latency_ms, other.process_latency_ms); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_executed()).compareTo(other.is_set_executed()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_executed()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executed, other.executed); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_capacity()).compareTo(other.is_set_capacity()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_capacity()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.capacity, other.capacity); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("BoltAggregateStats("); + boolean first = true; + + if (is_set_execute_latency_ms()) { + sb.append("execute_latency_ms:"); + sb.append(this.execute_latency_ms); + first = false; + } + if (is_set_process_latency_ms()) { + if (!first) sb.append(", "); + sb.append("process_latency_ms:"); + sb.append(this.process_latency_ms); + first = false; + } + if (is_set_executed()) { + if (!first) sb.append(", "); + sb.append("executed:"); + sb.append(this.executed); + first = false; + } + if (is_set_capacity()) { + if (!first) sb.append(", "); + sb.append("capacity:"); + sb.append(this.capacity); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class BoltAggregateStatsStandardSchemeFactory implements SchemeFactory { + public BoltAggregateStatsStandardScheme getScheme() { + return new BoltAggregateStatsStandardScheme(); + } + } + + private static class BoltAggregateStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, BoltAggregateStats struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // EXECUTE_LATENCY_MS + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.execute_latency_ms = iprot.readDouble(); + struct.set_execute_latency_ms_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // PROCESS_LATENCY_MS + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.process_latency_ms = iprot.readDouble(); + struct.set_process_latency_ms_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // EXECUTED + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.executed = iprot.readI64(); + struct.set_executed_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // CAPACITY + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.capacity = iprot.readDouble(); + struct.set_capacity_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, BoltAggregateStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.is_set_execute_latency_ms()) { + oprot.writeFieldBegin(EXECUTE_LATENCY_MS_FIELD_DESC); + oprot.writeDouble(struct.execute_latency_ms); + oprot.writeFieldEnd(); + } + if (struct.is_set_process_latency_ms()) { + oprot.writeFieldBegin(PROCESS_LATENCY_MS_FIELD_DESC); + oprot.writeDouble(struct.process_latency_ms); + oprot.writeFieldEnd(); + } + if (struct.is_set_executed()) { + oprot.writeFieldBegin(EXECUTED_FIELD_DESC); + oprot.writeI64(struct.executed); + oprot.writeFieldEnd(); + } + if (struct.is_set_capacity()) { + oprot.writeFieldBegin(CAPACITY_FIELD_DESC); + oprot.writeDouble(struct.capacity); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class BoltAggregateStatsTupleSchemeFactory implements SchemeFactory { + public BoltAggregateStatsTupleScheme getScheme() { + return new BoltAggregateStatsTupleScheme(); + } + } + + private static class BoltAggregateStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, BoltAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_execute_latency_ms()) { + optionals.set(0); + } + if (struct.is_set_process_latency_ms()) { + optionals.set(1); + } + if (struct.is_set_executed()) { + optionals.set(2); + } + if (struct.is_set_capacity()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.is_set_execute_latency_ms()) { + oprot.writeDouble(struct.execute_latency_ms); + } + if (struct.is_set_process_latency_ms()) { + oprot.writeDouble(struct.process_latency_ms); + } + if (struct.is_set_executed()) { + oprot.writeI64(struct.executed); + } + if (struct.is_set_capacity()) { + oprot.writeDouble(struct.capacity); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, BoltAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.execute_latency_ms = iprot.readDouble(); + struct.set_execute_latency_ms_isSet(true); + } + if (incoming.get(1)) { + struct.process_latency_ms = iprot.readDouble(); + struct.set_process_latency_ms_isSet(true); + } + if (incoming.get(2)) { + struct.executed = iprot.readI64(); + struct.set_executed_isSet(true); + } + if (incoming.get(3)) { + struct.capacity = iprot.readDouble(); + struct.set_capacity_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java index ce2f5f43dfd..8c74ca2d1e6 100644 --- a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java +++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterWorkerHeartbeat"); @@ -635,17 +635,17 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, ClusterWorkerHeartb case 2: // EXECUTOR_STATS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map454 = iprot.readMapBegin(); - struct.executor_stats = new HashMap(2*_map454.size); - ExecutorInfo _key455; - ExecutorStats _val456; - for (int _i457 = 0; _i457 < _map454.size; ++_i457) + org.apache.thrift.protocol.TMap _map570 = iprot.readMapBegin(); + struct.executor_stats = new HashMap(2*_map570.size); + ExecutorInfo _key571; + ExecutorStats _val572; + for (int _i573 = 0; _i573 < _map570.size; ++_i573) { - _key455 = new ExecutorInfo(); - _key455.read(iprot); - _val456 = new ExecutorStats(); - _val456.read(iprot); - struct.executor_stats.put(_key455, _val456); + _key571 = new ExecutorInfo(); + _key571.read(iprot); + _val572 = new ExecutorStats(); + _val572.read(iprot); + struct.executor_stats.put(_key571, _val572); } iprot.readMapEnd(); } @@ -692,10 +692,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, ClusterWorkerHeart oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size())); - for (Map.Entry _iter458 : struct.executor_stats.entrySet()) + for (Map.Entry _iter574 : struct.executor_stats.entrySet()) { - _iter458.getKey().write(oprot); - _iter458.getValue().write(oprot); + _iter574.getKey().write(oprot); + _iter574.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -727,10 +727,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartb oprot.writeString(struct.storm_id); { oprot.writeI32(struct.executor_stats.size()); - for (Map.Entry _iter459 : struct.executor_stats.entrySet()) + for (Map.Entry _iter575 : struct.executor_stats.entrySet()) { - _iter459.getKey().write(oprot); - _iter459.getValue().write(oprot); + _iter575.getKey().write(oprot); + _iter575.getValue().write(oprot); } } oprot.writeI32(struct.time_secs); @@ -743,17 +743,17 @@ public void read(org.apache.thrift.protocol.TProtocol prot, ClusterWorkerHeartbe struct.storm_id = iprot.readString(); struct.set_storm_id_isSet(true); { - org.apache.thrift.protocol.TMap _map460 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.executor_stats = new HashMap(2*_map460.size); - ExecutorInfo _key461; - ExecutorStats _val462; - for (int _i463 = 0; _i463 < _map460.size; ++_i463) + org.apache.thrift.protocol.TMap _map576 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executor_stats = new HashMap(2*_map576.size); + ExecutorInfo _key577; + ExecutorStats _val578; + for (int _i579 = 0; _i579 < _map576.size; ++_i579) { - _key461 = new ExecutorInfo(); - _key461.read(iprot); - _val462 = new ExecutorStats(); - _val462.read(iprot); - struct.executor_stats.put(_key461, _val462); + _key577 = new ExecutorInfo(); + _key577.read(iprot); + _val578 = new ExecutorStats(); + _val578.read(iprot); + struct.executor_stats.put(_key577, _val578); } } struct.set_executor_stats_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java new file mode 100644 index 00000000000..0eea7c2f712 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java @@ -0,0 +1,902 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22") +public class CommonAggregateStats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CommonAggregateStats"); + + private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("emitted", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("transferred", org.apache.thrift.protocol.TType.I64, (short)4); + private static final org.apache.thrift.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("acked", org.apache.thrift.protocol.TType.I64, (short)5); + private static final org.apache.thrift.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("failed", org.apache.thrift.protocol.TType.I64, (short)6); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CommonAggregateStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CommonAggregateStatsTupleSchemeFactory()); + } + + private int num_executors; // optional + private int num_tasks; // optional + private long emitted; // optional + private long transferred; // optional + private long acked; // optional + private long failed; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NUM_EXECUTORS((short)1, "num_executors"), + NUM_TASKS((short)2, "num_tasks"), + EMITTED((short)3, "emitted"), + TRANSFERRED((short)4, "transferred"), + ACKED((short)5, "acked"), + FAILED((short)6, "failed"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NUM_EXECUTORS + return NUM_EXECUTORS; + case 2: // NUM_TASKS + return NUM_TASKS; + case 3: // EMITTED + return EMITTED; + case 4: // TRANSFERRED + return TRANSFERRED; + case 5: // ACKED + return ACKED; + case 6: // FAILED + return FAILED; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __NUM_EXECUTORS_ISSET_ID = 0; + private static final int __NUM_TASKS_ISSET_ID = 1; + private static final int __EMITTED_ISSET_ID = 2; + private static final int __TRANSFERRED_ISSET_ID = 3; + private static final int __ACKED_ISSET_ID = 4; + private static final int __FAILED_ISSET_ID = 5; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.NUM_EXECUTORS,_Fields.NUM_TASKS,_Fields.EMITTED,_Fields.TRANSFERRED,_Fields.ACKED,_Fields.FAILED}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.EMITTED, new org.apache.thrift.meta_data.FieldMetaData("emitted", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("transferred", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.ACKED, new org.apache.thrift.meta_data.FieldMetaData("acked", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.FAILED, new org.apache.thrift.meta_data.FieldMetaData("failed", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CommonAggregateStats.class, metaDataMap); + } + + public CommonAggregateStats() { + } + + /** + * Performs a deep copy on other. + */ + public CommonAggregateStats(CommonAggregateStats other) { + __isset_bitfield = other.__isset_bitfield; + this.num_executors = other.num_executors; + this.num_tasks = other.num_tasks; + this.emitted = other.emitted; + this.transferred = other.transferred; + this.acked = other.acked; + this.failed = other.failed; + } + + public CommonAggregateStats deepCopy() { + return new CommonAggregateStats(this); + } + + @Override + public void clear() { + set_num_executors_isSet(false); + this.num_executors = 0; + set_num_tasks_isSet(false); + this.num_tasks = 0; + set_emitted_isSet(false); + this.emitted = 0; + set_transferred_isSet(false); + this.transferred = 0; + set_acked_isSet(false); + this.acked = 0; + set_failed_isSet(false); + this.failed = 0; + } + + public int get_num_executors() { + return this.num_executors; + } + + public void set_num_executors(int num_executors) { + this.num_executors = num_executors; + set_num_executors_isSet(true); + } + + public void unset_num_executors() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_executors() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + public void set_num_executors_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value); + } + + public int get_num_tasks() { + return this.num_tasks; + } + + public void set_num_tasks(int num_tasks) { + this.num_tasks = num_tasks; + set_num_tasks_isSet(true); + } + + public void unset_num_tasks() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_tasks() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + public void set_num_tasks_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value); + } + + public long get_emitted() { + return this.emitted; + } + + public void set_emitted(long emitted) { + this.emitted = emitted; + set_emitted_isSet(true); + } + + public void unset_emitted() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EMITTED_ISSET_ID); + } + + /** Returns true if field emitted is set (has been assigned a value) and false otherwise */ + public boolean is_set_emitted() { + return EncodingUtils.testBit(__isset_bitfield, __EMITTED_ISSET_ID); + } + + public void set_emitted_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EMITTED_ISSET_ID, value); + } + + public long get_transferred() { + return this.transferred; + } + + public void set_transferred(long transferred) { + this.transferred = transferred; + set_transferred_isSet(true); + } + + public void unset_transferred() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TRANSFERRED_ISSET_ID); + } + + /** Returns true if field transferred is set (has been assigned a value) and false otherwise */ + public boolean is_set_transferred() { + return EncodingUtils.testBit(__isset_bitfield, __TRANSFERRED_ISSET_ID); + } + + public void set_transferred_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TRANSFERRED_ISSET_ID, value); + } + + public long get_acked() { + return this.acked; + } + + public void set_acked(long acked) { + this.acked = acked; + set_acked_isSet(true); + } + + public void unset_acked() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ACKED_ISSET_ID); + } + + /** Returns true if field acked is set (has been assigned a value) and false otherwise */ + public boolean is_set_acked() { + return EncodingUtils.testBit(__isset_bitfield, __ACKED_ISSET_ID); + } + + public void set_acked_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ACKED_ISSET_ID, value); + } + + public long get_failed() { + return this.failed; + } + + public void set_failed(long failed) { + this.failed = failed; + set_failed_isSet(true); + } + + public void unset_failed() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FAILED_ISSET_ID); + } + + /** Returns true if field failed is set (has been assigned a value) and false otherwise */ + public boolean is_set_failed() { + return EncodingUtils.testBit(__isset_bitfield, __FAILED_ISSET_ID); + } + + public void set_failed_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FAILED_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NUM_EXECUTORS: + if (value == null) { + unset_num_executors(); + } else { + set_num_executors((Integer)value); + } + break; + + case NUM_TASKS: + if (value == null) { + unset_num_tasks(); + } else { + set_num_tasks((Integer)value); + } + break; + + case EMITTED: + if (value == null) { + unset_emitted(); + } else { + set_emitted((Long)value); + } + break; + + case TRANSFERRED: + if (value == null) { + unset_transferred(); + } else { + set_transferred((Long)value); + } + break; + + case ACKED: + if (value == null) { + unset_acked(); + } else { + set_acked((Long)value); + } + break; + + case FAILED: + if (value == null) { + unset_failed(); + } else { + set_failed((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NUM_EXECUTORS: + return Integer.valueOf(get_num_executors()); + + case NUM_TASKS: + return Integer.valueOf(get_num_tasks()); + + case EMITTED: + return Long.valueOf(get_emitted()); + + case TRANSFERRED: + return Long.valueOf(get_transferred()); + + case ACKED: + return Long.valueOf(get_acked()); + + case FAILED: + return Long.valueOf(get_failed()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NUM_EXECUTORS: + return is_set_num_executors(); + case NUM_TASKS: + return is_set_num_tasks(); + case EMITTED: + return is_set_emitted(); + case TRANSFERRED: + return is_set_transferred(); + case ACKED: + return is_set_acked(); + case FAILED: + return is_set_failed(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CommonAggregateStats) + return this.equals((CommonAggregateStats)that); + return false; + } + + public boolean equals(CommonAggregateStats that) { + if (that == null) + return false; + + boolean this_present_num_executors = true && this.is_set_num_executors(); + boolean that_present_num_executors = true && that.is_set_num_executors(); + if (this_present_num_executors || that_present_num_executors) { + if (!(this_present_num_executors && that_present_num_executors)) + return false; + if (this.num_executors != that.num_executors) + return false; + } + + boolean this_present_num_tasks = true && this.is_set_num_tasks(); + boolean that_present_num_tasks = true && that.is_set_num_tasks(); + if (this_present_num_tasks || that_present_num_tasks) { + if (!(this_present_num_tasks && that_present_num_tasks)) + return false; + if (this.num_tasks != that.num_tasks) + return false; + } + + boolean this_present_emitted = true && this.is_set_emitted(); + boolean that_present_emitted = true && that.is_set_emitted(); + if (this_present_emitted || that_present_emitted) { + if (!(this_present_emitted && that_present_emitted)) + return false; + if (this.emitted != that.emitted) + return false; + } + + boolean this_present_transferred = true && this.is_set_transferred(); + boolean that_present_transferred = true && that.is_set_transferred(); + if (this_present_transferred || that_present_transferred) { + if (!(this_present_transferred && that_present_transferred)) + return false; + if (this.transferred != that.transferred) + return false; + } + + boolean this_present_acked = true && this.is_set_acked(); + boolean that_present_acked = true && that.is_set_acked(); + if (this_present_acked || that_present_acked) { + if (!(this_present_acked && that_present_acked)) + return false; + if (this.acked != that.acked) + return false; + } + + boolean this_present_failed = true && this.is_set_failed(); + boolean that_present_failed = true && that.is_set_failed(); + if (this_present_failed || that_present_failed) { + if (!(this_present_failed && that_present_failed)) + return false; + if (this.failed != that.failed) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_num_executors = true && (is_set_num_executors()); + list.add(present_num_executors); + if (present_num_executors) + list.add(num_executors); + + boolean present_num_tasks = true && (is_set_num_tasks()); + list.add(present_num_tasks); + if (present_num_tasks) + list.add(num_tasks); + + boolean present_emitted = true && (is_set_emitted()); + list.add(present_emitted); + if (present_emitted) + list.add(emitted); + + boolean present_transferred = true && (is_set_transferred()); + list.add(present_transferred); + if (present_transferred) + list.add(transferred); + + boolean present_acked = true && (is_set_acked()); + list.add(present_acked); + if (present_acked) + list.add(acked); + + boolean present_failed = true && (is_set_failed()); + list.add(present_failed); + if (present_failed) + list.add(failed); + + return list.hashCode(); + } + + @Override + public int compareTo(CommonAggregateStats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_executors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_tasks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_emitted()).compareTo(other.is_set_emitted()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_emitted()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.emitted, other.emitted); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_transferred()).compareTo(other.is_set_transferred()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_transferred()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.transferred, other.transferred); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_acked()).compareTo(other.is_set_acked()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_acked()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acked, other.acked); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_failed()).compareTo(other.is_set_failed()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_failed()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.failed, other.failed); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CommonAggregateStats("); + boolean first = true; + + if (is_set_num_executors()) { + sb.append("num_executors:"); + sb.append(this.num_executors); + first = false; + } + if (is_set_num_tasks()) { + if (!first) sb.append(", "); + sb.append("num_tasks:"); + sb.append(this.num_tasks); + first = false; + } + if (is_set_emitted()) { + if (!first) sb.append(", "); + sb.append("emitted:"); + sb.append(this.emitted); + first = false; + } + if (is_set_transferred()) { + if (!first) sb.append(", "); + sb.append("transferred:"); + sb.append(this.transferred); + first = false; + } + if (is_set_acked()) { + if (!first) sb.append(", "); + sb.append("acked:"); + sb.append(this.acked); + first = false; + } + if (is_set_failed()) { + if (!first) sb.append(", "); + sb.append("failed:"); + sb.append(this.failed); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CommonAggregateStatsStandardSchemeFactory implements SchemeFactory { + public CommonAggregateStatsStandardScheme getScheme() { + return new CommonAggregateStatsStandardScheme(); + } + } + + private static class CommonAggregateStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CommonAggregateStats struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NUM_EXECUTORS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_executors = iprot.readI32(); + struct.set_num_executors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NUM_TASKS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // EMITTED + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.emitted = iprot.readI64(); + struct.set_emitted_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TRANSFERRED + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.transferred = iprot.readI64(); + struct.set_transferred_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // ACKED + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.acked = iprot.readI64(); + struct.set_acked_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // FAILED + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.failed = iprot.readI64(); + struct.set_failed_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CommonAggregateStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.is_set_num_executors()) { + oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); + oprot.writeI32(struct.num_executors); + oprot.writeFieldEnd(); + } + if (struct.is_set_num_tasks()) { + oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC); + oprot.writeI32(struct.num_tasks); + oprot.writeFieldEnd(); + } + if (struct.is_set_emitted()) { + oprot.writeFieldBegin(EMITTED_FIELD_DESC); + oprot.writeI64(struct.emitted); + oprot.writeFieldEnd(); + } + if (struct.is_set_transferred()) { + oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC); + oprot.writeI64(struct.transferred); + oprot.writeFieldEnd(); + } + if (struct.is_set_acked()) { + oprot.writeFieldBegin(ACKED_FIELD_DESC); + oprot.writeI64(struct.acked); + oprot.writeFieldEnd(); + } + if (struct.is_set_failed()) { + oprot.writeFieldBegin(FAILED_FIELD_DESC); + oprot.writeI64(struct.failed); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CommonAggregateStatsTupleSchemeFactory implements SchemeFactory { + public CommonAggregateStatsTupleScheme getScheme() { + return new CommonAggregateStatsTupleScheme(); + } + } + + private static class CommonAggregateStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CommonAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_num_executors()) { + optionals.set(0); + } + if (struct.is_set_num_tasks()) { + optionals.set(1); + } + if (struct.is_set_emitted()) { + optionals.set(2); + } + if (struct.is_set_transferred()) { + optionals.set(3); + } + if (struct.is_set_acked()) { + optionals.set(4); + } + if (struct.is_set_failed()) { + optionals.set(5); + } + oprot.writeBitSet(optionals, 6); + if (struct.is_set_num_executors()) { + oprot.writeI32(struct.num_executors); + } + if (struct.is_set_num_tasks()) { + oprot.writeI32(struct.num_tasks); + } + if (struct.is_set_emitted()) { + oprot.writeI64(struct.emitted); + } + if (struct.is_set_transferred()) { + oprot.writeI64(struct.transferred); + } + if (struct.is_set_acked()) { + oprot.writeI64(struct.acked); + } + if (struct.is_set_failed()) { + oprot.writeI64(struct.failed); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CommonAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(6); + if (incoming.get(0)) { + struct.num_executors = iprot.readI32(); + struct.set_num_executors_isSet(true); + } + if (incoming.get(1)) { + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + } + if (incoming.get(2)) { + struct.emitted = iprot.readI64(); + struct.set_emitted_isSet(true); + } + if (incoming.get(3)) { + struct.transferred = iprot.readI64(); + struct.set_transferred_isSet(true); + } + if (incoming.get(4)) { + struct.acked = iprot.readI64(); + struct.set_acked_isSet(true); + } + if (incoming.get(5)) { + struct.failed = iprot.readI64(); + struct.set_failed_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java new file mode 100644 index 00000000000..ae74b847bb8 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java @@ -0,0 +1,752 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22") +public class ComponentAggregateStats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentAggregateStats"); + + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField COMMON_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("common_stats", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField SPECIFIC_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("specific_stats", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField LAST_ERROR_FIELD_DESC = new org.apache.thrift.protocol.TField("last_error", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ComponentAggregateStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ComponentAggregateStatsTupleSchemeFactory()); + } + + private ComponentType type; // optional + private CommonAggregateStats common_stats; // optional + private SpecificAggregateStats specific_stats; // optional + private ErrorInfo last_error; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see ComponentType + */ + TYPE((short)1, "type"), + COMMON_STATS((short)2, "common_stats"), + SPECIFIC_STATS((short)3, "specific_stats"), + LAST_ERROR((short)4, "last_error"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPE + return TYPE; + case 2: // COMMON_STATS + return COMMON_STATS; + case 3: // SPECIFIC_STATS + return SPECIFIC_STATS; + case 4: // LAST_ERROR + return LAST_ERROR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.TYPE,_Fields.COMMON_STATS,_Fields.SPECIFIC_STATS,_Fields.LAST_ERROR}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ComponentType.class))); + tmpMap.put(_Fields.COMMON_STATS, new org.apache.thrift.meta_data.FieldMetaData("common_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CommonAggregateStats.class))); + tmpMap.put(_Fields.SPECIFIC_STATS, new org.apache.thrift.meta_data.FieldMetaData("specific_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpecificAggregateStats.class))); + tmpMap.put(_Fields.LAST_ERROR, new org.apache.thrift.meta_data.FieldMetaData("last_error", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentAggregateStats.class, metaDataMap); + } + + public ComponentAggregateStats() { + } + + /** + * Performs a deep copy on other. + */ + public ComponentAggregateStats(ComponentAggregateStats other) { + if (other.is_set_type()) { + this.type = other.type; + } + if (other.is_set_common_stats()) { + this.common_stats = new CommonAggregateStats(other.common_stats); + } + if (other.is_set_specific_stats()) { + this.specific_stats = new SpecificAggregateStats(other.specific_stats); + } + if (other.is_set_last_error()) { + this.last_error = new ErrorInfo(other.last_error); + } + } + + public ComponentAggregateStats deepCopy() { + return new ComponentAggregateStats(this); + } + + @Override + public void clear() { + this.type = null; + this.common_stats = null; + this.specific_stats = null; + this.last_error = null; + } + + /** + * + * @see ComponentType + */ + public ComponentType get_type() { + return this.type; + } + + /** + * + * @see ComponentType + */ + public void set_type(ComponentType type) { + this.type = type; + } + + public void unset_type() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean is_set_type() { + return this.type != null; + } + + public void set_type_isSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public CommonAggregateStats get_common_stats() { + return this.common_stats; + } + + public void set_common_stats(CommonAggregateStats common_stats) { + this.common_stats = common_stats; + } + + public void unset_common_stats() { + this.common_stats = null; + } + + /** Returns true if field common_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_common_stats() { + return this.common_stats != null; + } + + public void set_common_stats_isSet(boolean value) { + if (!value) { + this.common_stats = null; + } + } + + public SpecificAggregateStats get_specific_stats() { + return this.specific_stats; + } + + public void set_specific_stats(SpecificAggregateStats specific_stats) { + this.specific_stats = specific_stats; + } + + public void unset_specific_stats() { + this.specific_stats = null; + } + + /** Returns true if field specific_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_specific_stats() { + return this.specific_stats != null; + } + + public void set_specific_stats_isSet(boolean value) { + if (!value) { + this.specific_stats = null; + } + } + + public ErrorInfo get_last_error() { + return this.last_error; + } + + public void set_last_error(ErrorInfo last_error) { + this.last_error = last_error; + } + + public void unset_last_error() { + this.last_error = null; + } + + /** Returns true if field last_error is set (has been assigned a value) and false otherwise */ + public boolean is_set_last_error() { + return this.last_error != null; + } + + public void set_last_error_isSet(boolean value) { + if (!value) { + this.last_error = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPE: + if (value == null) { + unset_type(); + } else { + set_type((ComponentType)value); + } + break; + + case COMMON_STATS: + if (value == null) { + unset_common_stats(); + } else { + set_common_stats((CommonAggregateStats)value); + } + break; + + case SPECIFIC_STATS: + if (value == null) { + unset_specific_stats(); + } else { + set_specific_stats((SpecificAggregateStats)value); + } + break; + + case LAST_ERROR: + if (value == null) { + unset_last_error(); + } else { + set_last_error((ErrorInfo)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPE: + return get_type(); + + case COMMON_STATS: + return get_common_stats(); + + case SPECIFIC_STATS: + return get_specific_stats(); + + case LAST_ERROR: + return get_last_error(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPE: + return is_set_type(); + case COMMON_STATS: + return is_set_common_stats(); + case SPECIFIC_STATS: + return is_set_specific_stats(); + case LAST_ERROR: + return is_set_last_error(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ComponentAggregateStats) + return this.equals((ComponentAggregateStats)that); + return false; + } + + public boolean equals(ComponentAggregateStats that) { + if (that == null) + return false; + + boolean this_present_type = true && this.is_set_type(); + boolean that_present_type = true && that.is_set_type(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_common_stats = true && this.is_set_common_stats(); + boolean that_present_common_stats = true && that.is_set_common_stats(); + if (this_present_common_stats || that_present_common_stats) { + if (!(this_present_common_stats && that_present_common_stats)) + return false; + if (!this.common_stats.equals(that.common_stats)) + return false; + } + + boolean this_present_specific_stats = true && this.is_set_specific_stats(); + boolean that_present_specific_stats = true && that.is_set_specific_stats(); + if (this_present_specific_stats || that_present_specific_stats) { + if (!(this_present_specific_stats && that_present_specific_stats)) + return false; + if (!this.specific_stats.equals(that.specific_stats)) + return false; + } + + boolean this_present_last_error = true && this.is_set_last_error(); + boolean that_present_last_error = true && that.is_set_last_error(); + if (this_present_last_error || that_present_last_error) { + if (!(this_present_last_error && that_present_last_error)) + return false; + if (!this.last_error.equals(that.last_error)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_type = true && (is_set_type()); + list.add(present_type); + if (present_type) + list.add(type.getValue()); + + boolean present_common_stats = true && (is_set_common_stats()); + list.add(present_common_stats); + if (present_common_stats) + list.add(common_stats); + + boolean present_specific_stats = true && (is_set_specific_stats()); + list.add(present_specific_stats); + if (present_specific_stats) + list.add(specific_stats); + + boolean present_last_error = true && (is_set_last_error()); + list.add(present_last_error); + if (present_last_error) + list.add(last_error); + + return list.hashCode(); + } + + @Override + public int compareTo(ComponentAggregateStats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_type()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_common_stats()).compareTo(other.is_set_common_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_common_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common_stats, other.common_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_specific_stats()).compareTo(other.is_set_specific_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_specific_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.specific_stats, other.specific_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_last_error()).compareTo(other.is_set_last_error()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_last_error()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.last_error, other.last_error); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ComponentAggregateStats("); + boolean first = true; + + if (is_set_type()) { + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + } + if (is_set_common_stats()) { + if (!first) sb.append(", "); + sb.append("common_stats:"); + if (this.common_stats == null) { + sb.append("null"); + } else { + sb.append(this.common_stats); + } + first = false; + } + if (is_set_specific_stats()) { + if (!first) sb.append(", "); + sb.append("specific_stats:"); + if (this.specific_stats == null) { + sb.append("null"); + } else { + sb.append(this.specific_stats); + } + first = false; + } + if (is_set_last_error()) { + if (!first) sb.append(", "); + sb.append("last_error:"); + if (this.last_error == null) { + sb.append("null"); + } else { + sb.append(this.last_error); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (common_stats != null) { + common_stats.validate(); + } + if (last_error != null) { + last_error.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ComponentAggregateStatsStandardSchemeFactory implements SchemeFactory { + public ComponentAggregateStatsStandardScheme getScheme() { + return new ComponentAggregateStatsStandardScheme(); + } + } + + private static class ComponentAggregateStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentAggregateStats struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32()); + struct.set_type_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMMON_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.common_stats = new CommonAggregateStats(); + struct.common_stats.read(iprot); + struct.set_common_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SPECIFIC_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.specific_stats = new SpecificAggregateStats(); + struct.specific_stats.read(iprot); + struct.set_specific_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // LAST_ERROR + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.last_error = new ErrorInfo(); + struct.last_error.read(iprot); + struct.set_last_error_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentAggregateStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.type != null) { + if (struct.is_set_type()) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + } + if (struct.common_stats != null) { + if (struct.is_set_common_stats()) { + oprot.writeFieldBegin(COMMON_STATS_FIELD_DESC); + struct.common_stats.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.specific_stats != null) { + if (struct.is_set_specific_stats()) { + oprot.writeFieldBegin(SPECIFIC_STATS_FIELD_DESC); + struct.specific_stats.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.last_error != null) { + if (struct.is_set_last_error()) { + oprot.writeFieldBegin(LAST_ERROR_FIELD_DESC); + struct.last_error.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ComponentAggregateStatsTupleSchemeFactory implements SchemeFactory { + public ComponentAggregateStatsTupleScheme getScheme() { + return new ComponentAggregateStatsTupleScheme(); + } + } + + private static class ComponentAggregateStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ComponentAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_type()) { + optionals.set(0); + } + if (struct.is_set_common_stats()) { + optionals.set(1); + } + if (struct.is_set_specific_stats()) { + optionals.set(2); + } + if (struct.is_set_last_error()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.is_set_type()) { + oprot.writeI32(struct.type.getValue()); + } + if (struct.is_set_common_stats()) { + struct.common_stats.write(oprot); + } + if (struct.is_set_specific_stats()) { + struct.specific_stats.write(oprot); + } + if (struct.is_set_last_error()) { + struct.last_error.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ComponentAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32()); + struct.set_type_isSet(true); + } + if (incoming.get(1)) { + struct.common_stats = new CommonAggregateStats(); + struct.common_stats.read(iprot); + struct.set_common_stats_isSet(true); + } + if (incoming.get(2)) { + struct.specific_stats = new SpecificAggregateStats(); + struct.specific_stats.read(iprot); + struct.set_specific_stats_isSet(true); + } + if (incoming.get(3)) { + struct.last_error = new ErrorInfo(); + struct.last_error.read(iprot); + struct.set_last_error_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java new file mode 100644 index 00000000000..1bb080c32a2 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java @@ -0,0 +1,2194 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-29") +public class ComponentPageInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentPageInfo"); + + private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField COMPONENT_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("component_type", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_name", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)6); + private static final org.apache.thrift.protocol.TField WINDOW_TO_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_stats", org.apache.thrift.protocol.TType.MAP, (short)7); + private static final org.apache.thrift.protocol.TField GSID_TO_INPUT_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("gsid_to_input_stats", org.apache.thrift.protocol.TType.MAP, (short)8); + private static final org.apache.thrift.protocol.TField SID_TO_OUTPUT_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("sid_to_output_stats", org.apache.thrift.protocol.TType.MAP, (short)9); + private static final org.apache.thrift.protocol.TField EXEC_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("exec_stats", org.apache.thrift.protocol.TType.LIST, (short)10); + private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.LIST, (short)11); + private static final org.apache.thrift.protocol.TField EVENTLOG_HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("eventlog_host", org.apache.thrift.protocol.TType.STRING, (short)12); + private static final org.apache.thrift.protocol.TField EVENTLOG_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("eventlog_port", org.apache.thrift.protocol.TType.I32, (short)13); + private static final org.apache.thrift.protocol.TField DEBUG_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("debug_options", org.apache.thrift.protocol.TType.STRUCT, (short)14); + private static final org.apache.thrift.protocol.TField TOPOLOGY_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_status", org.apache.thrift.protocol.TType.STRING, (short)15); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ComponentPageInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ComponentPageInfoTupleSchemeFactory()); + } + + private String component_id; // required + private ComponentType component_type; // required + private String topology_id; // optional + private String topology_name; // optional + private int num_executors; // optional + private int num_tasks; // optional + private Map window_to_stats; // optional + private Map gsid_to_input_stats; // optional + private Map sid_to_output_stats; // optional + private List exec_stats; // optional + private List errors; // optional + private String eventlog_host; // optional + private int eventlog_port; // optional + private DebugOptions debug_options; // optional + private String topology_status; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COMPONENT_ID((short)1, "component_id"), + /** + * + * @see ComponentType + */ + COMPONENT_TYPE((short)2, "component_type"), + TOPOLOGY_ID((short)3, "topology_id"), + TOPOLOGY_NAME((short)4, "topology_name"), + NUM_EXECUTORS((short)5, "num_executors"), + NUM_TASKS((short)6, "num_tasks"), + WINDOW_TO_STATS((short)7, "window_to_stats"), + GSID_TO_INPUT_STATS((short)8, "gsid_to_input_stats"), + SID_TO_OUTPUT_STATS((short)9, "sid_to_output_stats"), + EXEC_STATS((short)10, "exec_stats"), + ERRORS((short)11, "errors"), + EVENTLOG_HOST((short)12, "eventlog_host"), + EVENTLOG_PORT((short)13, "eventlog_port"), + DEBUG_OPTIONS((short)14, "debug_options"), + TOPOLOGY_STATUS((short)15, "topology_status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COMPONENT_ID + return COMPONENT_ID; + case 2: // COMPONENT_TYPE + return COMPONENT_TYPE; + case 3: // TOPOLOGY_ID + return TOPOLOGY_ID; + case 4: // TOPOLOGY_NAME + return TOPOLOGY_NAME; + case 5: // NUM_EXECUTORS + return NUM_EXECUTORS; + case 6: // NUM_TASKS + return NUM_TASKS; + case 7: // WINDOW_TO_STATS + return WINDOW_TO_STATS; + case 8: // GSID_TO_INPUT_STATS + return GSID_TO_INPUT_STATS; + case 9: // SID_TO_OUTPUT_STATS + return SID_TO_OUTPUT_STATS; + case 10: // EXEC_STATS + return EXEC_STATS; + case 11: // ERRORS + return ERRORS; + case 12: // EVENTLOG_HOST + return EVENTLOG_HOST; + case 13: // EVENTLOG_PORT + return EVENTLOG_PORT; + case 14: // DEBUG_OPTIONS + return DEBUG_OPTIONS; + case 15: // TOPOLOGY_STATUS + return TOPOLOGY_STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __NUM_EXECUTORS_ISSET_ID = 0; + private static final int __NUM_TASKS_ISSET_ID = 1; + private static final int __EVENTLOG_PORT_ISSET_ID = 2; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.TOPOLOGY_ID,_Fields.TOPOLOGY_NAME,_Fields.NUM_EXECUTORS,_Fields.NUM_TASKS,_Fields.WINDOW_TO_STATS,_Fields.GSID_TO_INPUT_STATS,_Fields.SID_TO_OUTPUT_STATS,_Fields.EXEC_STATS,_Fields.ERRORS,_Fields.EVENTLOG_HOST,_Fields.EVENTLOG_PORT,_Fields.DEBUG_OPTIONS,_Fields.TOPOLOGY_STATUS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.COMPONENT_TYPE, new org.apache.thrift.meta_data.FieldMetaData("component_type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ComponentType.class))); + tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topology_name", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.WINDOW_TO_STATS, new org.apache.thrift.meta_data.FieldMetaData("window_to_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class)))); + tmpMap.put(_Fields.GSID_TO_INPUT_STATS, new org.apache.thrift.meta_data.FieldMetaData("gsid_to_input_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class)))); + tmpMap.put(_Fields.SID_TO_OUTPUT_STATS, new org.apache.thrift.meta_data.FieldMetaData("sid_to_output_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class)))); + tmpMap.put(_Fields.EXEC_STATS, new org.apache.thrift.meta_data.FieldMetaData("exec_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorAggregateStats.class)))); + tmpMap.put(_Fields.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class)))); + tmpMap.put(_Fields.EVENTLOG_HOST, new org.apache.thrift.meta_data.FieldMetaData("eventlog_host", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.EVENTLOG_PORT, new org.apache.thrift.meta_data.FieldMetaData("eventlog_port", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.DEBUG_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("debug_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class))); + tmpMap.put(_Fields.TOPOLOGY_STATUS, new org.apache.thrift.meta_data.FieldMetaData("topology_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentPageInfo.class, metaDataMap); + } + + public ComponentPageInfo() { + } + + public ComponentPageInfo( + String component_id, + ComponentType component_type) + { + this(); + this.component_id = component_id; + this.component_type = component_type; + } + + /** + * Performs a deep copy on other. + */ + public ComponentPageInfo(ComponentPageInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_component_id()) { + this.component_id = other.component_id; + } + if (other.is_set_component_type()) { + this.component_type = other.component_type; + } + if (other.is_set_topology_id()) { + this.topology_id = other.topology_id; + } + if (other.is_set_topology_name()) { + this.topology_name = other.topology_name; + } + this.num_executors = other.num_executors; + this.num_tasks = other.num_tasks; + if (other.is_set_window_to_stats()) { + Map __this__window_to_stats = new HashMap(other.window_to_stats.size()); + for (Map.Entry other_element : other.window_to_stats.entrySet()) { + + String other_element_key = other_element.getKey(); + ComponentAggregateStats other_element_value = other_element.getValue(); + + String __this__window_to_stats_copy_key = other_element_key; + + ComponentAggregateStats __this__window_to_stats_copy_value = new ComponentAggregateStats(other_element_value); + + __this__window_to_stats.put(__this__window_to_stats_copy_key, __this__window_to_stats_copy_value); + } + this.window_to_stats = __this__window_to_stats; + } + if (other.is_set_gsid_to_input_stats()) { + Map __this__gsid_to_input_stats = new HashMap(other.gsid_to_input_stats.size()); + for (Map.Entry other_element : other.gsid_to_input_stats.entrySet()) { + + GlobalStreamId other_element_key = other_element.getKey(); + ComponentAggregateStats other_element_value = other_element.getValue(); + + GlobalStreamId __this__gsid_to_input_stats_copy_key = new GlobalStreamId(other_element_key); + + ComponentAggregateStats __this__gsid_to_input_stats_copy_value = new ComponentAggregateStats(other_element_value); + + __this__gsid_to_input_stats.put(__this__gsid_to_input_stats_copy_key, __this__gsid_to_input_stats_copy_value); + } + this.gsid_to_input_stats = __this__gsid_to_input_stats; + } + if (other.is_set_sid_to_output_stats()) { + Map __this__sid_to_output_stats = new HashMap(other.sid_to_output_stats.size()); + for (Map.Entry other_element : other.sid_to_output_stats.entrySet()) { + + String other_element_key = other_element.getKey(); + ComponentAggregateStats other_element_value = other_element.getValue(); + + String __this__sid_to_output_stats_copy_key = other_element_key; + + ComponentAggregateStats __this__sid_to_output_stats_copy_value = new ComponentAggregateStats(other_element_value); + + __this__sid_to_output_stats.put(__this__sid_to_output_stats_copy_key, __this__sid_to_output_stats_copy_value); + } + this.sid_to_output_stats = __this__sid_to_output_stats; + } + if (other.is_set_exec_stats()) { + List __this__exec_stats = new ArrayList(other.exec_stats.size()); + for (ExecutorAggregateStats other_element : other.exec_stats) { + __this__exec_stats.add(new ExecutorAggregateStats(other_element)); + } + this.exec_stats = __this__exec_stats; + } + if (other.is_set_errors()) { + List __this__errors = new ArrayList(other.errors.size()); + for (ErrorInfo other_element : other.errors) { + __this__errors.add(new ErrorInfo(other_element)); + } + this.errors = __this__errors; + } + if (other.is_set_eventlog_host()) { + this.eventlog_host = other.eventlog_host; + } + this.eventlog_port = other.eventlog_port; + if (other.is_set_debug_options()) { + this.debug_options = new DebugOptions(other.debug_options); + } + if (other.is_set_topology_status()) { + this.topology_status = other.topology_status; + } + } + + public ComponentPageInfo deepCopy() { + return new ComponentPageInfo(this); + } + + @Override + public void clear() { + this.component_id = null; + this.component_type = null; + this.topology_id = null; + this.topology_name = null; + set_num_executors_isSet(false); + this.num_executors = 0; + set_num_tasks_isSet(false); + this.num_tasks = 0; + this.window_to_stats = null; + this.gsid_to_input_stats = null; + this.sid_to_output_stats = null; + this.exec_stats = null; + this.errors = null; + this.eventlog_host = null; + set_eventlog_port_isSet(false); + this.eventlog_port = 0; + this.debug_options = null; + this.topology_status = null; + } + + public String get_component_id() { + return this.component_id; + } + + public void set_component_id(String component_id) { + this.component_id = component_id; + } + + public void unset_component_id() { + this.component_id = null; + } + + /** Returns true if field component_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_component_id() { + return this.component_id != null; + } + + public void set_component_id_isSet(boolean value) { + if (!value) { + this.component_id = null; + } + } + + /** + * + * @see ComponentType + */ + public ComponentType get_component_type() { + return this.component_type; + } + + /** + * + * @see ComponentType + */ + public void set_component_type(ComponentType component_type) { + this.component_type = component_type; + } + + public void unset_component_type() { + this.component_type = null; + } + + /** Returns true if field component_type is set (has been assigned a value) and false otherwise */ + public boolean is_set_component_type() { + return this.component_type != null; + } + + public void set_component_type_isSet(boolean value) { + if (!value) { + this.component_type = null; + } + } + + public String get_topology_id() { + return this.topology_id; + } + + public void set_topology_id(String topology_id) { + this.topology_id = topology_id; + } + + public void unset_topology_id() { + this.topology_id = null; + } + + /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_id() { + return this.topology_id != null; + } + + public void set_topology_id_isSet(boolean value) { + if (!value) { + this.topology_id = null; + } + } + + public String get_topology_name() { + return this.topology_name; + } + + public void set_topology_name(String topology_name) { + this.topology_name = topology_name; + } + + public void unset_topology_name() { + this.topology_name = null; + } + + /** Returns true if field topology_name is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_name() { + return this.topology_name != null; + } + + public void set_topology_name_isSet(boolean value) { + if (!value) { + this.topology_name = null; + } + } + + public int get_num_executors() { + return this.num_executors; + } + + public void set_num_executors(int num_executors) { + this.num_executors = num_executors; + set_num_executors_isSet(true); + } + + public void unset_num_executors() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_executors() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + public void set_num_executors_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value); + } + + public int get_num_tasks() { + return this.num_tasks; + } + + public void set_num_tasks(int num_tasks) { + this.num_tasks = num_tasks; + set_num_tasks_isSet(true); + } + + public void unset_num_tasks() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_tasks() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + public void set_num_tasks_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value); + } + + public int get_window_to_stats_size() { + return (this.window_to_stats == null) ? 0 : this.window_to_stats.size(); + } + + public void put_to_window_to_stats(String key, ComponentAggregateStats val) { + if (this.window_to_stats == null) { + this.window_to_stats = new HashMap(); + } + this.window_to_stats.put(key, val); + } + + public Map get_window_to_stats() { + return this.window_to_stats; + } + + public void set_window_to_stats(Map window_to_stats) { + this.window_to_stats = window_to_stats; + } + + public void unset_window_to_stats() { + this.window_to_stats = null; + } + + /** Returns true if field window_to_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_window_to_stats() { + return this.window_to_stats != null; + } + + public void set_window_to_stats_isSet(boolean value) { + if (!value) { + this.window_to_stats = null; + } + } + + public int get_gsid_to_input_stats_size() { + return (this.gsid_to_input_stats == null) ? 0 : this.gsid_to_input_stats.size(); + } + + public void put_to_gsid_to_input_stats(GlobalStreamId key, ComponentAggregateStats val) { + if (this.gsid_to_input_stats == null) { + this.gsid_to_input_stats = new HashMap(); + } + this.gsid_to_input_stats.put(key, val); + } + + public Map get_gsid_to_input_stats() { + return this.gsid_to_input_stats; + } + + public void set_gsid_to_input_stats(Map gsid_to_input_stats) { + this.gsid_to_input_stats = gsid_to_input_stats; + } + + public void unset_gsid_to_input_stats() { + this.gsid_to_input_stats = null; + } + + /** Returns true if field gsid_to_input_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_gsid_to_input_stats() { + return this.gsid_to_input_stats != null; + } + + public void set_gsid_to_input_stats_isSet(boolean value) { + if (!value) { + this.gsid_to_input_stats = null; + } + } + + public int get_sid_to_output_stats_size() { + return (this.sid_to_output_stats == null) ? 0 : this.sid_to_output_stats.size(); + } + + public void put_to_sid_to_output_stats(String key, ComponentAggregateStats val) { + if (this.sid_to_output_stats == null) { + this.sid_to_output_stats = new HashMap(); + } + this.sid_to_output_stats.put(key, val); + } + + public Map get_sid_to_output_stats() { + return this.sid_to_output_stats; + } + + public void set_sid_to_output_stats(Map sid_to_output_stats) { + this.sid_to_output_stats = sid_to_output_stats; + } + + public void unset_sid_to_output_stats() { + this.sid_to_output_stats = null; + } + + /** Returns true if field sid_to_output_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_sid_to_output_stats() { + return this.sid_to_output_stats != null; + } + + public void set_sid_to_output_stats_isSet(boolean value) { + if (!value) { + this.sid_to_output_stats = null; + } + } + + public int get_exec_stats_size() { + return (this.exec_stats == null) ? 0 : this.exec_stats.size(); + } + + public java.util.Iterator get_exec_stats_iterator() { + return (this.exec_stats == null) ? null : this.exec_stats.iterator(); + } + + public void add_to_exec_stats(ExecutorAggregateStats elem) { + if (this.exec_stats == null) { + this.exec_stats = new ArrayList(); + } + this.exec_stats.add(elem); + } + + public List get_exec_stats() { + return this.exec_stats; + } + + public void set_exec_stats(List exec_stats) { + this.exec_stats = exec_stats; + } + + public void unset_exec_stats() { + this.exec_stats = null; + } + + /** Returns true if field exec_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_exec_stats() { + return this.exec_stats != null; + } + + public void set_exec_stats_isSet(boolean value) { + if (!value) { + this.exec_stats = null; + } + } + + public int get_errors_size() { + return (this.errors == null) ? 0 : this.errors.size(); + } + + public java.util.Iterator get_errors_iterator() { + return (this.errors == null) ? null : this.errors.iterator(); + } + + public void add_to_errors(ErrorInfo elem) { + if (this.errors == null) { + this.errors = new ArrayList(); + } + this.errors.add(elem); + } + + public List get_errors() { + return this.errors; + } + + public void set_errors(List errors) { + this.errors = errors; + } + + public void unset_errors() { + this.errors = null; + } + + /** Returns true if field errors is set (has been assigned a value) and false otherwise */ + public boolean is_set_errors() { + return this.errors != null; + } + + public void set_errors_isSet(boolean value) { + if (!value) { + this.errors = null; + } + } + + public String get_eventlog_host() { + return this.eventlog_host; + } + + public void set_eventlog_host(String eventlog_host) { + this.eventlog_host = eventlog_host; + } + + public void unset_eventlog_host() { + this.eventlog_host = null; + } + + /** Returns true if field eventlog_host is set (has been assigned a value) and false otherwise */ + public boolean is_set_eventlog_host() { + return this.eventlog_host != null; + } + + public void set_eventlog_host_isSet(boolean value) { + if (!value) { + this.eventlog_host = null; + } + } + + public int get_eventlog_port() { + return this.eventlog_port; + } + + public void set_eventlog_port(int eventlog_port) { + this.eventlog_port = eventlog_port; + set_eventlog_port_isSet(true); + } + + public void unset_eventlog_port() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID); + } + + /** Returns true if field eventlog_port is set (has been assigned a value) and false otherwise */ + public boolean is_set_eventlog_port() { + return EncodingUtils.testBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID); + } + + public void set_eventlog_port_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID, value); + } + + public DebugOptions get_debug_options() { + return this.debug_options; + } + + public void set_debug_options(DebugOptions debug_options) { + this.debug_options = debug_options; + } + + public void unset_debug_options() { + this.debug_options = null; + } + + /** Returns true if field debug_options is set (has been assigned a value) and false otherwise */ + public boolean is_set_debug_options() { + return this.debug_options != null; + } + + public void set_debug_options_isSet(boolean value) { + if (!value) { + this.debug_options = null; + } + } + + public String get_topology_status() { + return this.topology_status; + } + + public void set_topology_status(String topology_status) { + this.topology_status = topology_status; + } + + public void unset_topology_status() { + this.topology_status = null; + } + + /** Returns true if field topology_status is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_status() { + return this.topology_status != null; + } + + public void set_topology_status_isSet(boolean value) { + if (!value) { + this.topology_status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COMPONENT_ID: + if (value == null) { + unset_component_id(); + } else { + set_component_id((String)value); + } + break; + + case COMPONENT_TYPE: + if (value == null) { + unset_component_type(); + } else { + set_component_type((ComponentType)value); + } + break; + + case TOPOLOGY_ID: + if (value == null) { + unset_topology_id(); + } else { + set_topology_id((String)value); + } + break; + + case TOPOLOGY_NAME: + if (value == null) { + unset_topology_name(); + } else { + set_topology_name((String)value); + } + break; + + case NUM_EXECUTORS: + if (value == null) { + unset_num_executors(); + } else { + set_num_executors((Integer)value); + } + break; + + case NUM_TASKS: + if (value == null) { + unset_num_tasks(); + } else { + set_num_tasks((Integer)value); + } + break; + + case WINDOW_TO_STATS: + if (value == null) { + unset_window_to_stats(); + } else { + set_window_to_stats((Map)value); + } + break; + + case GSID_TO_INPUT_STATS: + if (value == null) { + unset_gsid_to_input_stats(); + } else { + set_gsid_to_input_stats((Map)value); + } + break; + + case SID_TO_OUTPUT_STATS: + if (value == null) { + unset_sid_to_output_stats(); + } else { + set_sid_to_output_stats((Map)value); + } + break; + + case EXEC_STATS: + if (value == null) { + unset_exec_stats(); + } else { + set_exec_stats((List)value); + } + break; + + case ERRORS: + if (value == null) { + unset_errors(); + } else { + set_errors((List)value); + } + break; + + case EVENTLOG_HOST: + if (value == null) { + unset_eventlog_host(); + } else { + set_eventlog_host((String)value); + } + break; + + case EVENTLOG_PORT: + if (value == null) { + unset_eventlog_port(); + } else { + set_eventlog_port((Integer)value); + } + break; + + case DEBUG_OPTIONS: + if (value == null) { + unset_debug_options(); + } else { + set_debug_options((DebugOptions)value); + } + break; + + case TOPOLOGY_STATUS: + if (value == null) { + unset_topology_status(); + } else { + set_topology_status((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COMPONENT_ID: + return get_component_id(); + + case COMPONENT_TYPE: + return get_component_type(); + + case TOPOLOGY_ID: + return get_topology_id(); + + case TOPOLOGY_NAME: + return get_topology_name(); + + case NUM_EXECUTORS: + return Integer.valueOf(get_num_executors()); + + case NUM_TASKS: + return Integer.valueOf(get_num_tasks()); + + case WINDOW_TO_STATS: + return get_window_to_stats(); + + case GSID_TO_INPUT_STATS: + return get_gsid_to_input_stats(); + + case SID_TO_OUTPUT_STATS: + return get_sid_to_output_stats(); + + case EXEC_STATS: + return get_exec_stats(); + + case ERRORS: + return get_errors(); + + case EVENTLOG_HOST: + return get_eventlog_host(); + + case EVENTLOG_PORT: + return Integer.valueOf(get_eventlog_port()); + + case DEBUG_OPTIONS: + return get_debug_options(); + + case TOPOLOGY_STATUS: + return get_topology_status(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COMPONENT_ID: + return is_set_component_id(); + case COMPONENT_TYPE: + return is_set_component_type(); + case TOPOLOGY_ID: + return is_set_topology_id(); + case TOPOLOGY_NAME: + return is_set_topology_name(); + case NUM_EXECUTORS: + return is_set_num_executors(); + case NUM_TASKS: + return is_set_num_tasks(); + case WINDOW_TO_STATS: + return is_set_window_to_stats(); + case GSID_TO_INPUT_STATS: + return is_set_gsid_to_input_stats(); + case SID_TO_OUTPUT_STATS: + return is_set_sid_to_output_stats(); + case EXEC_STATS: + return is_set_exec_stats(); + case ERRORS: + return is_set_errors(); + case EVENTLOG_HOST: + return is_set_eventlog_host(); + case EVENTLOG_PORT: + return is_set_eventlog_port(); + case DEBUG_OPTIONS: + return is_set_debug_options(); + case TOPOLOGY_STATUS: + return is_set_topology_status(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ComponentPageInfo) + return this.equals((ComponentPageInfo)that); + return false; + } + + public boolean equals(ComponentPageInfo that) { + if (that == null) + return false; + + boolean this_present_component_id = true && this.is_set_component_id(); + boolean that_present_component_id = true && that.is_set_component_id(); + if (this_present_component_id || that_present_component_id) { + if (!(this_present_component_id && that_present_component_id)) + return false; + if (!this.component_id.equals(that.component_id)) + return false; + } + + boolean this_present_component_type = true && this.is_set_component_type(); + boolean that_present_component_type = true && that.is_set_component_type(); + if (this_present_component_type || that_present_component_type) { + if (!(this_present_component_type && that_present_component_type)) + return false; + if (!this.component_type.equals(that.component_type)) + return false; + } + + boolean this_present_topology_id = true && this.is_set_topology_id(); + boolean that_present_topology_id = true && that.is_set_topology_id(); + if (this_present_topology_id || that_present_topology_id) { + if (!(this_present_topology_id && that_present_topology_id)) + return false; + if (!this.topology_id.equals(that.topology_id)) + return false; + } + + boolean this_present_topology_name = true && this.is_set_topology_name(); + boolean that_present_topology_name = true && that.is_set_topology_name(); + if (this_present_topology_name || that_present_topology_name) { + if (!(this_present_topology_name && that_present_topology_name)) + return false; + if (!this.topology_name.equals(that.topology_name)) + return false; + } + + boolean this_present_num_executors = true && this.is_set_num_executors(); + boolean that_present_num_executors = true && that.is_set_num_executors(); + if (this_present_num_executors || that_present_num_executors) { + if (!(this_present_num_executors && that_present_num_executors)) + return false; + if (this.num_executors != that.num_executors) + return false; + } + + boolean this_present_num_tasks = true && this.is_set_num_tasks(); + boolean that_present_num_tasks = true && that.is_set_num_tasks(); + if (this_present_num_tasks || that_present_num_tasks) { + if (!(this_present_num_tasks && that_present_num_tasks)) + return false; + if (this.num_tasks != that.num_tasks) + return false; + } + + boolean this_present_window_to_stats = true && this.is_set_window_to_stats(); + boolean that_present_window_to_stats = true && that.is_set_window_to_stats(); + if (this_present_window_to_stats || that_present_window_to_stats) { + if (!(this_present_window_to_stats && that_present_window_to_stats)) + return false; + if (!this.window_to_stats.equals(that.window_to_stats)) + return false; + } + + boolean this_present_gsid_to_input_stats = true && this.is_set_gsid_to_input_stats(); + boolean that_present_gsid_to_input_stats = true && that.is_set_gsid_to_input_stats(); + if (this_present_gsid_to_input_stats || that_present_gsid_to_input_stats) { + if (!(this_present_gsid_to_input_stats && that_present_gsid_to_input_stats)) + return false; + if (!this.gsid_to_input_stats.equals(that.gsid_to_input_stats)) + return false; + } + + boolean this_present_sid_to_output_stats = true && this.is_set_sid_to_output_stats(); + boolean that_present_sid_to_output_stats = true && that.is_set_sid_to_output_stats(); + if (this_present_sid_to_output_stats || that_present_sid_to_output_stats) { + if (!(this_present_sid_to_output_stats && that_present_sid_to_output_stats)) + return false; + if (!this.sid_to_output_stats.equals(that.sid_to_output_stats)) + return false; + } + + boolean this_present_exec_stats = true && this.is_set_exec_stats(); + boolean that_present_exec_stats = true && that.is_set_exec_stats(); + if (this_present_exec_stats || that_present_exec_stats) { + if (!(this_present_exec_stats && that_present_exec_stats)) + return false; + if (!this.exec_stats.equals(that.exec_stats)) + return false; + } + + boolean this_present_errors = true && this.is_set_errors(); + boolean that_present_errors = true && that.is_set_errors(); + if (this_present_errors || that_present_errors) { + if (!(this_present_errors && that_present_errors)) + return false; + if (!this.errors.equals(that.errors)) + return false; + } + + boolean this_present_eventlog_host = true && this.is_set_eventlog_host(); + boolean that_present_eventlog_host = true && that.is_set_eventlog_host(); + if (this_present_eventlog_host || that_present_eventlog_host) { + if (!(this_present_eventlog_host && that_present_eventlog_host)) + return false; + if (!this.eventlog_host.equals(that.eventlog_host)) + return false; + } + + boolean this_present_eventlog_port = true && this.is_set_eventlog_port(); + boolean that_present_eventlog_port = true && that.is_set_eventlog_port(); + if (this_present_eventlog_port || that_present_eventlog_port) { + if (!(this_present_eventlog_port && that_present_eventlog_port)) + return false; + if (this.eventlog_port != that.eventlog_port) + return false; + } + + boolean this_present_debug_options = true && this.is_set_debug_options(); + boolean that_present_debug_options = true && that.is_set_debug_options(); + if (this_present_debug_options || that_present_debug_options) { + if (!(this_present_debug_options && that_present_debug_options)) + return false; + if (!this.debug_options.equals(that.debug_options)) + return false; + } + + boolean this_present_topology_status = true && this.is_set_topology_status(); + boolean that_present_topology_status = true && that.is_set_topology_status(); + if (this_present_topology_status || that_present_topology_status) { + if (!(this_present_topology_status && that_present_topology_status)) + return false; + if (!this.topology_status.equals(that.topology_status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_component_id = true && (is_set_component_id()); + list.add(present_component_id); + if (present_component_id) + list.add(component_id); + + boolean present_component_type = true && (is_set_component_type()); + list.add(present_component_type); + if (present_component_type) + list.add(component_type.getValue()); + + boolean present_topology_id = true && (is_set_topology_id()); + list.add(present_topology_id); + if (present_topology_id) + list.add(topology_id); + + boolean present_topology_name = true && (is_set_topology_name()); + list.add(present_topology_name); + if (present_topology_name) + list.add(topology_name); + + boolean present_num_executors = true && (is_set_num_executors()); + list.add(present_num_executors); + if (present_num_executors) + list.add(num_executors); + + boolean present_num_tasks = true && (is_set_num_tasks()); + list.add(present_num_tasks); + if (present_num_tasks) + list.add(num_tasks); + + boolean present_window_to_stats = true && (is_set_window_to_stats()); + list.add(present_window_to_stats); + if (present_window_to_stats) + list.add(window_to_stats); + + boolean present_gsid_to_input_stats = true && (is_set_gsid_to_input_stats()); + list.add(present_gsid_to_input_stats); + if (present_gsid_to_input_stats) + list.add(gsid_to_input_stats); + + boolean present_sid_to_output_stats = true && (is_set_sid_to_output_stats()); + list.add(present_sid_to_output_stats); + if (present_sid_to_output_stats) + list.add(sid_to_output_stats); + + boolean present_exec_stats = true && (is_set_exec_stats()); + list.add(present_exec_stats); + if (present_exec_stats) + list.add(exec_stats); + + boolean present_errors = true && (is_set_errors()); + list.add(present_errors); + if (present_errors) + list.add(errors); + + boolean present_eventlog_host = true && (is_set_eventlog_host()); + list.add(present_eventlog_host); + if (present_eventlog_host) + list.add(eventlog_host); + + boolean present_eventlog_port = true && (is_set_eventlog_port()); + list.add(present_eventlog_port); + if (present_eventlog_port) + list.add(eventlog_port); + + boolean present_debug_options = true && (is_set_debug_options()); + list.add(present_debug_options); + if (present_debug_options) + list.add(debug_options); + + boolean present_topology_status = true && (is_set_topology_status()); + list.add(present_topology_status); + if (present_topology_status) + list.add(topology_status); + + return list.hashCode(); + } + + @Override + public int compareTo(ComponentPageInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(other.is_set_component_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_component_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, other.component_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_component_type()).compareTo(other.is_set_component_type()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_component_type()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_type, other.component_type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology_name()).compareTo(other.is_set_topology_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_name, other.topology_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_executors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_tasks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_window_to_stats()).compareTo(other.is_set_window_to_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window_to_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_stats, other.window_to_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_gsid_to_input_stats()).compareTo(other.is_set_gsid_to_input_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_gsid_to_input_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.gsid_to_input_stats, other.gsid_to_input_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_sid_to_output_stats()).compareTo(other.is_set_sid_to_output_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_sid_to_output_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sid_to_output_stats, other.sid_to_output_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_exec_stats()).compareTo(other.is_set_exec_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_exec_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exec_stats, other.exec_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_errors()).compareTo(other.is_set_errors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_errors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errors, other.errors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_eventlog_host()).compareTo(other.is_set_eventlog_host()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_eventlog_host()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eventlog_host, other.eventlog_host); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_eventlog_port()).compareTo(other.is_set_eventlog_port()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_eventlog_port()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eventlog_port, other.eventlog_port); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_debug_options()).compareTo(other.is_set_debug_options()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_debug_options()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug_options, other.debug_options); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology_status()).compareTo(other.is_set_topology_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_status, other.topology_status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ComponentPageInfo("); + boolean first = true; + + sb.append("component_id:"); + if (this.component_id == null) { + sb.append("null"); + } else { + sb.append(this.component_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("component_type:"); + if (this.component_type == null) { + sb.append("null"); + } else { + sb.append(this.component_type); + } + first = false; + if (is_set_topology_id()) { + if (!first) sb.append(", "); + sb.append("topology_id:"); + if (this.topology_id == null) { + sb.append("null"); + } else { + sb.append(this.topology_id); + } + first = false; + } + if (is_set_topology_name()) { + if (!first) sb.append(", "); + sb.append("topology_name:"); + if (this.topology_name == null) { + sb.append("null"); + } else { + sb.append(this.topology_name); + } + first = false; + } + if (is_set_num_executors()) { + if (!first) sb.append(", "); + sb.append("num_executors:"); + sb.append(this.num_executors); + first = false; + } + if (is_set_num_tasks()) { + if (!first) sb.append(", "); + sb.append("num_tasks:"); + sb.append(this.num_tasks); + first = false; + } + if (is_set_window_to_stats()) { + if (!first) sb.append(", "); + sb.append("window_to_stats:"); + if (this.window_to_stats == null) { + sb.append("null"); + } else { + sb.append(this.window_to_stats); + } + first = false; + } + if (is_set_gsid_to_input_stats()) { + if (!first) sb.append(", "); + sb.append("gsid_to_input_stats:"); + if (this.gsid_to_input_stats == null) { + sb.append("null"); + } else { + sb.append(this.gsid_to_input_stats); + } + first = false; + } + if (is_set_sid_to_output_stats()) { + if (!first) sb.append(", "); + sb.append("sid_to_output_stats:"); + if (this.sid_to_output_stats == null) { + sb.append("null"); + } else { + sb.append(this.sid_to_output_stats); + } + first = false; + } + if (is_set_exec_stats()) { + if (!first) sb.append(", "); + sb.append("exec_stats:"); + if (this.exec_stats == null) { + sb.append("null"); + } else { + sb.append(this.exec_stats); + } + first = false; + } + if (is_set_errors()) { + if (!first) sb.append(", "); + sb.append("errors:"); + if (this.errors == null) { + sb.append("null"); + } else { + sb.append(this.errors); + } + first = false; + } + if (is_set_eventlog_host()) { + if (!first) sb.append(", "); + sb.append("eventlog_host:"); + if (this.eventlog_host == null) { + sb.append("null"); + } else { + sb.append(this.eventlog_host); + } + first = false; + } + if (is_set_eventlog_port()) { + if (!first) sb.append(", "); + sb.append("eventlog_port:"); + sb.append(this.eventlog_port); + first = false; + } + if (is_set_debug_options()) { + if (!first) sb.append(", "); + sb.append("debug_options:"); + if (this.debug_options == null) { + sb.append("null"); + } else { + sb.append(this.debug_options); + } + first = false; + } + if (is_set_topology_status()) { + if (!first) sb.append(", "); + sb.append("topology_status:"); + if (this.topology_status == null) { + sb.append("null"); + } else { + sb.append(this.topology_status); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_component_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_id' is unset! Struct:" + toString()); + } + + if (!is_set_component_type()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_type' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (debug_options != null) { + debug_options.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ComponentPageInfoStandardSchemeFactory implements SchemeFactory { + public ComponentPageInfoStandardScheme getScheme() { + return new ComponentPageInfoStandardScheme(); + } + } + + private static class ComponentPageInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentPageInfo struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COMPONENT_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.component_id = iprot.readString(); + struct.set_component_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMPONENT_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.component_type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32()); + struct.set_component_type_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TOPOLOGY_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology_id = iprot.readString(); + struct.set_topology_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TOPOLOGY_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology_name = iprot.readString(); + struct.set_topology_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // NUM_EXECUTORS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_executors = iprot.readI32(); + struct.set_num_executors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // NUM_TASKS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // WINDOW_TO_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map404 = iprot.readMapBegin(); + struct.window_to_stats = new HashMap(2*_map404.size); + String _key405; + ComponentAggregateStats _val406; + for (int _i407 = 0; _i407 < _map404.size; ++_i407) + { + _key405 = iprot.readString(); + _val406 = new ComponentAggregateStats(); + _val406.read(iprot); + struct.window_to_stats.put(_key405, _val406); + } + iprot.readMapEnd(); + } + struct.set_window_to_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // GSID_TO_INPUT_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map408 = iprot.readMapBegin(); + struct.gsid_to_input_stats = new HashMap(2*_map408.size); + GlobalStreamId _key409; + ComponentAggregateStats _val410; + for (int _i411 = 0; _i411 < _map408.size; ++_i411) + { + _key409 = new GlobalStreamId(); + _key409.read(iprot); + _val410 = new ComponentAggregateStats(); + _val410.read(iprot); + struct.gsid_to_input_stats.put(_key409, _val410); + } + iprot.readMapEnd(); + } + struct.set_gsid_to_input_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 9: // SID_TO_OUTPUT_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map412 = iprot.readMapBegin(); + struct.sid_to_output_stats = new HashMap(2*_map412.size); + String _key413; + ComponentAggregateStats _val414; + for (int _i415 = 0; _i415 < _map412.size; ++_i415) + { + _key413 = iprot.readString(); + _val414 = new ComponentAggregateStats(); + _val414.read(iprot); + struct.sid_to_output_stats.put(_key413, _val414); + } + iprot.readMapEnd(); + } + struct.set_sid_to_output_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 10: // EXEC_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list416 = iprot.readListBegin(); + struct.exec_stats = new ArrayList(_list416.size); + ExecutorAggregateStats _elem417; + for (int _i418 = 0; _i418 < _list416.size; ++_i418) + { + _elem417 = new ExecutorAggregateStats(); + _elem417.read(iprot); + struct.exec_stats.add(_elem417); + } + iprot.readListEnd(); + } + struct.set_exec_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 11: // ERRORS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list419 = iprot.readListBegin(); + struct.errors = new ArrayList(_list419.size); + ErrorInfo _elem420; + for (int _i421 = 0; _i421 < _list419.size; ++_i421) + { + _elem420 = new ErrorInfo(); + _elem420.read(iprot); + struct.errors.add(_elem420); + } + iprot.readListEnd(); + } + struct.set_errors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 12: // EVENTLOG_HOST + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.eventlog_host = iprot.readString(); + struct.set_eventlog_host_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 13: // EVENTLOG_PORT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.eventlog_port = iprot.readI32(); + struct.set_eventlog_port_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 14: // DEBUG_OPTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.debug_options = new DebugOptions(); + struct.debug_options.read(iprot); + struct.set_debug_options_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 15: // TOPOLOGY_STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology_status = iprot.readString(); + struct.set_topology_status_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentPageInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.component_id != null) { + oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); + oprot.writeString(struct.component_id); + oprot.writeFieldEnd(); + } + if (struct.component_type != null) { + oprot.writeFieldBegin(COMPONENT_TYPE_FIELD_DESC); + oprot.writeI32(struct.component_type.getValue()); + oprot.writeFieldEnd(); + } + if (struct.topology_id != null) { + if (struct.is_set_topology_id()) { + oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC); + oprot.writeString(struct.topology_id); + oprot.writeFieldEnd(); + } + } + if (struct.topology_name != null) { + if (struct.is_set_topology_name()) { + oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC); + oprot.writeString(struct.topology_name); + oprot.writeFieldEnd(); + } + } + if (struct.is_set_num_executors()) { + oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); + oprot.writeI32(struct.num_executors); + oprot.writeFieldEnd(); + } + if (struct.is_set_num_tasks()) { + oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC); + oprot.writeI32(struct.num_tasks); + oprot.writeFieldEnd(); + } + if (struct.window_to_stats != null) { + if (struct.is_set_window_to_stats()) { + oprot.writeFieldBegin(WINDOW_TO_STATS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.window_to_stats.size())); + for (Map.Entry _iter422 : struct.window_to_stats.entrySet()) + { + oprot.writeString(_iter422.getKey()); + _iter422.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.gsid_to_input_stats != null) { + if (struct.is_set_gsid_to_input_stats()) { + oprot.writeFieldBegin(GSID_TO_INPUT_STATS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.gsid_to_input_stats.size())); + for (Map.Entry _iter423 : struct.gsid_to_input_stats.entrySet()) + { + _iter423.getKey().write(oprot); + _iter423.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.sid_to_output_stats != null) { + if (struct.is_set_sid_to_output_stats()) { + oprot.writeFieldBegin(SID_TO_OUTPUT_STATS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.sid_to_output_stats.size())); + for (Map.Entry _iter424 : struct.sid_to_output_stats.entrySet()) + { + oprot.writeString(_iter424.getKey()); + _iter424.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.exec_stats != null) { + if (struct.is_set_exec_stats()) { + oprot.writeFieldBegin(EXEC_STATS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.exec_stats.size())); + for (ExecutorAggregateStats _iter425 : struct.exec_stats) + { + _iter425.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.errors != null) { + if (struct.is_set_errors()) { + oprot.writeFieldBegin(ERRORS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.errors.size())); + for (ErrorInfo _iter426 : struct.errors) + { + _iter426.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.eventlog_host != null) { + if (struct.is_set_eventlog_host()) { + oprot.writeFieldBegin(EVENTLOG_HOST_FIELD_DESC); + oprot.writeString(struct.eventlog_host); + oprot.writeFieldEnd(); + } + } + if (struct.is_set_eventlog_port()) { + oprot.writeFieldBegin(EVENTLOG_PORT_FIELD_DESC); + oprot.writeI32(struct.eventlog_port); + oprot.writeFieldEnd(); + } + if (struct.debug_options != null) { + if (struct.is_set_debug_options()) { + oprot.writeFieldBegin(DEBUG_OPTIONS_FIELD_DESC); + struct.debug_options.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.topology_status != null) { + if (struct.is_set_topology_status()) { + oprot.writeFieldBegin(TOPOLOGY_STATUS_FIELD_DESC); + oprot.writeString(struct.topology_status); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ComponentPageInfoTupleSchemeFactory implements SchemeFactory { + public ComponentPageInfoTupleScheme getScheme() { + return new ComponentPageInfoTupleScheme(); + } + } + + private static class ComponentPageInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ComponentPageInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.component_id); + oprot.writeI32(struct.component_type.getValue()); + BitSet optionals = new BitSet(); + if (struct.is_set_topology_id()) { + optionals.set(0); + } + if (struct.is_set_topology_name()) { + optionals.set(1); + } + if (struct.is_set_num_executors()) { + optionals.set(2); + } + if (struct.is_set_num_tasks()) { + optionals.set(3); + } + if (struct.is_set_window_to_stats()) { + optionals.set(4); + } + if (struct.is_set_gsid_to_input_stats()) { + optionals.set(5); + } + if (struct.is_set_sid_to_output_stats()) { + optionals.set(6); + } + if (struct.is_set_exec_stats()) { + optionals.set(7); + } + if (struct.is_set_errors()) { + optionals.set(8); + } + if (struct.is_set_eventlog_host()) { + optionals.set(9); + } + if (struct.is_set_eventlog_port()) { + optionals.set(10); + } + if (struct.is_set_debug_options()) { + optionals.set(11); + } + if (struct.is_set_topology_status()) { + optionals.set(12); + } + oprot.writeBitSet(optionals, 13); + if (struct.is_set_topology_id()) { + oprot.writeString(struct.topology_id); + } + if (struct.is_set_topology_name()) { + oprot.writeString(struct.topology_name); + } + if (struct.is_set_num_executors()) { + oprot.writeI32(struct.num_executors); + } + if (struct.is_set_num_tasks()) { + oprot.writeI32(struct.num_tasks); + } + if (struct.is_set_window_to_stats()) { + { + oprot.writeI32(struct.window_to_stats.size()); + for (Map.Entry _iter427 : struct.window_to_stats.entrySet()) + { + oprot.writeString(_iter427.getKey()); + _iter427.getValue().write(oprot); + } + } + } + if (struct.is_set_gsid_to_input_stats()) { + { + oprot.writeI32(struct.gsid_to_input_stats.size()); + for (Map.Entry _iter428 : struct.gsid_to_input_stats.entrySet()) + { + _iter428.getKey().write(oprot); + _iter428.getValue().write(oprot); + } + } + } + if (struct.is_set_sid_to_output_stats()) { + { + oprot.writeI32(struct.sid_to_output_stats.size()); + for (Map.Entry _iter429 : struct.sid_to_output_stats.entrySet()) + { + oprot.writeString(_iter429.getKey()); + _iter429.getValue().write(oprot); + } + } + } + if (struct.is_set_exec_stats()) { + { + oprot.writeI32(struct.exec_stats.size()); + for (ExecutorAggregateStats _iter430 : struct.exec_stats) + { + _iter430.write(oprot); + } + } + } + if (struct.is_set_errors()) { + { + oprot.writeI32(struct.errors.size()); + for (ErrorInfo _iter431 : struct.errors) + { + _iter431.write(oprot); + } + } + } + if (struct.is_set_eventlog_host()) { + oprot.writeString(struct.eventlog_host); + } + if (struct.is_set_eventlog_port()) { + oprot.writeI32(struct.eventlog_port); + } + if (struct.is_set_debug_options()) { + struct.debug_options.write(oprot); + } + if (struct.is_set_topology_status()) { + oprot.writeString(struct.topology_status); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ComponentPageInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.component_id = iprot.readString(); + struct.set_component_id_isSet(true); + struct.component_type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32()); + struct.set_component_type_isSet(true); + BitSet incoming = iprot.readBitSet(13); + if (incoming.get(0)) { + struct.topology_id = iprot.readString(); + struct.set_topology_id_isSet(true); + } + if (incoming.get(1)) { + struct.topology_name = iprot.readString(); + struct.set_topology_name_isSet(true); + } + if (incoming.get(2)) { + struct.num_executors = iprot.readI32(); + struct.set_num_executors_isSet(true); + } + if (incoming.get(3)) { + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + } + if (incoming.get(4)) { + { + org.apache.thrift.protocol.TMap _map432 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.window_to_stats = new HashMap(2*_map432.size); + String _key433; + ComponentAggregateStats _val434; + for (int _i435 = 0; _i435 < _map432.size; ++_i435) + { + _key433 = iprot.readString(); + _val434 = new ComponentAggregateStats(); + _val434.read(iprot); + struct.window_to_stats.put(_key433, _val434); + } + } + struct.set_window_to_stats_isSet(true); + } + if (incoming.get(5)) { + { + org.apache.thrift.protocol.TMap _map436 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.gsid_to_input_stats = new HashMap(2*_map436.size); + GlobalStreamId _key437; + ComponentAggregateStats _val438; + for (int _i439 = 0; _i439 < _map436.size; ++_i439) + { + _key437 = new GlobalStreamId(); + _key437.read(iprot); + _val438 = new ComponentAggregateStats(); + _val438.read(iprot); + struct.gsid_to_input_stats.put(_key437, _val438); + } + } + struct.set_gsid_to_input_stats_isSet(true); + } + if (incoming.get(6)) { + { + org.apache.thrift.protocol.TMap _map440 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.sid_to_output_stats = new HashMap(2*_map440.size); + String _key441; + ComponentAggregateStats _val442; + for (int _i443 = 0; _i443 < _map440.size; ++_i443) + { + _key441 = iprot.readString(); + _val442 = new ComponentAggregateStats(); + _val442.read(iprot); + struct.sid_to_output_stats.put(_key441, _val442); + } + } + struct.set_sid_to_output_stats_isSet(true); + } + if (incoming.get(7)) { + { + org.apache.thrift.protocol.TList _list444 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.exec_stats = new ArrayList(_list444.size); + ExecutorAggregateStats _elem445; + for (int _i446 = 0; _i446 < _list444.size; ++_i446) + { + _elem445 = new ExecutorAggregateStats(); + _elem445.read(iprot); + struct.exec_stats.add(_elem445); + } + } + struct.set_exec_stats_isSet(true); + } + if (incoming.get(8)) { + { + org.apache.thrift.protocol.TList _list447 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.errors = new ArrayList(_list447.size); + ErrorInfo _elem448; + for (int _i449 = 0; _i449 < _list447.size; ++_i449) + { + _elem448 = new ErrorInfo(); + _elem448.read(iprot); + struct.errors.add(_elem448); + } + } + struct.set_errors_isSet(true); + } + if (incoming.get(9)) { + struct.eventlog_host = iprot.readString(); + struct.set_eventlog_host_isSet(true); + } + if (incoming.get(10)) { + struct.eventlog_port = iprot.readI32(); + struct.set_eventlog_port_isSet(true); + } + if (incoming.get(11)) { + struct.debug_options = new DebugOptions(); + struct.debug_options.read(iprot); + struct.set_debug_options_isSet(true); + } + if (incoming.get(12)) { + struct.topology_status = iprot.readString(); + struct.set_topology_status_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentType.java b/storm-core/src/jvm/backtype/storm/generated/ComponentType.java new file mode 100644 index 00000000000..9efdba44dc5 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/ComponentType.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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum ComponentType implements org.apache.thrift.TEnum { + BOLT(1), + SPOUT(2); + + private final int value; + + private ComponentType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static ComponentType findByValue(int value) { + switch (value) { + case 1: + return BOLT; + case 2: + return SPOUT; + default: + return null; + } + } +} diff --git a/storm-core/src/jvm/backtype/storm/generated/Credentials.java b/storm-core/src/jvm/backtype/storm/generated/Credentials.java index 8484b18bd94..b7d0ee699b1 100644 --- a/storm-core/src/jvm/backtype/storm/generated/Credentials.java +++ b/storm-core/src/jvm/backtype/storm/generated/Credentials.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class Credentials implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials"); @@ -365,15 +365,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Credentials struct) case 1: // CREDS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map344 = iprot.readMapBegin(); - struct.creds = new HashMap(2*_map344.size); - String _key345; - String _val346; - for (int _i347 = 0; _i347 < _map344.size; ++_i347) + org.apache.thrift.protocol.TMap _map460 = iprot.readMapBegin(); + struct.creds = new HashMap(2*_map460.size); + String _key461; + String _val462; + for (int _i463 = 0; _i463 < _map460.size; ++_i463) { - _key345 = iprot.readString(); - _val346 = iprot.readString(); - struct.creds.put(_key345, _val346); + _key461 = iprot.readString(); + _val462 = iprot.readString(); + struct.creds.put(_key461, _val462); } iprot.readMapEnd(); } @@ -399,10 +399,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, Credentials struct oprot.writeFieldBegin(CREDS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.creds.size())); - for (Map.Entry _iter348 : struct.creds.entrySet()) + for (Map.Entry _iter464 : struct.creds.entrySet()) { - oprot.writeString(_iter348.getKey()); - oprot.writeString(_iter348.getValue()); + oprot.writeString(_iter464.getKey()); + oprot.writeString(_iter464.getValue()); } oprot.writeMapEnd(); } @@ -427,10 +427,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, Credentials struct) TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.creds.size()); - for (Map.Entry _iter349 : struct.creds.entrySet()) + for (Map.Entry _iter465 : struct.creds.entrySet()) { - oprot.writeString(_iter349.getKey()); - oprot.writeString(_iter349.getValue()); + oprot.writeString(_iter465.getKey()); + oprot.writeString(_iter465.getValue()); } } } @@ -439,15 +439,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, Credentials struct) public void read(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; { - org.apache.thrift.protocol.TMap _map350 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.creds = new HashMap(2*_map350.size); - String _key351; - String _val352; - for (int _i353 = 0; _i353 < _map350.size; ++_i353) + org.apache.thrift.protocol.TMap _map466 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.creds = new HashMap(2*_map466.size); + String _key467; + String _val468; + for (int _i469 = 0; _i469 < _map466.size; ++_i469) { - _key351 = iprot.readString(); - _val352 = iprot.readString(); - struct.creds.put(_key351, _val352); + _key467 = iprot.readString(); + _val468 = iprot.readString(); + struct.creds.put(_key467, _val468); } } struct.set_creds_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java new file mode 100644 index 00000000000..629caa600d9 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java @@ -0,0 +1,526 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22") +public class ExecutorAggregateStats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorAggregateStats"); + + private static final org.apache.thrift.protocol.TField EXEC_SUMMARY_FIELD_DESC = new org.apache.thrift.protocol.TField("exec_summary", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("stats", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecutorAggregateStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecutorAggregateStatsTupleSchemeFactory()); + } + + private ExecutorSummary exec_summary; // optional + private ComponentAggregateStats stats; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + EXEC_SUMMARY((short)1, "exec_summary"), + STATS((short)2, "stats"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // EXEC_SUMMARY + return EXEC_SUMMARY; + case 2: // STATS + return STATS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.EXEC_SUMMARY,_Fields.STATS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.EXEC_SUMMARY, new org.apache.thrift.meta_data.FieldMetaData("exec_summary", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSummary.class))); + tmpMap.put(_Fields.STATS, new org.apache.thrift.meta_data.FieldMetaData("stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorAggregateStats.class, metaDataMap); + } + + public ExecutorAggregateStats() { + } + + /** + * Performs a deep copy on other. + */ + public ExecutorAggregateStats(ExecutorAggregateStats other) { + if (other.is_set_exec_summary()) { + this.exec_summary = new ExecutorSummary(other.exec_summary); + } + if (other.is_set_stats()) { + this.stats = new ComponentAggregateStats(other.stats); + } + } + + public ExecutorAggregateStats deepCopy() { + return new ExecutorAggregateStats(this); + } + + @Override + public void clear() { + this.exec_summary = null; + this.stats = null; + } + + public ExecutorSummary get_exec_summary() { + return this.exec_summary; + } + + public void set_exec_summary(ExecutorSummary exec_summary) { + this.exec_summary = exec_summary; + } + + public void unset_exec_summary() { + this.exec_summary = null; + } + + /** Returns true if field exec_summary is set (has been assigned a value) and false otherwise */ + public boolean is_set_exec_summary() { + return this.exec_summary != null; + } + + public void set_exec_summary_isSet(boolean value) { + if (!value) { + this.exec_summary = null; + } + } + + public ComponentAggregateStats get_stats() { + return this.stats; + } + + public void set_stats(ComponentAggregateStats stats) { + this.stats = stats; + } + + public void unset_stats() { + this.stats = null; + } + + /** Returns true if field stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_stats() { + return this.stats != null; + } + + public void set_stats_isSet(boolean value) { + if (!value) { + this.stats = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EXEC_SUMMARY: + if (value == null) { + unset_exec_summary(); + } else { + set_exec_summary((ExecutorSummary)value); + } + break; + + case STATS: + if (value == null) { + unset_stats(); + } else { + set_stats((ComponentAggregateStats)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EXEC_SUMMARY: + return get_exec_summary(); + + case STATS: + return get_stats(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case EXEC_SUMMARY: + return is_set_exec_summary(); + case STATS: + return is_set_stats(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ExecutorAggregateStats) + return this.equals((ExecutorAggregateStats)that); + return false; + } + + public boolean equals(ExecutorAggregateStats that) { + if (that == null) + return false; + + boolean this_present_exec_summary = true && this.is_set_exec_summary(); + boolean that_present_exec_summary = true && that.is_set_exec_summary(); + if (this_present_exec_summary || that_present_exec_summary) { + if (!(this_present_exec_summary && that_present_exec_summary)) + return false; + if (!this.exec_summary.equals(that.exec_summary)) + return false; + } + + boolean this_present_stats = true && this.is_set_stats(); + boolean that_present_stats = true && that.is_set_stats(); + if (this_present_stats || that_present_stats) { + if (!(this_present_stats && that_present_stats)) + return false; + if (!this.stats.equals(that.stats)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_exec_summary = true && (is_set_exec_summary()); + list.add(present_exec_summary); + if (present_exec_summary) + list.add(exec_summary); + + boolean present_stats = true && (is_set_stats()); + list.add(present_stats); + if (present_stats) + list.add(stats); + + return list.hashCode(); + } + + @Override + public int compareTo(ExecutorAggregateStats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_exec_summary()).compareTo(other.is_set_exec_summary()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_exec_summary()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exec_summary, other.exec_summary); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_stats()).compareTo(other.is_set_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, other.stats); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ExecutorAggregateStats("); + boolean first = true; + + if (is_set_exec_summary()) { + sb.append("exec_summary:"); + if (this.exec_summary == null) { + sb.append("null"); + } else { + sb.append(this.exec_summary); + } + first = false; + } + if (is_set_stats()) { + if (!first) sb.append(", "); + sb.append("stats:"); + if (this.stats == null) { + sb.append("null"); + } else { + sb.append(this.stats); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (exec_summary != null) { + exec_summary.validate(); + } + if (stats != null) { + stats.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ExecutorAggregateStatsStandardSchemeFactory implements SchemeFactory { + public ExecutorAggregateStatsStandardScheme getScheme() { + return new ExecutorAggregateStatsStandardScheme(); + } + } + + private static class ExecutorAggregateStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorAggregateStats struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // EXEC_SUMMARY + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.exec_summary = new ExecutorSummary(); + struct.exec_summary.read(iprot); + struct.set_exec_summary_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STATS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.stats = new ComponentAggregateStats(); + struct.stats.read(iprot); + struct.set_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorAggregateStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.exec_summary != null) { + if (struct.is_set_exec_summary()) { + oprot.writeFieldBegin(EXEC_SUMMARY_FIELD_DESC); + struct.exec_summary.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.stats != null) { + if (struct.is_set_stats()) { + oprot.writeFieldBegin(STATS_FIELD_DESC); + struct.stats.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecutorAggregateStatsTupleSchemeFactory implements SchemeFactory { + public ExecutorAggregateStatsTupleScheme getScheme() { + return new ExecutorAggregateStatsTupleScheme(); + } + } + + private static class ExecutorAggregateStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_exec_summary()) { + optionals.set(0); + } + if (struct.is_set_stats()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_exec_summary()) { + struct.exec_summary.write(oprot); + } + if (struct.is_set_stats()) { + struct.stats.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.exec_summary = new ExecutorSummary(); + struct.exec_summary.read(iprot); + struct.set_exec_summary_isSet(true); + } + if (incoming.get(1)) { + struct.stats = new ComponentAggregateStats(); + struct.stats.read(iprot); + struct.set_stats_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java index fd9740b4e61..b7af5cad848 100644 --- a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java +++ b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class LSApprovedWorkers implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSApprovedWorkers"); @@ -365,15 +365,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, LSApprovedWorkers s case 1: // APPROVED_WORKERS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map482 = iprot.readMapBegin(); - struct.approved_workers = new HashMap(2*_map482.size); - String _key483; - int _val484; - for (int _i485 = 0; _i485 < _map482.size; ++_i485) + org.apache.thrift.protocol.TMap _map598 = iprot.readMapBegin(); + struct.approved_workers = new HashMap(2*_map598.size); + String _key599; + int _val600; + for (int _i601 = 0; _i601 < _map598.size; ++_i601) { - _key483 = iprot.readString(); - _val484 = iprot.readI32(); - struct.approved_workers.put(_key483, _val484); + _key599 = iprot.readString(); + _val600 = iprot.readI32(); + struct.approved_workers.put(_key599, _val600); } iprot.readMapEnd(); } @@ -399,10 +399,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSApprovedWorkers oprot.writeFieldBegin(APPROVED_WORKERS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.approved_workers.size())); - for (Map.Entry _iter486 : struct.approved_workers.entrySet()) + for (Map.Entry _iter602 : struct.approved_workers.entrySet()) { - oprot.writeString(_iter486.getKey()); - oprot.writeI32(_iter486.getValue()); + oprot.writeString(_iter602.getKey()); + oprot.writeI32(_iter602.getValue()); } oprot.writeMapEnd(); } @@ -427,10 +427,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers s TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.approved_workers.size()); - for (Map.Entry _iter487 : struct.approved_workers.entrySet()) + for (Map.Entry _iter603 : struct.approved_workers.entrySet()) { - oprot.writeString(_iter487.getKey()); - oprot.writeI32(_iter487.getValue()); + oprot.writeString(_iter603.getKey()); + oprot.writeI32(_iter603.getValue()); } } } @@ -439,15 +439,15 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers s public void read(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; { - org.apache.thrift.protocol.TMap _map488 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.approved_workers = new HashMap(2*_map488.size); - String _key489; - int _val490; - for (int _i491 = 0; _i491 < _map488.size; ++_i491) + org.apache.thrift.protocol.TMap _map604 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.approved_workers = new HashMap(2*_map604.size); + String _key605; + int _val606; + for (int _i607 = 0; _i607 < _map604.size; ++_i607) { - _key489 = iprot.readString(); - _val490 = iprot.readI32(); - struct.approved_workers.put(_key489, _val490); + _key605 = iprot.readString(); + _val606 = iprot.readI32(); + struct.approved_workers.put(_key605, _val606); } } struct.set_approved_workers_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java index 1d85ceca3c7..728a5332d3a 100644 --- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java +++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class LSSupervisorAssignments implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorAssignments"); @@ -376,16 +376,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, LSSupervisorAssignm case 1: // ASSIGNMENTS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map492 = iprot.readMapBegin(); - struct.assignments = new HashMap(2*_map492.size); - int _key493; - LocalAssignment _val494; - for (int _i495 = 0; _i495 < _map492.size; ++_i495) + org.apache.thrift.protocol.TMap _map608 = iprot.readMapBegin(); + struct.assignments = new HashMap(2*_map608.size); + int _key609; + LocalAssignment _val610; + for (int _i611 = 0; _i611 < _map608.size; ++_i611) { - _key493 = iprot.readI32(); - _val494 = new LocalAssignment(); - _val494.read(iprot); - struct.assignments.put(_key493, _val494); + _key609 = iprot.readI32(); + _val610 = new LocalAssignment(); + _val610.read(iprot); + struct.assignments.put(_key609, _val610); } iprot.readMapEnd(); } @@ -411,10 +411,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSSupervisorAssign oprot.writeFieldBegin(ASSIGNMENTS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.assignments.size())); - for (Map.Entry _iter496 : struct.assignments.entrySet()) + for (Map.Entry _iter612 : struct.assignments.entrySet()) { - oprot.writeI32(_iter496.getKey()); - _iter496.getValue().write(oprot); + oprot.writeI32(_iter612.getKey()); + _iter612.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -439,10 +439,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignm TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.assignments.size()); - for (Map.Entry _iter497 : struct.assignments.entrySet()) + for (Map.Entry _iter613 : struct.assignments.entrySet()) { - oprot.writeI32(_iter497.getKey()); - _iter497.getValue().write(oprot); + oprot.writeI32(_iter613.getKey()); + _iter613.getValue().write(oprot); } } } @@ -451,16 +451,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignm public void read(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignments struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; { - org.apache.thrift.protocol.TMap _map498 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.assignments = new HashMap(2*_map498.size); - int _key499; - LocalAssignment _val500; - for (int _i501 = 0; _i501 < _map498.size; ++_i501) + org.apache.thrift.protocol.TMap _map614 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.assignments = new HashMap(2*_map614.size); + int _key615; + LocalAssignment _val616; + for (int _i617 = 0; _i617 < _map614.size; ++_i617) { - _key499 = iprot.readI32(); - _val500 = new LocalAssignment(); - _val500.read(iprot); - struct.assignments.put(_key499, _val500); + _key615 = iprot.readI32(); + _val616 = new LocalAssignment(); + _val616.read(iprot); + struct.assignments.put(_key615, _val616); } } struct.set_assignments_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java index 98726cc660b..6c88563f5e4 100644 --- a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java +++ b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class LSWorkerHeartbeat implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSWorkerHeartbeat"); @@ -638,14 +638,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, LSWorkerHeartbeat s case 3: // EXECUTORS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list502 = iprot.readListBegin(); - struct.executors = new ArrayList(_list502.size); - ExecutorInfo _elem503; - for (int _i504 = 0; _i504 < _list502.size; ++_i504) + org.apache.thrift.protocol.TList _list618 = iprot.readListBegin(); + struct.executors = new ArrayList(_list618.size); + ExecutorInfo _elem619; + for (int _i620 = 0; _i620 < _list618.size; ++_i620) { - _elem503 = new ExecutorInfo(); - _elem503.read(iprot); - struct.executors.add(_elem503); + _elem619 = new ExecutorInfo(); + _elem619.read(iprot); + struct.executors.add(_elem619); } iprot.readListEnd(); } @@ -687,9 +687,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LSWorkerHeartbeat oprot.writeFieldBegin(EXECUTORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size())); - for (ExecutorInfo _iter505 : struct.executors) + for (ExecutorInfo _iter621 : struct.executors) { - _iter505.write(oprot); + _iter621.write(oprot); } oprot.writeListEnd(); } @@ -719,9 +719,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LSWorkerHeartbeat s oprot.writeString(struct.topology_id); { oprot.writeI32(struct.executors.size()); - for (ExecutorInfo _iter506 : struct.executors) + for (ExecutorInfo _iter622 : struct.executors) { - _iter506.write(oprot); + _iter622.write(oprot); } } oprot.writeI32(struct.port); @@ -735,14 +735,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, LSWorkerHeartbeat st struct.topology_id = iprot.readString(); struct.set_topology_id_isSet(true); { - org.apache.thrift.protocol.TList _list507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.executors = new ArrayList(_list507.size); - ExecutorInfo _elem508; - for (int _i509 = 0; _i509 < _list507.size; ++_i509) + org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executors = new ArrayList(_list623.size); + ExecutorInfo _elem624; + for (int _i625 = 0; _i625 < _list623.size; ++_i625) { - _elem508 = new ExecutorInfo(); - _elem508.read(iprot); - struct.executors.add(_elem508); + _elem624 = new ExecutorInfo(); + _elem624.read(iprot); + struct.executors.add(_elem624); } } struct.set_executors_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java index d985cba51be..65068bc7cd2 100644 --- a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java +++ b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class LocalAssignment implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalAssignment"); @@ -464,14 +464,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, LocalAssignment str case 2: // EXECUTORS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list474 = iprot.readListBegin(); - struct.executors = new ArrayList(_list474.size); - ExecutorInfo _elem475; - for (int _i476 = 0; _i476 < _list474.size; ++_i476) + org.apache.thrift.protocol.TList _list590 = iprot.readListBegin(); + struct.executors = new ArrayList(_list590.size); + ExecutorInfo _elem591; + for (int _i592 = 0; _i592 < _list590.size; ++_i592) { - _elem475 = new ExecutorInfo(); - _elem475.read(iprot); - struct.executors.add(_elem475); + _elem591 = new ExecutorInfo(); + _elem591.read(iprot); + struct.executors.add(_elem591); } iprot.readListEnd(); } @@ -502,9 +502,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LocalAssignment st oprot.writeFieldBegin(EXECUTORS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size())); - for (ExecutorInfo _iter477 : struct.executors) + for (ExecutorInfo _iter593 : struct.executors) { - _iter477.write(oprot); + _iter593.write(oprot); } oprot.writeListEnd(); } @@ -530,9 +530,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LocalAssignment str oprot.writeString(struct.topology_id); { oprot.writeI32(struct.executors.size()); - for (ExecutorInfo _iter478 : struct.executors) + for (ExecutorInfo _iter594 : struct.executors) { - _iter478.write(oprot); + _iter594.write(oprot); } } } @@ -543,14 +543,14 @@ public void read(org.apache.thrift.protocol.TProtocol prot, LocalAssignment stru struct.topology_id = iprot.readString(); struct.set_topology_id_isSet(true); { - org.apache.thrift.protocol.TList _list479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.executors = new ArrayList(_list479.size); - ExecutorInfo _elem480; - for (int _i481 = 0; _i481 < _list479.size; ++_i481) + org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.executors = new ArrayList(_list595.size); + ExecutorInfo _elem596; + for (int _i597 = 0; _i597 < _list595.size; ++_i597) { - _elem480 = new ExecutorInfo(); - _elem480.read(iprot); - struct.executors.add(_elem480); + _elem596 = new ExecutorInfo(); + _elem596.read(iprot); + struct.executors.add(_elem596); } } struct.set_executors_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java index 0ce2d7a0c20..1589379f8e7 100644 --- a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java +++ b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class LocalStateData implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalStateData"); @@ -376,16 +376,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, LocalStateData stru case 1: // SERIALIZED_PARTS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map464 = iprot.readMapBegin(); - struct.serialized_parts = new HashMap(2*_map464.size); - String _key465; - ThriftSerializedObject _val466; - for (int _i467 = 0; _i467 < _map464.size; ++_i467) + org.apache.thrift.protocol.TMap _map580 = iprot.readMapBegin(); + struct.serialized_parts = new HashMap(2*_map580.size); + String _key581; + ThriftSerializedObject _val582; + for (int _i583 = 0; _i583 < _map580.size; ++_i583) { - _key465 = iprot.readString(); - _val466 = new ThriftSerializedObject(); - _val466.read(iprot); - struct.serialized_parts.put(_key465, _val466); + _key581 = iprot.readString(); + _val582 = new ThriftSerializedObject(); + _val582.read(iprot); + struct.serialized_parts.put(_key581, _val582); } iprot.readMapEnd(); } @@ -411,10 +411,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LocalStateData str oprot.writeFieldBegin(SERIALIZED_PARTS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.serialized_parts.size())); - for (Map.Entry _iter468 : struct.serialized_parts.entrySet()) + for (Map.Entry _iter584 : struct.serialized_parts.entrySet()) { - oprot.writeString(_iter468.getKey()); - _iter468.getValue().write(oprot); + oprot.writeString(_iter584.getKey()); + _iter584.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -439,10 +439,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LocalStateData stru TTupleProtocol oprot = (TTupleProtocol) prot; { oprot.writeI32(struct.serialized_parts.size()); - for (Map.Entry _iter469 : struct.serialized_parts.entrySet()) + for (Map.Entry _iter585 : struct.serialized_parts.entrySet()) { - oprot.writeString(_iter469.getKey()); - _iter469.getValue().write(oprot); + oprot.writeString(_iter585.getKey()); + _iter585.getValue().write(oprot); } } } @@ -451,16 +451,16 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LocalStateData stru public void read(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException { TTupleProtocol iprot = (TTupleProtocol) prot; { - org.apache.thrift.protocol.TMap _map470 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.serialized_parts = new HashMap(2*_map470.size); - String _key471; - ThriftSerializedObject _val472; - for (int _i473 = 0; _i473 < _map470.size; ++_i473) + org.apache.thrift.protocol.TMap _map586 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.serialized_parts = new HashMap(2*_map586.size); + String _key587; + ThriftSerializedObject _val588; + for (int _i589 = 0; _i589 < _map586.size; ++_i589) { - _key471 = iprot.readString(); - _val472 = new ThriftSerializedObject(); - _val472.read(iprot); - struct.serialized_parts.put(_key471, _val472); + _key587 = iprot.readString(); + _val588 = new ThriftSerializedObject(); + _val588.read(iprot); + struct.serialized_parts.put(_key587, _val588); } } struct.set_serialized_parts_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java index 318ed86433f..1420db418be 100644 --- a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java +++ b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-24") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-5") public class LogConfig implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LogConfig"); @@ -368,16 +368,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, LogConfig struct) t case 2: // NAMED_LOGGER_LEVEL if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map510 = iprot.readMapBegin(); - struct.named_logger_level = new HashMap(2*_map510.size); - String _key511; - LogLevel _val512; - for (int _i513 = 0; _i513 < _map510.size; ++_i513) + org.apache.thrift.protocol.TMap _map626 = iprot.readMapBegin(); + struct.named_logger_level = new HashMap(2*_map626.size); + String _key627; + LogLevel _val628; + for (int _i629 = 0; _i629 < _map626.size; ++_i629) { - _key511 = iprot.readString(); - _val512 = new LogLevel(); - _val512.read(iprot); - struct.named_logger_level.put(_key511, _val512); + _key627 = iprot.readString(); + _val628 = new LogLevel(); + _val628.read(iprot); + struct.named_logger_level.put(_key627, _val628); } iprot.readMapEnd(); } @@ -404,10 +404,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, LogConfig struct) oprot.writeFieldBegin(NAMED_LOGGER_LEVEL_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.named_logger_level.size())); - for (Map.Entry _iter514 : struct.named_logger_level.entrySet()) + for (Map.Entry _iter630 : struct.named_logger_level.entrySet()) { - oprot.writeString(_iter514.getKey()); - _iter514.getValue().write(oprot); + oprot.writeString(_iter630.getKey()); + _iter630.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -439,10 +439,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) t if (struct.is_set_named_logger_level()) { { oprot.writeI32(struct.named_logger_level.size()); - for (Map.Entry _iter515 : struct.named_logger_level.entrySet()) + for (Map.Entry _iter631 : struct.named_logger_level.entrySet()) { - oprot.writeString(_iter515.getKey()); - _iter515.getValue().write(oprot); + oprot.writeString(_iter631.getKey()); + _iter631.getValue().write(oprot); } } } @@ -454,16 +454,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) th BitSet incoming = iprot.readBitSet(1); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map516 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.named_logger_level = new HashMap(2*_map516.size); - String _key517; - LogLevel _val518; - for (int _i519 = 0; _i519 < _map516.size; ++_i519) + org.apache.thrift.protocol.TMap _map632 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.named_logger_level = new HashMap(2*_map632.size); + String _key633; + LogLevel _val634; + for (int _i635 = 0; _i635 < _map632.size; ++_i635) { - _key517 = iprot.readString(); - _val518 = new LogLevel(); - _val518.read(iprot); - struct.named_logger_level.put(_key517, _val518); + _key633 = iprot.readString(); + _val634 = new LogLevel(); + _val634.read(iprot); + struct.named_logger_level.put(_key633, _val634); } } struct.set_named_logger_level_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java index 1d0114e4edb..4632059c7d3 100644 --- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java +++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-24") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-5") public class Nimbus { public interface Iface { @@ -108,6 +108,10 @@ public interface Iface { public TopologyInfo getTopologyInfoWithOpts(String id, GetInfoOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; + public TopologyPageInfo getTopologyPageInfo(String id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; + + public ComponentPageInfo getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; + public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException; /** @@ -168,6 +172,10 @@ public interface AsyncIface { public void getTopologyInfoWithOpts(String id, GetInfoOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getTopologyPageInfo(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; public void getTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; @@ -732,6 +740,69 @@ public TopologyInfo recv_getTopologyInfoWithOpts() throws NotAliveException, Aut throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result"); } + public TopologyPageInfo getTopologyPageInfo(String id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException + { + send_getTopologyPageInfo(id, window, is_include_sys); + return recv_getTopologyPageInfo(); + } + + public void send_getTopologyPageInfo(String id, String window, boolean is_include_sys) throws org.apache.thrift.TException + { + getTopologyPageInfo_args args = new getTopologyPageInfo_args(); + args.set_id(id); + args.set_window(window); + args.set_is_include_sys(is_include_sys); + sendBase("getTopologyPageInfo", args); + } + + public TopologyPageInfo recv_getTopologyPageInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException + { + getTopologyPageInfo_result result = new getTopologyPageInfo_result(); + receiveBase(result, "getTopologyPageInfo"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + if (result.aze != null) { + throw result.aze; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyPageInfo failed: unknown result"); + } + + public ComponentPageInfo getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException + { + send_getComponentPageInfo(topology_id, component_id, window, is_include_sys); + return recv_getComponentPageInfo(); + } + + public void send_getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws org.apache.thrift.TException + { + getComponentPageInfo_args args = new getComponentPageInfo_args(); + args.set_topology_id(topology_id); + args.set_component_id(component_id); + args.set_window(window); + args.set_is_include_sys(is_include_sys); + sendBase("getComponentPageInfo", args); + } + + public ComponentPageInfo recv_getComponentPageInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException + { + getComponentPageInfo_result result = new getComponentPageInfo_result(); + receiveBase(result, "getComponentPageInfo"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + if (result.aze != null) { + throw result.aze; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getComponentPageInfo failed: unknown result"); + } + public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException { send_getTopologyConf(id); @@ -1516,6 +1587,85 @@ public TopologyInfo getResult() throws NotAliveException, AuthorizationException } } + public void getTopologyPageInfo(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getTopologyPageInfo_call method_call = new getTopologyPageInfo_call(id, window, is_include_sys, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getTopologyPageInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + private String window; + private boolean is_include_sys; + public getTopologyPageInfo_call(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + this.window = window; + this.is_include_sys = is_include_sys; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyPageInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTopologyPageInfo_args args = new getTopologyPageInfo_args(); + args.set_id(id); + args.set_window(window); + args.set_is_include_sys(is_include_sys); + args.write(prot); + prot.writeMessageEnd(); + } + + public TopologyPageInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getTopologyPageInfo(); + } + } + + public void getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getComponentPageInfo_call method_call = new getComponentPageInfo_call(topology_id, component_id, window, is_include_sys, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getComponentPageInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + private String topology_id; + private String component_id; + private String window; + private boolean is_include_sys; + public getComponentPageInfo_call(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.topology_id = topology_id; + this.component_id = component_id; + this.window = window; + this.is_include_sys = is_include_sys; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getComponentPageInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getComponentPageInfo_args args = new getComponentPageInfo_args(); + args.set_topology_id(topology_id); + args.set_component_id(component_id); + args.set_window(window); + args.set_is_include_sys(is_include_sys); + args.write(prot); + prot.writeMessageEnd(); + } + + public ComponentPageInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getComponentPageInfo(); + } + } + public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { checkReady(); getTopologyConf_call method_call = new getTopologyConf_call(id, resultHandler, this, ___protocolFactory, ___transport); @@ -1645,6 +1795,8 @@ protected Processor(I iface, Map extends org.apache.thrift.ProcessFunction { + public getTopologyPageInfo() { + super("getTopologyPageInfo"); + } + + public getTopologyPageInfo_args getEmptyArgsInstance() { + return new getTopologyPageInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public getTopologyPageInfo_result getResult(I iface, getTopologyPageInfo_args args) throws org.apache.thrift.TException { + getTopologyPageInfo_result result = new getTopologyPageInfo_result(); + try { + result.success = iface.getTopologyPageInfo(args.id, args.window, args.is_include_sys); + } catch (NotAliveException e) { + result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + + public static class getComponentPageInfo extends org.apache.thrift.ProcessFunction { + public getComponentPageInfo() { + super("getComponentPageInfo"); + } + + public getComponentPageInfo_args getEmptyArgsInstance() { + return new getComponentPageInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public getComponentPageInfo_result getResult(I iface, getComponentPageInfo_args args) throws org.apache.thrift.TException { + getComponentPageInfo_result result = new getComponentPageInfo_result(); + try { + result.success = iface.getComponentPageInfo(args.topology_id, args.component_id, args.window, args.is_include_sys); + } catch (NotAliveException e) { + result.e = e; + } catch (AuthorizationException aze) { + result.aze = aze; + } + return result; + } + } + public static class getTopologyConf extends org.apache.thrift.ProcessFunction { public getTopologyConf() { super("getTopologyConf"); @@ -2264,6 +2468,8 @@ protected AsyncProcessor(I iface, Map extends org.apache.thrift.AsyncProcessFunction { + public getTopologyPageInfo() { + super("getTopologyPageInfo"); + } + + public getTopologyPageInfo_args getEmptyArgsInstance() { + return new getTopologyPageInfo_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TopologyPageInfo o) { + getTopologyPageInfo_result result = new getTopologyPageInfo_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTopologyPageInfo_result result = new getTopologyPageInfo_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTopologyPageInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getTopologyPageInfo(args.id, args.window, args.is_include_sys,resultHandler); + } + } + + public static class getComponentPageInfo extends org.apache.thrift.AsyncProcessFunction { + public getComponentPageInfo() { + super("getComponentPageInfo"); + } + + public getComponentPageInfo_args getEmptyArgsInstance() { + return new getComponentPageInfo_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(ComponentPageInfo o) { + getComponentPageInfo_result result = new getComponentPageInfo_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getComponentPageInfo_result result = new getComponentPageInfo_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof AuthorizationException) { + result.aze = (AuthorizationException) e; + result.set_aze_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getComponentPageInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getComponentPageInfo(args.topology_id, args.component_id, args.window, args.is_include_sys,resultHandler); + } + } + public static class getTopologyConf extends org.apache.thrift.AsyncProcessFunction { public getTopologyConf() { super("getTopologyConf"); @@ -21609,6 +21939,2388 @@ public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyInfoWithO } + public static class getTopologyPageInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyPageInfo_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField WINDOW_FIELD_DESC = new org.apache.thrift.protocol.TField("window", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField IS_INCLUDE_SYS_FIELD_DESC = new org.apache.thrift.protocol.TField("is_include_sys", org.apache.thrift.protocol.TType.BOOL, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyPageInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyPageInfo_argsTupleSchemeFactory()); + } + + private String id; // required + private String window; // required + private boolean is_include_sys; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"), + WINDOW((short)2, "window"), + IS_INCLUDE_SYS((short)3, "is_include_sys"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + case 2: // WINDOW + return WINDOW; + case 3: // IS_INCLUDE_SYS + return IS_INCLUDE_SYS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __IS_INCLUDE_SYS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.WINDOW, new org.apache.thrift.meta_data.FieldMetaData("window", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.IS_INCLUDE_SYS, new org.apache.thrift.meta_data.FieldMetaData("is_include_sys", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyPageInfo_args.class, metaDataMap); + } + + public getTopologyPageInfo_args() { + } + + public getTopologyPageInfo_args( + String id, + String window, + boolean is_include_sys) + { + this(); + this.id = id; + this.window = window; + this.is_include_sys = is_include_sys; + set_is_include_sys_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public getTopologyPageInfo_args(getTopologyPageInfo_args other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_id()) { + this.id = other.id; + } + if (other.is_set_window()) { + this.window = other.window; + } + this.is_include_sys = other.is_include_sys; + } + + public getTopologyPageInfo_args deepCopy() { + return new getTopologyPageInfo_args(this); + } + + @Override + public void clear() { + this.id = null; + this.window = null; + set_is_include_sys_isSet(false); + this.is_include_sys = false; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public String get_window() { + return this.window; + } + + public void set_window(String window) { + this.window = window; + } + + public void unset_window() { + this.window = null; + } + + /** Returns true if field window is set (has been assigned a value) and false otherwise */ + public boolean is_set_window() { + return this.window != null; + } + + public void set_window_isSet(boolean value) { + if (!value) { + this.window = null; + } + } + + public boolean is_is_include_sys() { + return this.is_include_sys; + } + + public void set_is_include_sys(boolean is_include_sys) { + this.is_include_sys = is_include_sys; + set_is_include_sys_isSet(true); + } + + public void unset_is_include_sys() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID); + } + + /** Returns true if field is_include_sys is set (has been assigned a value) and false otherwise */ + public boolean is_set_is_include_sys() { + return EncodingUtils.testBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID); + } + + public void set_is_include_sys_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + case WINDOW: + if (value == null) { + unset_window(); + } else { + set_window((String)value); + } + break; + + case IS_INCLUDE_SYS: + if (value == null) { + unset_is_include_sys(); + } else { + set_is_include_sys((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + case WINDOW: + return get_window(); + + case IS_INCLUDE_SYS: + return Boolean.valueOf(is_is_include_sys()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + case WINDOW: + return is_set_window(); + case IS_INCLUDE_SYS: + return is_set_is_include_sys(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyPageInfo_args) + return this.equals((getTopologyPageInfo_args)that); + return false; + } + + public boolean equals(getTopologyPageInfo_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + boolean this_present_window = true && this.is_set_window(); + boolean that_present_window = true && that.is_set_window(); + if (this_present_window || that_present_window) { + if (!(this_present_window && that_present_window)) + return false; + if (!this.window.equals(that.window)) + return false; + } + + boolean this_present_is_include_sys = true; + boolean that_present_is_include_sys = true; + if (this_present_is_include_sys || that_present_is_include_sys) { + if (!(this_present_is_include_sys && that_present_is_include_sys)) + return false; + if (this.is_include_sys != that.is_include_sys) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + boolean present_window = true && (is_set_window()); + list.add(present_window); + if (present_window) + list.add(window); + + boolean present_is_include_sys = true; + list.add(present_is_include_sys); + if (present_is_include_sys) + list.add(is_include_sys); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyPageInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_window()).compareTo(other.is_set_window()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window, other.window); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_is_include_sys()).compareTo(other.is_set_is_include_sys()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_is_include_sys()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.is_include_sys, other.is_include_sys); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyPageInfo_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + if (!first) sb.append(", "); + sb.append("window:"); + if (this.window == null) { + sb.append("null"); + } else { + sb.append(this.window); + } + first = false; + if (!first) sb.append(", "); + sb.append("is_include_sys:"); + sb.append(this.is_include_sys); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyPageInfo_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyPageInfo_argsStandardScheme getScheme() { + return new getTopologyPageInfo_argsStandardScheme(); + } + } + + private static class getTopologyPageInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // WINDOW + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.window = iprot.readString(); + struct.set_window_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // IS_INCLUDE_SYS + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.is_include_sys = iprot.readBool(); + struct.set_is_include_sys_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + if (struct.window != null) { + oprot.writeFieldBegin(WINDOW_FIELD_DESC); + oprot.writeString(struct.window); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(IS_INCLUDE_SYS_FIELD_DESC); + oprot.writeBool(struct.is_include_sys); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyPageInfo_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyPageInfo_argsTupleScheme getScheme() { + return new getTopologyPageInfo_argsTupleScheme(); + } + } + + private static class getTopologyPageInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + if (struct.is_set_window()) { + optionals.set(1); + } + if (struct.is_set_is_include_sys()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + if (struct.is_set_window()) { + oprot.writeString(struct.window); + } + if (struct.is_set_is_include_sys()) { + oprot.writeBool(struct.is_include_sys); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + if (incoming.get(1)) { + struct.window = iprot.readString(); + struct.set_window_isSet(true); + } + if (incoming.get(2)) { + struct.is_include_sys = iprot.readBool(); + struct.set_is_include_sys_isSet(true); + } + } + } + + } + + public static class getTopologyPageInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyPageInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyPageInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyPageInfo_resultTupleSchemeFactory()); + } + + private TopologyPageInfo success; // required + private NotAliveException e; // required + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"), + AZE((short)2, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + case 2: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyPageInfo.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyPageInfo_result.class, metaDataMap); + } + + public getTopologyPageInfo_result() { + } + + public getTopologyPageInfo_result( + TopologyPageInfo success, + NotAliveException e, + AuthorizationException aze) + { + this(); + this.success = success; + this.e = e; + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyPageInfo_result(getTopologyPageInfo_result other) { + if (other.is_set_success()) { + this.success = new TopologyPageInfo(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public getTopologyPageInfo_result deepCopy() { + return new getTopologyPageInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + this.aze = null; + } + + public TopologyPageInfo get_success() { + return this.success; + } + + public void set_success(TopologyPageInfo success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((TopologyPageInfo)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyPageInfo_result) + return this.equals((getTopologyPageInfo_result)that); + return false; + } + + public boolean equals(getTopologyPageInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyPageInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyPageInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyPageInfo_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyPageInfo_resultStandardScheme getScheme() { + return new getTopologyPageInfo_resultStandardScheme(); + } + } + + private static class getTopologyPageInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TopologyPageInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyPageInfo_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyPageInfo_resultTupleScheme getScheme() { + return new getTopologyPageInfo_resultTupleScheme(); + } + } + + private static class getTopologyPageInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + if (struct.is_set_aze()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.success = new TopologyPageInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(2)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + + public static class getComponentPageInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getComponentPageInfo_args"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField WINDOW_FIELD_DESC = new org.apache.thrift.protocol.TField("window", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField IS_INCLUDE_SYS_FIELD_DESC = new org.apache.thrift.protocol.TField("is_include_sys", org.apache.thrift.protocol.TType.BOOL, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getComponentPageInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getComponentPageInfo_argsTupleSchemeFactory()); + } + + private String topology_id; // required + private String component_id; // required + private String window; // required + private boolean is_include_sys; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_ID((short)1, "topology_id"), + COMPONENT_ID((short)2, "component_id"), + WINDOW((short)3, "window"), + IS_INCLUDE_SYS((short)4, "is_include_sys"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_ID + return TOPOLOGY_ID; + case 2: // COMPONENT_ID + return COMPONENT_ID; + case 3: // WINDOW + return WINDOW; + case 4: // IS_INCLUDE_SYS + return IS_INCLUDE_SYS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __IS_INCLUDE_SYS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.WINDOW, new org.apache.thrift.meta_data.FieldMetaData("window", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.IS_INCLUDE_SYS, new org.apache.thrift.meta_data.FieldMetaData("is_include_sys", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getComponentPageInfo_args.class, metaDataMap); + } + + public getComponentPageInfo_args() { + } + + public getComponentPageInfo_args( + String topology_id, + String component_id, + String window, + boolean is_include_sys) + { + this(); + this.topology_id = topology_id; + this.component_id = component_id; + this.window = window; + this.is_include_sys = is_include_sys; + set_is_include_sys_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public getComponentPageInfo_args(getComponentPageInfo_args other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_topology_id()) { + this.topology_id = other.topology_id; + } + if (other.is_set_component_id()) { + this.component_id = other.component_id; + } + if (other.is_set_window()) { + this.window = other.window; + } + this.is_include_sys = other.is_include_sys; + } + + public getComponentPageInfo_args deepCopy() { + return new getComponentPageInfo_args(this); + } + + @Override + public void clear() { + this.topology_id = null; + this.component_id = null; + this.window = null; + set_is_include_sys_isSet(false); + this.is_include_sys = false; + } + + public String get_topology_id() { + return this.topology_id; + } + + public void set_topology_id(String topology_id) { + this.topology_id = topology_id; + } + + public void unset_topology_id() { + this.topology_id = null; + } + + /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_id() { + return this.topology_id != null; + } + + public void set_topology_id_isSet(boolean value) { + if (!value) { + this.topology_id = null; + } + } + + public String get_component_id() { + return this.component_id; + } + + public void set_component_id(String component_id) { + this.component_id = component_id; + } + + public void unset_component_id() { + this.component_id = null; + } + + /** Returns true if field component_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_component_id() { + return this.component_id != null; + } + + public void set_component_id_isSet(boolean value) { + if (!value) { + this.component_id = null; + } + } + + public String get_window() { + return this.window; + } + + public void set_window(String window) { + this.window = window; + } + + public void unset_window() { + this.window = null; + } + + /** Returns true if field window is set (has been assigned a value) and false otherwise */ + public boolean is_set_window() { + return this.window != null; + } + + public void set_window_isSet(boolean value) { + if (!value) { + this.window = null; + } + } + + public boolean is_is_include_sys() { + return this.is_include_sys; + } + + public void set_is_include_sys(boolean is_include_sys) { + this.is_include_sys = is_include_sys; + set_is_include_sys_isSet(true); + } + + public void unset_is_include_sys() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID); + } + + /** Returns true if field is_include_sys is set (has been assigned a value) and false otherwise */ + public boolean is_set_is_include_sys() { + return EncodingUtils.testBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID); + } + + public void set_is_include_sys_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_ID: + if (value == null) { + unset_topology_id(); + } else { + set_topology_id((String)value); + } + break; + + case COMPONENT_ID: + if (value == null) { + unset_component_id(); + } else { + set_component_id((String)value); + } + break; + + case WINDOW: + if (value == null) { + unset_window(); + } else { + set_window((String)value); + } + break; + + case IS_INCLUDE_SYS: + if (value == null) { + unset_is_include_sys(); + } else { + set_is_include_sys((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_ID: + return get_topology_id(); + + case COMPONENT_ID: + return get_component_id(); + + case WINDOW: + return get_window(); + + case IS_INCLUDE_SYS: + return Boolean.valueOf(is_is_include_sys()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_ID: + return is_set_topology_id(); + case COMPONENT_ID: + return is_set_component_id(); + case WINDOW: + return is_set_window(); + case IS_INCLUDE_SYS: + return is_set_is_include_sys(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getComponentPageInfo_args) + return this.equals((getComponentPageInfo_args)that); + return false; + } + + public boolean equals(getComponentPageInfo_args that) { + if (that == null) + return false; + + boolean this_present_topology_id = true && this.is_set_topology_id(); + boolean that_present_topology_id = true && that.is_set_topology_id(); + if (this_present_topology_id || that_present_topology_id) { + if (!(this_present_topology_id && that_present_topology_id)) + return false; + if (!this.topology_id.equals(that.topology_id)) + return false; + } + + boolean this_present_component_id = true && this.is_set_component_id(); + boolean that_present_component_id = true && that.is_set_component_id(); + if (this_present_component_id || that_present_component_id) { + if (!(this_present_component_id && that_present_component_id)) + return false; + if (!this.component_id.equals(that.component_id)) + return false; + } + + boolean this_present_window = true && this.is_set_window(); + boolean that_present_window = true && that.is_set_window(); + if (this_present_window || that_present_window) { + if (!(this_present_window && that_present_window)) + return false; + if (!this.window.equals(that.window)) + return false; + } + + boolean this_present_is_include_sys = true; + boolean that_present_is_include_sys = true; + if (this_present_is_include_sys || that_present_is_include_sys) { + if (!(this_present_is_include_sys && that_present_is_include_sys)) + return false; + if (this.is_include_sys != that.is_include_sys) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topology_id = true && (is_set_topology_id()); + list.add(present_topology_id); + if (present_topology_id) + list.add(topology_id); + + boolean present_component_id = true && (is_set_component_id()); + list.add(present_component_id); + if (present_component_id) + list.add(component_id); + + boolean present_window = true && (is_set_window()); + list.add(present_window); + if (present_window) + list.add(window); + + boolean present_is_include_sys = true; + list.add(present_is_include_sys); + if (present_is_include_sys) + list.add(is_include_sys); + + return list.hashCode(); + } + + @Override + public int compareTo(getComponentPageInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(other.is_set_component_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_component_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, other.component_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_window()).compareTo(other.is_set_window()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window, other.window); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_is_include_sys()).compareTo(other.is_set_is_include_sys()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_is_include_sys()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.is_include_sys, other.is_include_sys); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getComponentPageInfo_args("); + boolean first = true; + + sb.append("topology_id:"); + if (this.topology_id == null) { + sb.append("null"); + } else { + sb.append(this.topology_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("component_id:"); + if (this.component_id == null) { + sb.append("null"); + } else { + sb.append(this.component_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("window:"); + if (this.window == null) { + sb.append("null"); + } else { + sb.append(this.window); + } + first = false; + if (!first) sb.append(", "); + sb.append("is_include_sys:"); + sb.append(this.is_include_sys); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getComponentPageInfo_argsStandardSchemeFactory implements SchemeFactory { + public getComponentPageInfo_argsStandardScheme getScheme() { + return new getComponentPageInfo_argsStandardScheme(); + } + } + + private static class getComponentPageInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getComponentPageInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology_id = iprot.readString(); + struct.set_topology_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMPONENT_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.component_id = iprot.readString(); + struct.set_component_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // WINDOW + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.window = iprot.readString(); + struct.set_window_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // IS_INCLUDE_SYS + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.is_include_sys = iprot.readBool(); + struct.set_is_include_sys_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getComponentPageInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topology_id != null) { + oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC); + oprot.writeString(struct.topology_id); + oprot.writeFieldEnd(); + } + if (struct.component_id != null) { + oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); + oprot.writeString(struct.component_id); + oprot.writeFieldEnd(); + } + if (struct.window != null) { + oprot.writeFieldBegin(WINDOW_FIELD_DESC); + oprot.writeString(struct.window); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(IS_INCLUDE_SYS_FIELD_DESC); + oprot.writeBool(struct.is_include_sys); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getComponentPageInfo_argsTupleSchemeFactory implements SchemeFactory { + public getComponentPageInfo_argsTupleScheme getScheme() { + return new getComponentPageInfo_argsTupleScheme(); + } + } + + private static class getComponentPageInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_topology_id()) { + optionals.set(0); + } + if (struct.is_set_component_id()) { + optionals.set(1); + } + if (struct.is_set_window()) { + optionals.set(2); + } + if (struct.is_set_is_include_sys()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.is_set_topology_id()) { + oprot.writeString(struct.topology_id); + } + if (struct.is_set_component_id()) { + oprot.writeString(struct.component_id); + } + if (struct.is_set_window()) { + oprot.writeString(struct.window); + } + if (struct.is_set_is_include_sys()) { + oprot.writeBool(struct.is_include_sys); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.topology_id = iprot.readString(); + struct.set_topology_id_isSet(true); + } + if (incoming.get(1)) { + struct.component_id = iprot.readString(); + struct.set_component_id_isSet(true); + } + if (incoming.get(2)) { + struct.window = iprot.readString(); + struct.set_window_isSet(true); + } + if (incoming.get(3)) { + struct.is_include_sys = iprot.readBool(); + struct.set_is_include_sys_isSet(true); + } + } + } + + } + + public static class getComponentPageInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getComponentPageInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getComponentPageInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getComponentPageInfo_resultTupleSchemeFactory()); + } + + private ComponentPageInfo success; // required + private NotAliveException e; // required + private AuthorizationException aze; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"), + AZE((short)2, "aze"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + case 2: // AZE + return AZE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentPageInfo.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getComponentPageInfo_result.class, metaDataMap); + } + + public getComponentPageInfo_result() { + } + + public getComponentPageInfo_result( + ComponentPageInfo success, + NotAliveException e, + AuthorizationException aze) + { + this(); + this.success = success; + this.e = e; + this.aze = aze; + } + + /** + * Performs a deep copy on other. + */ + public getComponentPageInfo_result(getComponentPageInfo_result other) { + if (other.is_set_success()) { + this.success = new ComponentPageInfo(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_aze()) { + this.aze = new AuthorizationException(other.aze); + } + } + + public getComponentPageInfo_result deepCopy() { + return new getComponentPageInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + this.aze = null; + } + + public ComponentPageInfo get_success() { + return this.success; + } + + public void set_success(ComponentPageInfo success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public AuthorizationException get_aze() { + return this.aze; + } + + public void set_aze(AuthorizationException aze) { + this.aze = aze; + } + + public void unset_aze() { + this.aze = null; + } + + /** Returns true if field aze is set (has been assigned a value) and false otherwise */ + public boolean is_set_aze() { + return this.aze != null; + } + + public void set_aze_isSet(boolean value) { + if (!value) { + this.aze = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((ComponentPageInfo)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case AZE: + if (value == null) { + unset_aze(); + } else { + set_aze((AuthorizationException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + case AZE: + return get_aze(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + case AZE: + return is_set_aze(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getComponentPageInfo_result) + return this.equals((getComponentPageInfo_result)that); + return false; + } + + public boolean equals(getComponentPageInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_aze = true && this.is_set_aze(); + boolean that_present_aze = true && that.is_set_aze(); + if (this_present_aze || that_present_aze) { + if (!(this_present_aze && that_present_aze)) + return false; + if (!this.aze.equals(that.aze)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_aze = true && (is_set_aze()); + list.add(present_aze); + if (present_aze) + list.add(aze); + + return list.hashCode(); + } + + @Override + public int compareTo(getComponentPageInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_aze()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getComponentPageInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("aze:"); + if (this.aze == null) { + sb.append("null"); + } else { + sb.append(this.aze); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getComponentPageInfo_resultStandardSchemeFactory implements SchemeFactory { + public getComponentPageInfo_resultStandardScheme getScheme() { + return new getComponentPageInfo_resultStandardScheme(); + } + } + + private static class getComponentPageInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getComponentPageInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new ComponentPageInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // AZE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getComponentPageInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.aze != null) { + oprot.writeFieldBegin(AZE_FIELD_DESC); + struct.aze.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getComponentPageInfo_resultTupleSchemeFactory implements SchemeFactory { + public getComponentPageInfo_resultTupleScheme getScheme() { + return new getComponentPageInfo_resultTupleScheme(); + } + } + + private static class getComponentPageInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + if (struct.is_set_aze()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_aze()) { + struct.aze.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.success = new ComponentPageInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(2)) { + struct.aze = new AuthorizationException(); + struct.aze.read(iprot); + struct.set_aze_isSet(true); + } + } + } + + } + public static class getTopologyConf_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyConf_args"); diff --git a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java index f272cd81db8..55ee3b07a0c 100644 --- a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java +++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class NodeInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo"); @@ -461,13 +461,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, NodeInfo struct) th case 2: // PORT if (schemeField.type == org.apache.thrift.protocol.TType.SET) { { - org.apache.thrift.protocol.TSet _set380 = iprot.readSetBegin(); - struct.port = new HashSet(2*_set380.size); - long _elem381; - for (int _i382 = 0; _i382 < _set380.size; ++_i382) + org.apache.thrift.protocol.TSet _set496 = iprot.readSetBegin(); + struct.port = new HashSet(2*_set496.size); + long _elem497; + for (int _i498 = 0; _i498 < _set496.size; ++_i498) { - _elem381 = iprot.readI64(); - struct.port.add(_elem381); + _elem497 = iprot.readI64(); + struct.port.add(_elem497); } iprot.readSetEnd(); } @@ -498,9 +498,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, NodeInfo struct) t oprot.writeFieldBegin(PORT_FIELD_DESC); { oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size())); - for (long _iter383 : struct.port) + for (long _iter499 : struct.port) { - oprot.writeI64(_iter383); + oprot.writeI64(_iter499); } oprot.writeSetEnd(); } @@ -526,9 +526,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) th oprot.writeString(struct.node); { oprot.writeI32(struct.port.size()); - for (long _iter384 : struct.port) + for (long _iter500 : struct.port) { - oprot.writeI64(_iter384); + oprot.writeI64(_iter500); } } } @@ -539,13 +539,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, NodeInfo struct) thr struct.node = iprot.readString(); struct.set_node_isSet(true); { - org.apache.thrift.protocol.TSet _set385 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.port = new HashSet(2*_set385.size); - long _elem386; - for (int _i387 = 0; _i387 < _set385.size; ++_i387) + org.apache.thrift.protocol.TSet _set501 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.port = new HashSet(2*_set501.size); + long _elem502; + for (int _i503 = 0; _i503 < _set501.size; ++_i503) { - _elem386 = iprot.readI64(); - struct.port.add(_elem386); + _elem502 = iprot.readI64(); + struct.port.add(_elem502); } } struct.set_port_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java index 2cc7762cd84..5764c1d4667 100644 --- a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java +++ b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class RebalanceOptions implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RebalanceOptions"); @@ -529,15 +529,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, RebalanceOptions st case 3: // NUM_EXECUTORS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map334 = iprot.readMapBegin(); - struct.num_executors = new HashMap(2*_map334.size); - String _key335; - int _val336; - for (int _i337 = 0; _i337 < _map334.size; ++_i337) + org.apache.thrift.protocol.TMap _map450 = iprot.readMapBegin(); + struct.num_executors = new HashMap(2*_map450.size); + String _key451; + int _val452; + for (int _i453 = 0; _i453 < _map450.size; ++_i453) { - _key335 = iprot.readString(); - _val336 = iprot.readI32(); - struct.num_executors.put(_key335, _val336); + _key451 = iprot.readString(); + _val452 = iprot.readI32(); + struct.num_executors.put(_key451, _val452); } iprot.readMapEnd(); } @@ -574,10 +574,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, RebalanceOptions s oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.num_executors.size())); - for (Map.Entry _iter338 : struct.num_executors.entrySet()) + for (Map.Entry _iter454 : struct.num_executors.entrySet()) { - oprot.writeString(_iter338.getKey()); - oprot.writeI32(_iter338.getValue()); + oprot.writeString(_iter454.getKey()); + oprot.writeI32(_iter454.getValue()); } oprot.writeMapEnd(); } @@ -621,10 +621,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions st if (struct.is_set_num_executors()) { { oprot.writeI32(struct.num_executors.size()); - for (Map.Entry _iter339 : struct.num_executors.entrySet()) + for (Map.Entry _iter455 : struct.num_executors.entrySet()) { - oprot.writeString(_iter339.getKey()); - oprot.writeI32(_iter339.getValue()); + oprot.writeString(_iter455.getKey()); + oprot.writeI32(_iter455.getValue()); } } } @@ -644,15 +644,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions str } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map340 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.num_executors = new HashMap(2*_map340.size); - String _key341; - int _val342; - for (int _i343 = 0; _i343 < _map340.size; ++_i343) + org.apache.thrift.protocol.TMap _map456 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.num_executors = new HashMap(2*_map456.size); + String _key457; + int _val458; + for (int _i459 = 0; _i459 < _map456.size; ++_i459) { - _key341 = iprot.readString(); - _val342 = iprot.readI32(); - struct.num_executors.put(_key341, _val342); + _key457 = iprot.readString(); + _val458 = iprot.readI32(); + struct.num_executors.put(_key457, _val458); } } struct.set_num_executors_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/SpecificAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/SpecificAggregateStats.java new file mode 100644 index 00000000000..ed143f002cd --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/SpecificAggregateStats.java @@ -0,0 +1,387 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class SpecificAggregateStats extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpecificAggregateStats"); + private static final org.apache.thrift.protocol.TField BOLT_FIELD_DESC = new org.apache.thrift.protocol.TField("bolt", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SPOUT_FIELD_DESC = new org.apache.thrift.protocol.TField("spout", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOLT((short)1, "bolt"), + SPOUT((short)2, "spout"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOLT + return BOLT; + case 2: // SPOUT + return SPOUT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOLT, new org.apache.thrift.meta_data.FieldMetaData("bolt", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, BoltAggregateStats.class))); + tmpMap.put(_Fields.SPOUT, new org.apache.thrift.meta_data.FieldMetaData("spout", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpoutAggregateStats.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpecificAggregateStats.class, metaDataMap); + } + + public SpecificAggregateStats() { + super(); + } + + public SpecificAggregateStats(_Fields setField, Object value) { + super(setField, value); + } + + public SpecificAggregateStats(SpecificAggregateStats other) { + super(other); + } + public SpecificAggregateStats deepCopy() { + return new SpecificAggregateStats(this); + } + + public static SpecificAggregateStats bolt(BoltAggregateStats value) { + SpecificAggregateStats x = new SpecificAggregateStats(); + x.set_bolt(value); + return x; + } + + public static SpecificAggregateStats spout(SpoutAggregateStats value) { + SpecificAggregateStats x = new SpecificAggregateStats(); + x.set_spout(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case BOLT: + if (value instanceof BoltAggregateStats) { + break; + } + throw new ClassCastException("Was expecting value of type BoltAggregateStats for field 'bolt', but got " + value.getClass().getSimpleName()); + case SPOUT: + if (value instanceof SpoutAggregateStats) { + break; + } + throw new ClassCastException("Was expecting value of type SpoutAggregateStats for field 'spout', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case BOLT: + if (field.type == BOLT_FIELD_DESC.type) { + BoltAggregateStats bolt; + bolt = new BoltAggregateStats(); + bolt.read(iprot); + return bolt; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case SPOUT: + if (field.type == SPOUT_FIELD_DESC.type) { + SpoutAggregateStats spout; + spout = new SpoutAggregateStats(); + spout.read(iprot); + return spout; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOLT: + BoltAggregateStats bolt = (BoltAggregateStats)value_; + bolt.write(oprot); + return; + case SPOUT: + SpoutAggregateStats spout = (SpoutAggregateStats)value_; + spout.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case BOLT: + BoltAggregateStats bolt; + bolt = new BoltAggregateStats(); + bolt.read(iprot); + return bolt; + case SPOUT: + SpoutAggregateStats spout; + spout = new SpoutAggregateStats(); + spout.read(iprot); + return spout; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOLT: + BoltAggregateStats bolt = (BoltAggregateStats)value_; + bolt.write(oprot); + return; + case SPOUT: + SpoutAggregateStats spout = (SpoutAggregateStats)value_; + spout.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case BOLT: + return BOLT_FIELD_DESC; + case SPOUT: + return SPOUT_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public BoltAggregateStats get_bolt() { + if (getSetField() == _Fields.BOLT) { + return (BoltAggregateStats)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'bolt' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_bolt(BoltAggregateStats value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BOLT; + value_ = value; + } + + public SpoutAggregateStats get_spout() { + if (getSetField() == _Fields.SPOUT) { + return (SpoutAggregateStats)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'spout' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_spout(SpoutAggregateStats value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SPOUT; + value_ = value; + } + + public boolean is_set_bolt() { + return setField_ == _Fields.BOLT; + } + + + public boolean is_set_spout() { + return setField_ == _Fields.SPOUT; + } + + + public boolean equals(Object other) { + if (other instanceof SpecificAggregateStats) { + return equals((SpecificAggregateStats)other); + } else { + return false; + } + } + + public boolean equals(SpecificAggregateStats other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(SpecificAggregateStats other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java new file mode 100644 index 00000000000..cf5072c88cd --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java @@ -0,0 +1,407 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22") +public class SpoutAggregateStats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutAggregateStats"); + + private static final org.apache.thrift.protocol.TField COMPLETE_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("complete_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SpoutAggregateStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SpoutAggregateStatsTupleSchemeFactory()); + } + + private double complete_latency_ms; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COMPLETE_LATENCY_MS((short)1, "complete_latency_ms"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COMPLETE_LATENCY_MS + return COMPLETE_LATENCY_MS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __COMPLETE_LATENCY_MS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.COMPLETE_LATENCY_MS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COMPLETE_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("complete_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpoutAggregateStats.class, metaDataMap); + } + + public SpoutAggregateStats() { + } + + /** + * Performs a deep copy on other. + */ + public SpoutAggregateStats(SpoutAggregateStats other) { + __isset_bitfield = other.__isset_bitfield; + this.complete_latency_ms = other.complete_latency_ms; + } + + public SpoutAggregateStats deepCopy() { + return new SpoutAggregateStats(this); + } + + @Override + public void clear() { + set_complete_latency_ms_isSet(false); + this.complete_latency_ms = 0.0; + } + + public double get_complete_latency_ms() { + return this.complete_latency_ms; + } + + public void set_complete_latency_ms(double complete_latency_ms) { + this.complete_latency_ms = complete_latency_ms; + set_complete_latency_ms_isSet(true); + } + + public void unset_complete_latency_ms() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID); + } + + /** Returns true if field complete_latency_ms is set (has been assigned a value) and false otherwise */ + public boolean is_set_complete_latency_ms() { + return EncodingUtils.testBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID); + } + + public void set_complete_latency_ms_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COMPLETE_LATENCY_MS: + if (value == null) { + unset_complete_latency_ms(); + } else { + set_complete_latency_ms((Double)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COMPLETE_LATENCY_MS: + return Double.valueOf(get_complete_latency_ms()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COMPLETE_LATENCY_MS: + return is_set_complete_latency_ms(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SpoutAggregateStats) + return this.equals((SpoutAggregateStats)that); + return false; + } + + public boolean equals(SpoutAggregateStats that) { + if (that == null) + return false; + + boolean this_present_complete_latency_ms = true && this.is_set_complete_latency_ms(); + boolean that_present_complete_latency_ms = true && that.is_set_complete_latency_ms(); + if (this_present_complete_latency_ms || that_present_complete_latency_ms) { + if (!(this_present_complete_latency_ms && that_present_complete_latency_ms)) + return false; + if (this.complete_latency_ms != that.complete_latency_ms) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_complete_latency_ms = true && (is_set_complete_latency_ms()); + list.add(present_complete_latency_ms); + if (present_complete_latency_ms) + list.add(complete_latency_ms); + + return list.hashCode(); + } + + @Override + public int compareTo(SpoutAggregateStats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_complete_latency_ms()).compareTo(other.is_set_complete_latency_ms()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_complete_latency_ms()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.complete_latency_ms, other.complete_latency_ms); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SpoutAggregateStats("); + boolean first = true; + + if (is_set_complete_latency_ms()) { + sb.append("complete_latency_ms:"); + sb.append(this.complete_latency_ms); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SpoutAggregateStatsStandardSchemeFactory implements SchemeFactory { + public SpoutAggregateStatsStandardScheme getScheme() { + return new SpoutAggregateStatsStandardScheme(); + } + } + + private static class SpoutAggregateStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SpoutAggregateStats struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COMPLETE_LATENCY_MS + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.complete_latency_ms = iprot.readDouble(); + struct.set_complete_latency_ms_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SpoutAggregateStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.is_set_complete_latency_ms()) { + oprot.writeFieldBegin(COMPLETE_LATENCY_MS_FIELD_DESC); + oprot.writeDouble(struct.complete_latency_ms); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SpoutAggregateStatsTupleSchemeFactory implements SchemeFactory { + public SpoutAggregateStatsTupleScheme getScheme() { + return new SpoutAggregateStatsTupleScheme(); + } + } + + private static class SpoutAggregateStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SpoutAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_complete_latency_ms()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_complete_latency_ms()) { + oprot.writeDouble(struct.complete_latency_ms); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SpoutAggregateStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.complete_latency_ms = iprot.readDouble(); + struct.set_complete_latency_ms_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java index 250cc9c1e12..b880bc3391c 100644 --- a/storm-core/src/jvm/backtype/storm/generated/StormBase.java +++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class StormBase implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase"); @@ -1090,15 +1090,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, StormBase struct) t case 4: // COMPONENT_EXECUTORS if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map434 = iprot.readMapBegin(); - struct.component_executors = new HashMap(2*_map434.size); - String _key435; - int _val436; - for (int _i437 = 0; _i437 < _map434.size; ++_i437) + org.apache.thrift.protocol.TMap _map550 = iprot.readMapBegin(); + struct.component_executors = new HashMap(2*_map550.size); + String _key551; + int _val552; + for (int _i553 = 0; _i553 < _map550.size; ++_i553) { - _key435 = iprot.readString(); - _val436 = iprot.readI32(); - struct.component_executors.put(_key435, _val436); + _key551 = iprot.readString(); + _val552 = iprot.readI32(); + struct.component_executors.put(_key551, _val552); } iprot.readMapEnd(); } @@ -1143,16 +1143,16 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, StormBase struct) t case 9: // COMPONENT_DEBUG if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map438 = iprot.readMapBegin(); - struct.component_debug = new HashMap(2*_map438.size); - String _key439; - DebugOptions _val440; - for (int _i441 = 0; _i441 < _map438.size; ++_i441) + org.apache.thrift.protocol.TMap _map554 = iprot.readMapBegin(); + struct.component_debug = new HashMap(2*_map554.size); + String _key555; + DebugOptions _val556; + for (int _i557 = 0; _i557 < _map554.size; ++_i557) { - _key439 = iprot.readString(); - _val440 = new DebugOptions(); - _val440.read(iprot); - struct.component_debug.put(_key439, _val440); + _key555 = iprot.readString(); + _val556 = new DebugOptions(); + _val556.read(iprot); + struct.component_debug.put(_key555, _val556); } iprot.readMapEnd(); } @@ -1192,10 +1192,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, StormBase struct) oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.component_executors.size())); - for (Map.Entry _iter442 : struct.component_executors.entrySet()) + for (Map.Entry _iter558 : struct.component_executors.entrySet()) { - oprot.writeString(_iter442.getKey()); - oprot.writeI32(_iter442.getValue()); + oprot.writeString(_iter558.getKey()); + oprot.writeI32(_iter558.getValue()); } oprot.writeMapEnd(); } @@ -1233,10 +1233,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, StormBase struct) oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.component_debug.size())); - for (Map.Entry _iter443 : struct.component_debug.entrySet()) + for (Map.Entry _iter559 : struct.component_debug.entrySet()) { - oprot.writeString(_iter443.getKey()); - _iter443.getValue().write(oprot); + oprot.writeString(_iter559.getKey()); + _iter559.getValue().write(oprot); } oprot.writeMapEnd(); } @@ -1286,10 +1286,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, StormBase struct) t if (struct.is_set_component_executors()) { { oprot.writeI32(struct.component_executors.size()); - for (Map.Entry _iter444 : struct.component_executors.entrySet()) + for (Map.Entry _iter560 : struct.component_executors.entrySet()) { - oprot.writeString(_iter444.getKey()); - oprot.writeI32(_iter444.getValue()); + oprot.writeString(_iter560.getKey()); + oprot.writeI32(_iter560.getValue()); } } } @@ -1308,10 +1308,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, StormBase struct) t if (struct.is_set_component_debug()) { { oprot.writeI32(struct.component_debug.size()); - for (Map.Entry _iter445 : struct.component_debug.entrySet()) + for (Map.Entry _iter561 : struct.component_debug.entrySet()) { - oprot.writeString(_iter445.getKey()); - _iter445.getValue().write(oprot); + oprot.writeString(_iter561.getKey()); + _iter561.getValue().write(oprot); } } } @@ -1329,15 +1329,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, StormBase struct) th BitSet incoming = iprot.readBitSet(6); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map446 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.component_executors = new HashMap(2*_map446.size); - String _key447; - int _val448; - for (int _i449 = 0; _i449 < _map446.size; ++_i449) + org.apache.thrift.protocol.TMap _map562 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.component_executors = new HashMap(2*_map562.size); + String _key563; + int _val564; + for (int _i565 = 0; _i565 < _map562.size; ++_i565) { - _key447 = iprot.readString(); - _val448 = iprot.readI32(); - struct.component_executors.put(_key447, _val448); + _key563 = iprot.readString(); + _val564 = iprot.readI32(); + struct.component_executors.put(_key563, _val564); } } struct.set_component_executors_isSet(true); @@ -1361,16 +1361,16 @@ public void read(org.apache.thrift.protocol.TProtocol prot, StormBase struct) th } if (incoming.get(5)) { { - org.apache.thrift.protocol.TMap _map450 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.component_debug = new HashMap(2*_map450.size); - String _key451; - DebugOptions _val452; - for (int _i453 = 0; _i453 < _map450.size; ++_i453) + org.apache.thrift.protocol.TMap _map566 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.component_debug = new HashMap(2*_map566.size); + String _key567; + DebugOptions _val568; + for (int _i569 = 0; _i569 < _map566.size; ++_i569) { - _key451 = iprot.readString(); - _val452 = new DebugOptions(); - _val452.read(iprot); - struct.component_debug.put(_key451, _val452); + _key567 = iprot.readString(); + _val568 = new DebugOptions(); + _val568.read(iprot); + struct.component_debug.put(_key567, _val568); } } struct.set_component_debug_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java index bb4698c66d4..de48f99fbb9 100644 --- a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java +++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java @@ -51,7 +51,7 @@ import org.slf4j.LoggerFactory; @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3") +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") public class SupervisorInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo"); @@ -990,13 +990,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru case 4: // USED_PORTS if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list354 = iprot.readListBegin(); - struct.used_ports = new ArrayList(_list354.size); - long _elem355; - for (int _i356 = 0; _i356 < _list354.size; ++_i356) + org.apache.thrift.protocol.TList _list470 = iprot.readListBegin(); + struct.used_ports = new ArrayList(_list470.size); + long _elem471; + for (int _i472 = 0; _i472 < _list470.size; ++_i472) { - _elem355 = iprot.readI64(); - struct.used_ports.add(_elem355); + _elem471 = iprot.readI64(); + struct.used_ports.add(_elem471); } iprot.readListEnd(); } @@ -1008,13 +1008,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru case 5: // META if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list357 = iprot.readListBegin(); - struct.meta = new ArrayList(_list357.size); - long _elem358; - for (int _i359 = 0; _i359 < _list357.size; ++_i359) + org.apache.thrift.protocol.TList _list473 = iprot.readListBegin(); + struct.meta = new ArrayList(_list473.size); + long _elem474; + for (int _i475 = 0; _i475 < _list473.size; ++_i475) { - _elem358 = iprot.readI64(); - struct.meta.add(_elem358); + _elem474 = iprot.readI64(); + struct.meta.add(_elem474); } iprot.readListEnd(); } @@ -1026,15 +1026,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorInfo stru case 6: // SCHEDULER_META if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map360 = iprot.readMapBegin(); - struct.scheduler_meta = new HashMap(2*_map360.size); - String _key361; - String _val362; - for (int _i363 = 0; _i363 < _map360.size; ++_i363) + org.apache.thrift.protocol.TMap _map476 = iprot.readMapBegin(); + struct.scheduler_meta = new HashMap(2*_map476.size); + String _key477; + String _val478; + for (int _i479 = 0; _i479 < _map476.size; ++_i479) { - _key361 = iprot.readString(); - _val362 = iprot.readString(); - struct.scheduler_meta.put(_key361, _val362); + _key477 = iprot.readString(); + _val478 = iprot.readString(); + struct.scheduler_meta.put(_key477, _val478); } iprot.readMapEnd(); } @@ -1092,9 +1092,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldBegin(USED_PORTS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.used_ports.size())); - for (long _iter364 : struct.used_ports) + for (long _iter480 : struct.used_ports) { - oprot.writeI64(_iter364); + oprot.writeI64(_iter480); } oprot.writeListEnd(); } @@ -1106,9 +1106,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldBegin(META_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size())); - for (long _iter365 : struct.meta) + for (long _iter481 : struct.meta) { - oprot.writeI64(_iter365); + oprot.writeI64(_iter481); } oprot.writeListEnd(); } @@ -1120,10 +1120,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorInfo str oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.scheduler_meta.size())); - for (Map.Entry _iter366 : struct.scheduler_meta.entrySet()) + for (Map.Entry _iter482 : struct.scheduler_meta.entrySet()) { - oprot.writeString(_iter366.getKey()); - oprot.writeString(_iter366.getValue()); + oprot.writeString(_iter482.getKey()); + oprot.writeString(_iter482.getValue()); } oprot.writeMapEnd(); } @@ -1187,28 +1187,28 @@ public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo stru if (struct.is_set_used_ports()) { { oprot.writeI32(struct.used_ports.size()); - for (long _iter367 : struct.used_ports) + for (long _iter483 : struct.used_ports) { - oprot.writeI64(_iter367); + oprot.writeI64(_iter483); } } } if (struct.is_set_meta()) { { oprot.writeI32(struct.meta.size()); - for (long _iter368 : struct.meta) + for (long _iter484 : struct.meta) { - oprot.writeI64(_iter368); + oprot.writeI64(_iter484); } } } if (struct.is_set_scheduler_meta()) { { oprot.writeI32(struct.scheduler_meta.size()); - for (Map.Entry _iter369 : struct.scheduler_meta.entrySet()) + for (Map.Entry _iter485 : struct.scheduler_meta.entrySet()) { - oprot.writeString(_iter369.getKey()); - oprot.writeString(_iter369.getValue()); + oprot.writeString(_iter485.getKey()); + oprot.writeString(_iter485.getValue()); } } } @@ -1234,41 +1234,41 @@ public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorInfo struc } if (incoming.get(1)) { { - org.apache.thrift.protocol.TList _list370 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.used_ports = new ArrayList(_list370.size); - long _elem371; - for (int _i372 = 0; _i372 < _list370.size; ++_i372) + org.apache.thrift.protocol.TList _list486 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.used_ports = new ArrayList(_list486.size); + long _elem487; + for (int _i488 = 0; _i488 < _list486.size; ++_i488) { - _elem371 = iprot.readI64(); - struct.used_ports.add(_elem371); + _elem487 = iprot.readI64(); + struct.used_ports.add(_elem487); } } struct.set_used_ports_isSet(true); } if (incoming.get(2)) { { - org.apache.thrift.protocol.TList _list373 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); - struct.meta = new ArrayList(_list373.size); - long _elem374; - for (int _i375 = 0; _i375 < _list373.size; ++_i375) + org.apache.thrift.protocol.TList _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.meta = new ArrayList(_list489.size); + long _elem490; + for (int _i491 = 0; _i491 < _list489.size; ++_i491) { - _elem374 = iprot.readI64(); - struct.meta.add(_elem374); + _elem490 = iprot.readI64(); + struct.meta.add(_elem490); } } struct.set_meta_isSet(true); } if (incoming.get(3)) { { - org.apache.thrift.protocol.TMap _map376 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.scheduler_meta = new HashMap(2*_map376.size); - String _key377; - String _val378; - for (int _i379 = 0; _i379 < _map376.size; ++_i379) + org.apache.thrift.protocol.TMap _map492 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.scheduler_meta = new HashMap(2*_map492.size); + String _key493; + String _val494; + for (int _i495 = 0; _i495 < _map492.size; ++_i495) { - _key377 = iprot.readString(); - _val378 = iprot.readString(); - struct.scheduler_meta.put(_key377, _val378); + _key493 = iprot.readString(); + _val494 = iprot.readString(); + struct.scheduler_meta.put(_key493, _val494); } } struct.set_scheduler_meta_isSet(true); diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java new file mode 100644 index 00000000000..c31e23e74c7 --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java @@ -0,0 +1,2003 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") +public class TopologyPageInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyPageInfo"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)6); + private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)7); + private static final org.apache.thrift.protocol.TField TOPOLOGY_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_conf", org.apache.thrift.protocol.TType.STRING, (short)8); + private static final org.apache.thrift.protocol.TField ID_TO_SPOUT_AGG_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("id_to_spout_agg_stats", org.apache.thrift.protocol.TType.MAP, (short)9); + private static final org.apache.thrift.protocol.TField ID_TO_BOLT_AGG_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("id_to_bolt_agg_stats", org.apache.thrift.protocol.TType.MAP, (short)10); + private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)11); + private static final org.apache.thrift.protocol.TField TOPOLOGY_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_stats", org.apache.thrift.protocol.TType.STRUCT, (short)12); + private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)13); + private static final org.apache.thrift.protocol.TField DEBUG_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("debug_options", org.apache.thrift.protocol.TType.STRUCT, (short)14); + private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)15); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologyPageInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologyPageInfoTupleSchemeFactory()); + } + + private String id; // required + private String name; // optional + private int uptime_secs; // optional + private String status; // optional + private int num_tasks; // optional + private int num_workers; // optional + private int num_executors; // optional + private String topology_conf; // optional + private Map id_to_spout_agg_stats; // optional + private Map id_to_bolt_agg_stats; // optional + private String sched_status; // optional + private TopologyStats topology_stats; // optional + private String owner; // optional + private DebugOptions debug_options; // optional + private int replication_count; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"), + NAME((short)2, "name"), + UPTIME_SECS((short)3, "uptime_secs"), + STATUS((short)4, "status"), + NUM_TASKS((short)5, "num_tasks"), + NUM_WORKERS((short)6, "num_workers"), + NUM_EXECUTORS((short)7, "num_executors"), + TOPOLOGY_CONF((short)8, "topology_conf"), + ID_TO_SPOUT_AGG_STATS((short)9, "id_to_spout_agg_stats"), + ID_TO_BOLT_AGG_STATS((short)10, "id_to_bolt_agg_stats"), + SCHED_STATUS((short)11, "sched_status"), + TOPOLOGY_STATS((short)12, "topology_stats"), + OWNER((short)13, "owner"), + DEBUG_OPTIONS((short)14, "debug_options"), + REPLICATION_COUNT((short)15, "replication_count"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + case 2: // NAME + return NAME; + case 3: // UPTIME_SECS + return UPTIME_SECS; + case 4: // STATUS + return STATUS; + case 5: // NUM_TASKS + return NUM_TASKS; + case 6: // NUM_WORKERS + return NUM_WORKERS; + case 7: // NUM_EXECUTORS + return NUM_EXECUTORS; + case 8: // TOPOLOGY_CONF + return TOPOLOGY_CONF; + case 9: // ID_TO_SPOUT_AGG_STATS + return ID_TO_SPOUT_AGG_STATS; + case 10: // ID_TO_BOLT_AGG_STATS + return ID_TO_BOLT_AGG_STATS; + case 11: // SCHED_STATUS + return SCHED_STATUS; + case 12: // TOPOLOGY_STATS + return TOPOLOGY_STATS; + case 13: // OWNER + return OWNER; + case 14: // DEBUG_OPTIONS + return DEBUG_OPTIONS; + case 15: // REPLICATION_COUNT + return REPLICATION_COUNT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __UPTIME_SECS_ISSET_ID = 0; + private static final int __NUM_TASKS_ISSET_ID = 1; + private static final int __NUM_WORKERS_ISSET_ID = 2; + private static final int __NUM_EXECUTORS_ISSET_ID = 3; + private static final int __REPLICATION_COUNT_ISSET_ID = 4; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.NAME,_Fields.UPTIME_SECS,_Fields.STATUS,_Fields.NUM_TASKS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS,_Fields.TOPOLOGY_CONF,_Fields.ID_TO_SPOUT_AGG_STATS,_Fields.ID_TO_BOLT_AGG_STATS,_Fields.SCHED_STATUS,_Fields.TOPOLOGY_STATS,_Fields.OWNER,_Fields.DEBUG_OPTIONS,_Fields.REPLICATION_COUNT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.TOPOLOGY_CONF, new org.apache.thrift.meta_data.FieldMetaData("topology_conf", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ID_TO_SPOUT_AGG_STATS, new org.apache.thrift.meta_data.FieldMetaData("id_to_spout_agg_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class)))); + tmpMap.put(_Fields.ID_TO_BOLT_AGG_STATS, new org.apache.thrift.meta_data.FieldMetaData("id_to_bolt_agg_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class)))); + tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TOPOLOGY_STATS, new org.apache.thrift.meta_data.FieldMetaData("topology_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyStats.class))); + tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.DEBUG_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("debug_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class))); + tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyPageInfo.class, metaDataMap); + } + + public TopologyPageInfo() { + } + + public TopologyPageInfo( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public TopologyPageInfo(TopologyPageInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_id()) { + this.id = other.id; + } + if (other.is_set_name()) { + this.name = other.name; + } + this.uptime_secs = other.uptime_secs; + if (other.is_set_status()) { + this.status = other.status; + } + this.num_tasks = other.num_tasks; + this.num_workers = other.num_workers; + this.num_executors = other.num_executors; + if (other.is_set_topology_conf()) { + this.topology_conf = other.topology_conf; + } + if (other.is_set_id_to_spout_agg_stats()) { + Map __this__id_to_spout_agg_stats = new HashMap(other.id_to_spout_agg_stats.size()); + for (Map.Entry other_element : other.id_to_spout_agg_stats.entrySet()) { + + String other_element_key = other_element.getKey(); + ComponentAggregateStats other_element_value = other_element.getValue(); + + String __this__id_to_spout_agg_stats_copy_key = other_element_key; + + ComponentAggregateStats __this__id_to_spout_agg_stats_copy_value = new ComponentAggregateStats(other_element_value); + + __this__id_to_spout_agg_stats.put(__this__id_to_spout_agg_stats_copy_key, __this__id_to_spout_agg_stats_copy_value); + } + this.id_to_spout_agg_stats = __this__id_to_spout_agg_stats; + } + if (other.is_set_id_to_bolt_agg_stats()) { + Map __this__id_to_bolt_agg_stats = new HashMap(other.id_to_bolt_agg_stats.size()); + for (Map.Entry other_element : other.id_to_bolt_agg_stats.entrySet()) { + + String other_element_key = other_element.getKey(); + ComponentAggregateStats other_element_value = other_element.getValue(); + + String __this__id_to_bolt_agg_stats_copy_key = other_element_key; + + ComponentAggregateStats __this__id_to_bolt_agg_stats_copy_value = new ComponentAggregateStats(other_element_value); + + __this__id_to_bolt_agg_stats.put(__this__id_to_bolt_agg_stats_copy_key, __this__id_to_bolt_agg_stats_copy_value); + } + this.id_to_bolt_agg_stats = __this__id_to_bolt_agg_stats; + } + if (other.is_set_sched_status()) { + this.sched_status = other.sched_status; + } + if (other.is_set_topology_stats()) { + this.topology_stats = new TopologyStats(other.topology_stats); + } + if (other.is_set_owner()) { + this.owner = other.owner; + } + if (other.is_set_debug_options()) { + this.debug_options = new DebugOptions(other.debug_options); + } + this.replication_count = other.replication_count; + } + + public TopologyPageInfo deepCopy() { + return new TopologyPageInfo(this); + } + + @Override + public void clear() { + this.id = null; + this.name = null; + set_uptime_secs_isSet(false); + this.uptime_secs = 0; + this.status = null; + set_num_tasks_isSet(false); + this.num_tasks = 0; + set_num_workers_isSet(false); + this.num_workers = 0; + set_num_executors_isSet(false); + this.num_executors = 0; + this.topology_conf = null; + this.id_to_spout_agg_stats = null; + this.id_to_bolt_agg_stats = null; + this.sched_status = null; + this.topology_stats = null; + this.owner = null; + this.debug_options = null; + set_replication_count_isSet(false); + this.replication_count = 0; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public int get_uptime_secs() { + return this.uptime_secs; + } + + public void set_uptime_secs(int uptime_secs) { + this.uptime_secs = uptime_secs; + set_uptime_secs_isSet(true); + } + + public void unset_uptime_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_uptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + public void set_uptime_secs_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value); + } + + public String get_status() { + return this.status; + } + + public void set_status(String status) { + this.status = status; + } + + public void unset_status() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean is_set_status() { + return this.status != null; + } + + public void set_status_isSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public int get_num_tasks() { + return this.num_tasks; + } + + public void set_num_tasks(int num_tasks) { + this.num_tasks = num_tasks; + set_num_tasks_isSet(true); + } + + public void unset_num_tasks() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_tasks() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + public void set_num_tasks_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value); + } + + public int get_num_workers() { + return this.num_workers; + } + + public void set_num_workers(int num_workers) { + this.num_workers = num_workers; + set_num_workers_isSet(true); + } + + public void unset_num_workers() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + public void set_num_workers_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value); + } + + public int get_num_executors() { + return this.num_executors; + } + + public void set_num_executors(int num_executors) { + this.num_executors = num_executors; + set_num_executors_isSet(true); + } + + public void unset_num_executors() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_executors() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID); + } + + public void set_num_executors_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value); + } + + public String get_topology_conf() { + return this.topology_conf; + } + + public void set_topology_conf(String topology_conf) { + this.topology_conf = topology_conf; + } + + public void unset_topology_conf() { + this.topology_conf = null; + } + + /** Returns true if field topology_conf is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_conf() { + return this.topology_conf != null; + } + + public void set_topology_conf_isSet(boolean value) { + if (!value) { + this.topology_conf = null; + } + } + + public int get_id_to_spout_agg_stats_size() { + return (this.id_to_spout_agg_stats == null) ? 0 : this.id_to_spout_agg_stats.size(); + } + + public void put_to_id_to_spout_agg_stats(String key, ComponentAggregateStats val) { + if (this.id_to_spout_agg_stats == null) { + this.id_to_spout_agg_stats = new HashMap(); + } + this.id_to_spout_agg_stats.put(key, val); + } + + public Map get_id_to_spout_agg_stats() { + return this.id_to_spout_agg_stats; + } + + public void set_id_to_spout_agg_stats(Map id_to_spout_agg_stats) { + this.id_to_spout_agg_stats = id_to_spout_agg_stats; + } + + public void unset_id_to_spout_agg_stats() { + this.id_to_spout_agg_stats = null; + } + + /** Returns true if field id_to_spout_agg_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_id_to_spout_agg_stats() { + return this.id_to_spout_agg_stats != null; + } + + public void set_id_to_spout_agg_stats_isSet(boolean value) { + if (!value) { + this.id_to_spout_agg_stats = null; + } + } + + public int get_id_to_bolt_agg_stats_size() { + return (this.id_to_bolt_agg_stats == null) ? 0 : this.id_to_bolt_agg_stats.size(); + } + + public void put_to_id_to_bolt_agg_stats(String key, ComponentAggregateStats val) { + if (this.id_to_bolt_agg_stats == null) { + this.id_to_bolt_agg_stats = new HashMap(); + } + this.id_to_bolt_agg_stats.put(key, val); + } + + public Map get_id_to_bolt_agg_stats() { + return this.id_to_bolt_agg_stats; + } + + public void set_id_to_bolt_agg_stats(Map id_to_bolt_agg_stats) { + this.id_to_bolt_agg_stats = id_to_bolt_agg_stats; + } + + public void unset_id_to_bolt_agg_stats() { + this.id_to_bolt_agg_stats = null; + } + + /** Returns true if field id_to_bolt_agg_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_id_to_bolt_agg_stats() { + return this.id_to_bolt_agg_stats != null; + } + + public void set_id_to_bolt_agg_stats_isSet(boolean value) { + if (!value) { + this.id_to_bolt_agg_stats = null; + } + } + + public String get_sched_status() { + return this.sched_status; + } + + public void set_sched_status(String sched_status) { + this.sched_status = sched_status; + } + + public void unset_sched_status() { + this.sched_status = null; + } + + /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */ + public boolean is_set_sched_status() { + return this.sched_status != null; + } + + public void set_sched_status_isSet(boolean value) { + if (!value) { + this.sched_status = null; + } + } + + public TopologyStats get_topology_stats() { + return this.topology_stats; + } + + public void set_topology_stats(TopologyStats topology_stats) { + this.topology_stats = topology_stats; + } + + public void unset_topology_stats() { + this.topology_stats = null; + } + + /** Returns true if field topology_stats is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_stats() { + return this.topology_stats != null; + } + + public void set_topology_stats_isSet(boolean value) { + if (!value) { + this.topology_stats = null; + } + } + + public String get_owner() { + return this.owner; + } + + public void set_owner(String owner) { + this.owner = owner; + } + + public void unset_owner() { + this.owner = null; + } + + /** Returns true if field owner is set (has been assigned a value) and false otherwise */ + public boolean is_set_owner() { + return this.owner != null; + } + + public void set_owner_isSet(boolean value) { + if (!value) { + this.owner = null; + } + } + + public DebugOptions get_debug_options() { + return this.debug_options; + } + + public void set_debug_options(DebugOptions debug_options) { + this.debug_options = debug_options; + } + + public void unset_debug_options() { + this.debug_options = null; + } + + /** Returns true if field debug_options is set (has been assigned a value) and false otherwise */ + public boolean is_set_debug_options() { + return this.debug_options != null; + } + + public void set_debug_options_isSet(boolean value) { + if (!value) { + this.debug_options = null; + } + } + + public int get_replication_count() { + return this.replication_count; + } + + public void set_replication_count(int replication_count) { + this.replication_count = replication_count; + set_replication_count_isSet(true); + } + + public void unset_replication_count() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID); + } + + /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */ + public boolean is_set_replication_count() { + return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID); + } + + public void set_replication_count_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unset_uptime_secs(); + } else { + set_uptime_secs((Integer)value); + } + break; + + case STATUS: + if (value == null) { + unset_status(); + } else { + set_status((String)value); + } + break; + + case NUM_TASKS: + if (value == null) { + unset_num_tasks(); + } else { + set_num_tasks((Integer)value); + } + break; + + case NUM_WORKERS: + if (value == null) { + unset_num_workers(); + } else { + set_num_workers((Integer)value); + } + break; + + case NUM_EXECUTORS: + if (value == null) { + unset_num_executors(); + } else { + set_num_executors((Integer)value); + } + break; + + case TOPOLOGY_CONF: + if (value == null) { + unset_topology_conf(); + } else { + set_topology_conf((String)value); + } + break; + + case ID_TO_SPOUT_AGG_STATS: + if (value == null) { + unset_id_to_spout_agg_stats(); + } else { + set_id_to_spout_agg_stats((Map)value); + } + break; + + case ID_TO_BOLT_AGG_STATS: + if (value == null) { + unset_id_to_bolt_agg_stats(); + } else { + set_id_to_bolt_agg_stats((Map)value); + } + break; + + case SCHED_STATUS: + if (value == null) { + unset_sched_status(); + } else { + set_sched_status((String)value); + } + break; + + case TOPOLOGY_STATS: + if (value == null) { + unset_topology_stats(); + } else { + set_topology_stats((TopologyStats)value); + } + break; + + case OWNER: + if (value == null) { + unset_owner(); + } else { + set_owner((String)value); + } + break; + + case DEBUG_OPTIONS: + if (value == null) { + unset_debug_options(); + } else { + set_debug_options((DebugOptions)value); + } + break; + + case REPLICATION_COUNT: + if (value == null) { + unset_replication_count(); + } else { + set_replication_count((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + case NAME: + return get_name(); + + case UPTIME_SECS: + return Integer.valueOf(get_uptime_secs()); + + case STATUS: + return get_status(); + + case NUM_TASKS: + return Integer.valueOf(get_num_tasks()); + + case NUM_WORKERS: + return Integer.valueOf(get_num_workers()); + + case NUM_EXECUTORS: + return Integer.valueOf(get_num_executors()); + + case TOPOLOGY_CONF: + return get_topology_conf(); + + case ID_TO_SPOUT_AGG_STATS: + return get_id_to_spout_agg_stats(); + + case ID_TO_BOLT_AGG_STATS: + return get_id_to_bolt_agg_stats(); + + case SCHED_STATUS: + return get_sched_status(); + + case TOPOLOGY_STATS: + return get_topology_stats(); + + case OWNER: + return get_owner(); + + case DEBUG_OPTIONS: + return get_debug_options(); + + case REPLICATION_COUNT: + return Integer.valueOf(get_replication_count()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + case NAME: + return is_set_name(); + case UPTIME_SECS: + return is_set_uptime_secs(); + case STATUS: + return is_set_status(); + case NUM_TASKS: + return is_set_num_tasks(); + case NUM_WORKERS: + return is_set_num_workers(); + case NUM_EXECUTORS: + return is_set_num_executors(); + case TOPOLOGY_CONF: + return is_set_topology_conf(); + case ID_TO_SPOUT_AGG_STATS: + return is_set_id_to_spout_agg_stats(); + case ID_TO_BOLT_AGG_STATS: + return is_set_id_to_bolt_agg_stats(); + case SCHED_STATUS: + return is_set_sched_status(); + case TOPOLOGY_STATS: + return is_set_topology_stats(); + case OWNER: + return is_set_owner(); + case DEBUG_OPTIONS: + return is_set_debug_options(); + case REPLICATION_COUNT: + return is_set_replication_count(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TopologyPageInfo) + return this.equals((TopologyPageInfo)that); + return false; + } + + public boolean equals(TopologyPageInfo that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_uptime_secs = true && this.is_set_uptime_secs(); + boolean that_present_uptime_secs = true && that.is_set_uptime_secs(); + if (this_present_uptime_secs || that_present_uptime_secs) { + if (!(this_present_uptime_secs && that_present_uptime_secs)) + return false; + if (this.uptime_secs != that.uptime_secs) + return false; + } + + boolean this_present_status = true && this.is_set_status(); + boolean that_present_status = true && that.is_set_status(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_num_tasks = true && this.is_set_num_tasks(); + boolean that_present_num_tasks = true && that.is_set_num_tasks(); + if (this_present_num_tasks || that_present_num_tasks) { + if (!(this_present_num_tasks && that_present_num_tasks)) + return false; + if (this.num_tasks != that.num_tasks) + return false; + } + + boolean this_present_num_workers = true && this.is_set_num_workers(); + boolean that_present_num_workers = true && that.is_set_num_workers(); + if (this_present_num_workers || that_present_num_workers) { + if (!(this_present_num_workers && that_present_num_workers)) + return false; + if (this.num_workers != that.num_workers) + return false; + } + + boolean this_present_num_executors = true && this.is_set_num_executors(); + boolean that_present_num_executors = true && that.is_set_num_executors(); + if (this_present_num_executors || that_present_num_executors) { + if (!(this_present_num_executors && that_present_num_executors)) + return false; + if (this.num_executors != that.num_executors) + return false; + } + + boolean this_present_topology_conf = true && this.is_set_topology_conf(); + boolean that_present_topology_conf = true && that.is_set_topology_conf(); + if (this_present_topology_conf || that_present_topology_conf) { + if (!(this_present_topology_conf && that_present_topology_conf)) + return false; + if (!this.topology_conf.equals(that.topology_conf)) + return false; + } + + boolean this_present_id_to_spout_agg_stats = true && this.is_set_id_to_spout_agg_stats(); + boolean that_present_id_to_spout_agg_stats = true && that.is_set_id_to_spout_agg_stats(); + if (this_present_id_to_spout_agg_stats || that_present_id_to_spout_agg_stats) { + if (!(this_present_id_to_spout_agg_stats && that_present_id_to_spout_agg_stats)) + return false; + if (!this.id_to_spout_agg_stats.equals(that.id_to_spout_agg_stats)) + return false; + } + + boolean this_present_id_to_bolt_agg_stats = true && this.is_set_id_to_bolt_agg_stats(); + boolean that_present_id_to_bolt_agg_stats = true && that.is_set_id_to_bolt_agg_stats(); + if (this_present_id_to_bolt_agg_stats || that_present_id_to_bolt_agg_stats) { + if (!(this_present_id_to_bolt_agg_stats && that_present_id_to_bolt_agg_stats)) + return false; + if (!this.id_to_bolt_agg_stats.equals(that.id_to_bolt_agg_stats)) + return false; + } + + boolean this_present_sched_status = true && this.is_set_sched_status(); + boolean that_present_sched_status = true && that.is_set_sched_status(); + if (this_present_sched_status || that_present_sched_status) { + if (!(this_present_sched_status && that_present_sched_status)) + return false; + if (!this.sched_status.equals(that.sched_status)) + return false; + } + + boolean this_present_topology_stats = true && this.is_set_topology_stats(); + boolean that_present_topology_stats = true && that.is_set_topology_stats(); + if (this_present_topology_stats || that_present_topology_stats) { + if (!(this_present_topology_stats && that_present_topology_stats)) + return false; + if (!this.topology_stats.equals(that.topology_stats)) + return false; + } + + boolean this_present_owner = true && this.is_set_owner(); + boolean that_present_owner = true && that.is_set_owner(); + if (this_present_owner || that_present_owner) { + if (!(this_present_owner && that_present_owner)) + return false; + if (!this.owner.equals(that.owner)) + return false; + } + + boolean this_present_debug_options = true && this.is_set_debug_options(); + boolean that_present_debug_options = true && that.is_set_debug_options(); + if (this_present_debug_options || that_present_debug_options) { + if (!(this_present_debug_options && that_present_debug_options)) + return false; + if (!this.debug_options.equals(that.debug_options)) + return false; + } + + boolean this_present_replication_count = true && this.is_set_replication_count(); + boolean that_present_replication_count = true && that.is_set_replication_count(); + if (this_present_replication_count || that_present_replication_count) { + if (!(this_present_replication_count && that_present_replication_count)) + return false; + if (this.replication_count != that.replication_count) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_uptime_secs = true && (is_set_uptime_secs()); + list.add(present_uptime_secs); + if (present_uptime_secs) + list.add(uptime_secs); + + boolean present_status = true && (is_set_status()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_num_tasks = true && (is_set_num_tasks()); + list.add(present_num_tasks); + if (present_num_tasks) + list.add(num_tasks); + + boolean present_num_workers = true && (is_set_num_workers()); + list.add(present_num_workers); + if (present_num_workers) + list.add(num_workers); + + boolean present_num_executors = true && (is_set_num_executors()); + list.add(present_num_executors); + if (present_num_executors) + list.add(num_executors); + + boolean present_topology_conf = true && (is_set_topology_conf()); + list.add(present_topology_conf); + if (present_topology_conf) + list.add(topology_conf); + + boolean present_id_to_spout_agg_stats = true && (is_set_id_to_spout_agg_stats()); + list.add(present_id_to_spout_agg_stats); + if (present_id_to_spout_agg_stats) + list.add(id_to_spout_agg_stats); + + boolean present_id_to_bolt_agg_stats = true && (is_set_id_to_bolt_agg_stats()); + list.add(present_id_to_bolt_agg_stats); + if (present_id_to_bolt_agg_stats) + list.add(id_to_bolt_agg_stats); + + boolean present_sched_status = true && (is_set_sched_status()); + list.add(present_sched_status); + if (present_sched_status) + list.add(sched_status); + + boolean present_topology_stats = true && (is_set_topology_stats()); + list.add(present_topology_stats); + if (present_topology_stats) + list.add(topology_stats); + + boolean present_owner = true && (is_set_owner()); + list.add(present_owner); + if (present_owner) + list.add(owner); + + boolean present_debug_options = true && (is_set_debug_options()); + list.add(present_debug_options); + if (present_debug_options) + list.add(debug_options); + + boolean present_replication_count = true && (is_set_replication_count()); + list.add(present_replication_count); + if (present_replication_count) + list.add(replication_count); + + return list.hashCode(); + } + + @Override + public int compareTo(TopologyPageInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_tasks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_executors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology_conf()).compareTo(other.is_set_topology_conf()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_conf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_conf, other.topology_conf); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_id_to_spout_agg_stats()).compareTo(other.is_set_id_to_spout_agg_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id_to_spout_agg_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id_to_spout_agg_stats, other.id_to_spout_agg_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_id_to_bolt_agg_stats()).compareTo(other.is_set_id_to_bolt_agg_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id_to_bolt_agg_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id_to_bolt_agg_stats, other.id_to_bolt_agg_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(other.is_set_sched_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_sched_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, other.sched_status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology_stats()).compareTo(other.is_set_topology_stats()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_stats()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_stats, other.topology_stats); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_owner()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_debug_options()).compareTo(other.is_set_debug_options()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_debug_options()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug_options, other.debug_options); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(other.is_set_replication_count()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_replication_count()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, other.replication_count); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TopologyPageInfo("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + if (is_set_name()) { + if (!first) sb.append(", "); + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + } + if (is_set_uptime_secs()) { + if (!first) sb.append(", "); + sb.append("uptime_secs:"); + sb.append(this.uptime_secs); + first = false; + } + if (is_set_status()) { + if (!first) sb.append(", "); + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + } + if (is_set_num_tasks()) { + if (!first) sb.append(", "); + sb.append("num_tasks:"); + sb.append(this.num_tasks); + first = false; + } + if (is_set_num_workers()) { + if (!first) sb.append(", "); + sb.append("num_workers:"); + sb.append(this.num_workers); + first = false; + } + if (is_set_num_executors()) { + if (!first) sb.append(", "); + sb.append("num_executors:"); + sb.append(this.num_executors); + first = false; + } + if (is_set_topology_conf()) { + if (!first) sb.append(", "); + sb.append("topology_conf:"); + if (this.topology_conf == null) { + sb.append("null"); + } else { + sb.append(this.topology_conf); + } + first = false; + } + if (is_set_id_to_spout_agg_stats()) { + if (!first) sb.append(", "); + sb.append("id_to_spout_agg_stats:"); + if (this.id_to_spout_agg_stats == null) { + sb.append("null"); + } else { + sb.append(this.id_to_spout_agg_stats); + } + first = false; + } + if (is_set_id_to_bolt_agg_stats()) { + if (!first) sb.append(", "); + sb.append("id_to_bolt_agg_stats:"); + if (this.id_to_bolt_agg_stats == null) { + sb.append("null"); + } else { + sb.append(this.id_to_bolt_agg_stats); + } + first = false; + } + if (is_set_sched_status()) { + if (!first) sb.append(", "); + sb.append("sched_status:"); + if (this.sched_status == null) { + sb.append("null"); + } else { + sb.append(this.sched_status); + } + first = false; + } + if (is_set_topology_stats()) { + if (!first) sb.append(", "); + sb.append("topology_stats:"); + if (this.topology_stats == null) { + sb.append("null"); + } else { + sb.append(this.topology_stats); + } + first = false; + } + if (is_set_owner()) { + if (!first) sb.append(", "); + sb.append("owner:"); + if (this.owner == null) { + sb.append("null"); + } else { + sb.append(this.owner); + } + first = false; + } + if (is_set_debug_options()) { + if (!first) sb.append(", "); + sb.append("debug_options:"); + if (this.debug_options == null) { + sb.append("null"); + } else { + sb.append(this.debug_options); + } + first = false; + } + if (is_set_replication_count()) { + if (!first) sb.append(", "); + sb.append("replication_count:"); + sb.append(this.replication_count); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (topology_stats != null) { + topology_stats.validate(); + } + if (debug_options != null) { + debug_options.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TopologyPageInfoStandardSchemeFactory implements SchemeFactory { + public TopologyPageInfoStandardScheme getScheme() { + return new TopologyPageInfoStandardScheme(); + } + } + + private static class TopologyPageInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyPageInfo struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime_secs = iprot.readI32(); + struct.set_uptime_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.status = iprot.readString(); + struct.set_status_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // NUM_TASKS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // NUM_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_workers = iprot.readI32(); + struct.set_num_workers_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // NUM_EXECUTORS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_executors = iprot.readI32(); + struct.set_num_executors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // TOPOLOGY_CONF + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology_conf = iprot.readString(); + struct.set_topology_conf_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 9: // ID_TO_SPOUT_AGG_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map384 = iprot.readMapBegin(); + struct.id_to_spout_agg_stats = new HashMap(2*_map384.size); + String _key385; + ComponentAggregateStats _val386; + for (int _i387 = 0; _i387 < _map384.size; ++_i387) + { + _key385 = iprot.readString(); + _val386 = new ComponentAggregateStats(); + _val386.read(iprot); + struct.id_to_spout_agg_stats.put(_key385, _val386); + } + iprot.readMapEnd(); + } + struct.set_id_to_spout_agg_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 10: // ID_TO_BOLT_AGG_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map388 = iprot.readMapBegin(); + struct.id_to_bolt_agg_stats = new HashMap(2*_map388.size); + String _key389; + ComponentAggregateStats _val390; + for (int _i391 = 0; _i391 < _map388.size; ++_i391) + { + _key389 = iprot.readString(); + _val390 = new ComponentAggregateStats(); + _val390.read(iprot); + struct.id_to_bolt_agg_stats.put(_key389, _val390); + } + iprot.readMapEnd(); + } + struct.set_id_to_bolt_agg_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 11: // SCHED_STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.sched_status = iprot.readString(); + struct.set_sched_status_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 12: // TOPOLOGY_STATS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.topology_stats = new TopologyStats(); + struct.topology_stats.read(iprot); + struct.set_topology_stats_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 13: // OWNER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.owner = iprot.readString(); + struct.set_owner_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 14: // DEBUG_OPTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.debug_options = new DebugOptions(); + struct.debug_options.read(iprot); + struct.set_debug_options_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 15: // REPLICATION_COUNT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.replication_count = iprot.readI32(); + struct.set_replication_count_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyPageInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + if (struct.name != null) { + if (struct.is_set_name()) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + } + if (struct.is_set_uptime_secs()) { + oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); + oprot.writeI32(struct.uptime_secs); + oprot.writeFieldEnd(); + } + if (struct.status != null) { + if (struct.is_set_status()) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + oprot.writeString(struct.status); + oprot.writeFieldEnd(); + } + } + if (struct.is_set_num_tasks()) { + oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC); + oprot.writeI32(struct.num_tasks); + oprot.writeFieldEnd(); + } + if (struct.is_set_num_workers()) { + oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); + oprot.writeI32(struct.num_workers); + oprot.writeFieldEnd(); + } + if (struct.is_set_num_executors()) { + oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC); + oprot.writeI32(struct.num_executors); + oprot.writeFieldEnd(); + } + if (struct.topology_conf != null) { + if (struct.is_set_topology_conf()) { + oprot.writeFieldBegin(TOPOLOGY_CONF_FIELD_DESC); + oprot.writeString(struct.topology_conf); + oprot.writeFieldEnd(); + } + } + if (struct.id_to_spout_agg_stats != null) { + if (struct.is_set_id_to_spout_agg_stats()) { + oprot.writeFieldBegin(ID_TO_SPOUT_AGG_STATS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.id_to_spout_agg_stats.size())); + for (Map.Entry _iter392 : struct.id_to_spout_agg_stats.entrySet()) + { + oprot.writeString(_iter392.getKey()); + _iter392.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.id_to_bolt_agg_stats != null) { + if (struct.is_set_id_to_bolt_agg_stats()) { + oprot.writeFieldBegin(ID_TO_BOLT_AGG_STATS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.id_to_bolt_agg_stats.size())); + for (Map.Entry _iter393 : struct.id_to_bolt_agg_stats.entrySet()) + { + oprot.writeString(_iter393.getKey()); + _iter393.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.sched_status != null) { + if (struct.is_set_sched_status()) { + oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC); + oprot.writeString(struct.sched_status); + oprot.writeFieldEnd(); + } + } + if (struct.topology_stats != null) { + if (struct.is_set_topology_stats()) { + oprot.writeFieldBegin(TOPOLOGY_STATS_FIELD_DESC); + struct.topology_stats.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.owner != null) { + if (struct.is_set_owner()) { + oprot.writeFieldBegin(OWNER_FIELD_DESC); + oprot.writeString(struct.owner); + oprot.writeFieldEnd(); + } + } + if (struct.debug_options != null) { + if (struct.is_set_debug_options()) { + oprot.writeFieldBegin(DEBUG_OPTIONS_FIELD_DESC); + struct.debug_options.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.is_set_replication_count()) { + oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC); + oprot.writeI32(struct.replication_count); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TopologyPageInfoTupleSchemeFactory implements SchemeFactory { + public TopologyPageInfoTupleScheme getScheme() { + return new TopologyPageInfoTupleScheme(); + } + } + + private static class TopologyPageInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TopologyPageInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.id); + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_uptime_secs()) { + optionals.set(1); + } + if (struct.is_set_status()) { + optionals.set(2); + } + if (struct.is_set_num_tasks()) { + optionals.set(3); + } + if (struct.is_set_num_workers()) { + optionals.set(4); + } + if (struct.is_set_num_executors()) { + optionals.set(5); + } + if (struct.is_set_topology_conf()) { + optionals.set(6); + } + if (struct.is_set_id_to_spout_agg_stats()) { + optionals.set(7); + } + if (struct.is_set_id_to_bolt_agg_stats()) { + optionals.set(8); + } + if (struct.is_set_sched_status()) { + optionals.set(9); + } + if (struct.is_set_topology_stats()) { + optionals.set(10); + } + if (struct.is_set_owner()) { + optionals.set(11); + } + if (struct.is_set_debug_options()) { + optionals.set(12); + } + if (struct.is_set_replication_count()) { + optionals.set(13); + } + oprot.writeBitSet(optionals, 14); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_uptime_secs()) { + oprot.writeI32(struct.uptime_secs); + } + if (struct.is_set_status()) { + oprot.writeString(struct.status); + } + if (struct.is_set_num_tasks()) { + oprot.writeI32(struct.num_tasks); + } + if (struct.is_set_num_workers()) { + oprot.writeI32(struct.num_workers); + } + if (struct.is_set_num_executors()) { + oprot.writeI32(struct.num_executors); + } + if (struct.is_set_topology_conf()) { + oprot.writeString(struct.topology_conf); + } + if (struct.is_set_id_to_spout_agg_stats()) { + { + oprot.writeI32(struct.id_to_spout_agg_stats.size()); + for (Map.Entry _iter394 : struct.id_to_spout_agg_stats.entrySet()) + { + oprot.writeString(_iter394.getKey()); + _iter394.getValue().write(oprot); + } + } + } + if (struct.is_set_id_to_bolt_agg_stats()) { + { + oprot.writeI32(struct.id_to_bolt_agg_stats.size()); + for (Map.Entry _iter395 : struct.id_to_bolt_agg_stats.entrySet()) + { + oprot.writeString(_iter395.getKey()); + _iter395.getValue().write(oprot); + } + } + } + if (struct.is_set_sched_status()) { + oprot.writeString(struct.sched_status); + } + if (struct.is_set_topology_stats()) { + struct.topology_stats.write(oprot); + } + if (struct.is_set_owner()) { + oprot.writeString(struct.owner); + } + if (struct.is_set_debug_options()) { + struct.debug_options.write(oprot); + } + if (struct.is_set_replication_count()) { + oprot.writeI32(struct.replication_count); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TopologyPageInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.id = iprot.readString(); + struct.set_id_isSet(true); + BitSet incoming = iprot.readBitSet(14); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.uptime_secs = iprot.readI32(); + struct.set_uptime_secs_isSet(true); + } + if (incoming.get(2)) { + struct.status = iprot.readString(); + struct.set_status_isSet(true); + } + if (incoming.get(3)) { + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + } + if (incoming.get(4)) { + struct.num_workers = iprot.readI32(); + struct.set_num_workers_isSet(true); + } + if (incoming.get(5)) { + struct.num_executors = iprot.readI32(); + struct.set_num_executors_isSet(true); + } + if (incoming.get(6)) { + struct.topology_conf = iprot.readString(); + struct.set_topology_conf_isSet(true); + } + if (incoming.get(7)) { + { + org.apache.thrift.protocol.TMap _map396 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.id_to_spout_agg_stats = new HashMap(2*_map396.size); + String _key397; + ComponentAggregateStats _val398; + for (int _i399 = 0; _i399 < _map396.size; ++_i399) + { + _key397 = iprot.readString(); + _val398 = new ComponentAggregateStats(); + _val398.read(iprot); + struct.id_to_spout_agg_stats.put(_key397, _val398); + } + } + struct.set_id_to_spout_agg_stats_isSet(true); + } + if (incoming.get(8)) { + { + org.apache.thrift.protocol.TMap _map400 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.id_to_bolt_agg_stats = new HashMap(2*_map400.size); + String _key401; + ComponentAggregateStats _val402; + for (int _i403 = 0; _i403 < _map400.size; ++_i403) + { + _key401 = iprot.readString(); + _val402 = new ComponentAggregateStats(); + _val402.read(iprot); + struct.id_to_bolt_agg_stats.put(_key401, _val402); + } + } + struct.set_id_to_bolt_agg_stats_isSet(true); + } + if (incoming.get(9)) { + struct.sched_status = iprot.readString(); + struct.set_sched_status_isSet(true); + } + if (incoming.get(10)) { + struct.topology_stats = new TopologyStats(); + struct.topology_stats.read(iprot); + struct.set_topology_stats_isSet(true); + } + if (incoming.get(11)) { + struct.owner = iprot.readString(); + struct.set_owner_isSet(true); + } + if (incoming.get(12)) { + struct.debug_options = new DebugOptions(); + struct.debug_options.read(iprot); + struct.set_debug_options_isSet(true); + } + if (incoming.get(13)) { + struct.replication_count = iprot.readI32(); + struct.set_replication_count_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java new file mode 100644 index 00000000000..453ead23a0a --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java @@ -0,0 +1,1094 @@ +/** + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28") +public class TopologyStats implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyStats"); + + private static final org.apache.thrift.protocol.TField WINDOW_TO_EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_emitted", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField WINDOW_TO_TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_transferred", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField WINDOW_TO_COMPLETE_LATENCIES_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_complete_latencies_ms", org.apache.thrift.protocol.TType.MAP, (short)3); + private static final org.apache.thrift.protocol.TField WINDOW_TO_ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_acked", org.apache.thrift.protocol.TType.MAP, (short)4); + private static final org.apache.thrift.protocol.TField WINDOW_TO_FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_failed", org.apache.thrift.protocol.TType.MAP, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologyStatsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologyStatsTupleSchemeFactory()); + } + + private Map window_to_emitted; // optional + private Map window_to_transferred; // optional + private Map window_to_complete_latencies_ms; // optional + private Map window_to_acked; // optional + private Map window_to_failed; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + WINDOW_TO_EMITTED((short)1, "window_to_emitted"), + WINDOW_TO_TRANSFERRED((short)2, "window_to_transferred"), + WINDOW_TO_COMPLETE_LATENCIES_MS((short)3, "window_to_complete_latencies_ms"), + WINDOW_TO_ACKED((short)4, "window_to_acked"), + WINDOW_TO_FAILED((short)5, "window_to_failed"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // WINDOW_TO_EMITTED + return WINDOW_TO_EMITTED; + case 2: // WINDOW_TO_TRANSFERRED + return WINDOW_TO_TRANSFERRED; + case 3: // WINDOW_TO_COMPLETE_LATENCIES_MS + return WINDOW_TO_COMPLETE_LATENCIES_MS; + case 4: // WINDOW_TO_ACKED + return WINDOW_TO_ACKED; + case 5: // WINDOW_TO_FAILED + return WINDOW_TO_FAILED; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.WINDOW_TO_EMITTED,_Fields.WINDOW_TO_TRANSFERRED,_Fields.WINDOW_TO_COMPLETE_LATENCIES_MS,_Fields.WINDOW_TO_ACKED,_Fields.WINDOW_TO_FAILED}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.WINDOW_TO_EMITTED, new org.apache.thrift.meta_data.FieldMetaData("window_to_emitted", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + tmpMap.put(_Fields.WINDOW_TO_TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("window_to_transferred", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + tmpMap.put(_Fields.WINDOW_TO_COMPLETE_LATENCIES_MS, new org.apache.thrift.meta_data.FieldMetaData("window_to_complete_latencies_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))); + tmpMap.put(_Fields.WINDOW_TO_ACKED, new org.apache.thrift.meta_data.FieldMetaData("window_to_acked", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + tmpMap.put(_Fields.WINDOW_TO_FAILED, new org.apache.thrift.meta_data.FieldMetaData("window_to_failed", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyStats.class, metaDataMap); + } + + public TopologyStats() { + } + + /** + * Performs a deep copy on other. + */ + public TopologyStats(TopologyStats other) { + if (other.is_set_window_to_emitted()) { + Map __this__window_to_emitted = new HashMap(other.window_to_emitted); + this.window_to_emitted = __this__window_to_emitted; + } + if (other.is_set_window_to_transferred()) { + Map __this__window_to_transferred = new HashMap(other.window_to_transferred); + this.window_to_transferred = __this__window_to_transferred; + } + if (other.is_set_window_to_complete_latencies_ms()) { + Map __this__window_to_complete_latencies_ms = new HashMap(other.window_to_complete_latencies_ms); + this.window_to_complete_latencies_ms = __this__window_to_complete_latencies_ms; + } + if (other.is_set_window_to_acked()) { + Map __this__window_to_acked = new HashMap(other.window_to_acked); + this.window_to_acked = __this__window_to_acked; + } + if (other.is_set_window_to_failed()) { + Map __this__window_to_failed = new HashMap(other.window_to_failed); + this.window_to_failed = __this__window_to_failed; + } + } + + public TopologyStats deepCopy() { + return new TopologyStats(this); + } + + @Override + public void clear() { + this.window_to_emitted = null; + this.window_to_transferred = null; + this.window_to_complete_latencies_ms = null; + this.window_to_acked = null; + this.window_to_failed = null; + } + + public int get_window_to_emitted_size() { + return (this.window_to_emitted == null) ? 0 : this.window_to_emitted.size(); + } + + public void put_to_window_to_emitted(String key, long val) { + if (this.window_to_emitted == null) { + this.window_to_emitted = new HashMap(); + } + this.window_to_emitted.put(key, val); + } + + public Map get_window_to_emitted() { + return this.window_to_emitted; + } + + public void set_window_to_emitted(Map window_to_emitted) { + this.window_to_emitted = window_to_emitted; + } + + public void unset_window_to_emitted() { + this.window_to_emitted = null; + } + + /** Returns true if field window_to_emitted is set (has been assigned a value) and false otherwise */ + public boolean is_set_window_to_emitted() { + return this.window_to_emitted != null; + } + + public void set_window_to_emitted_isSet(boolean value) { + if (!value) { + this.window_to_emitted = null; + } + } + + public int get_window_to_transferred_size() { + return (this.window_to_transferred == null) ? 0 : this.window_to_transferred.size(); + } + + public void put_to_window_to_transferred(String key, long val) { + if (this.window_to_transferred == null) { + this.window_to_transferred = new HashMap(); + } + this.window_to_transferred.put(key, val); + } + + public Map get_window_to_transferred() { + return this.window_to_transferred; + } + + public void set_window_to_transferred(Map window_to_transferred) { + this.window_to_transferred = window_to_transferred; + } + + public void unset_window_to_transferred() { + this.window_to_transferred = null; + } + + /** Returns true if field window_to_transferred is set (has been assigned a value) and false otherwise */ + public boolean is_set_window_to_transferred() { + return this.window_to_transferred != null; + } + + public void set_window_to_transferred_isSet(boolean value) { + if (!value) { + this.window_to_transferred = null; + } + } + + public int get_window_to_complete_latencies_ms_size() { + return (this.window_to_complete_latencies_ms == null) ? 0 : this.window_to_complete_latencies_ms.size(); + } + + public void put_to_window_to_complete_latencies_ms(String key, double val) { + if (this.window_to_complete_latencies_ms == null) { + this.window_to_complete_latencies_ms = new HashMap(); + } + this.window_to_complete_latencies_ms.put(key, val); + } + + public Map get_window_to_complete_latencies_ms() { + return this.window_to_complete_latencies_ms; + } + + public void set_window_to_complete_latencies_ms(Map window_to_complete_latencies_ms) { + this.window_to_complete_latencies_ms = window_to_complete_latencies_ms; + } + + public void unset_window_to_complete_latencies_ms() { + this.window_to_complete_latencies_ms = null; + } + + /** Returns true if field window_to_complete_latencies_ms is set (has been assigned a value) and false otherwise */ + public boolean is_set_window_to_complete_latencies_ms() { + return this.window_to_complete_latencies_ms != null; + } + + public void set_window_to_complete_latencies_ms_isSet(boolean value) { + if (!value) { + this.window_to_complete_latencies_ms = null; + } + } + + public int get_window_to_acked_size() { + return (this.window_to_acked == null) ? 0 : this.window_to_acked.size(); + } + + public void put_to_window_to_acked(String key, long val) { + if (this.window_to_acked == null) { + this.window_to_acked = new HashMap(); + } + this.window_to_acked.put(key, val); + } + + public Map get_window_to_acked() { + return this.window_to_acked; + } + + public void set_window_to_acked(Map window_to_acked) { + this.window_to_acked = window_to_acked; + } + + public void unset_window_to_acked() { + this.window_to_acked = null; + } + + /** Returns true if field window_to_acked is set (has been assigned a value) and false otherwise */ + public boolean is_set_window_to_acked() { + return this.window_to_acked != null; + } + + public void set_window_to_acked_isSet(boolean value) { + if (!value) { + this.window_to_acked = null; + } + } + + public int get_window_to_failed_size() { + return (this.window_to_failed == null) ? 0 : this.window_to_failed.size(); + } + + public void put_to_window_to_failed(String key, long val) { + if (this.window_to_failed == null) { + this.window_to_failed = new HashMap(); + } + this.window_to_failed.put(key, val); + } + + public Map get_window_to_failed() { + return this.window_to_failed; + } + + public void set_window_to_failed(Map window_to_failed) { + this.window_to_failed = window_to_failed; + } + + public void unset_window_to_failed() { + this.window_to_failed = null; + } + + /** Returns true if field window_to_failed is set (has been assigned a value) and false otherwise */ + public boolean is_set_window_to_failed() { + return this.window_to_failed != null; + } + + public void set_window_to_failed_isSet(boolean value) { + if (!value) { + this.window_to_failed = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case WINDOW_TO_EMITTED: + if (value == null) { + unset_window_to_emitted(); + } else { + set_window_to_emitted((Map)value); + } + break; + + case WINDOW_TO_TRANSFERRED: + if (value == null) { + unset_window_to_transferred(); + } else { + set_window_to_transferred((Map)value); + } + break; + + case WINDOW_TO_COMPLETE_LATENCIES_MS: + if (value == null) { + unset_window_to_complete_latencies_ms(); + } else { + set_window_to_complete_latencies_ms((Map)value); + } + break; + + case WINDOW_TO_ACKED: + if (value == null) { + unset_window_to_acked(); + } else { + set_window_to_acked((Map)value); + } + break; + + case WINDOW_TO_FAILED: + if (value == null) { + unset_window_to_failed(); + } else { + set_window_to_failed((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case WINDOW_TO_EMITTED: + return get_window_to_emitted(); + + case WINDOW_TO_TRANSFERRED: + return get_window_to_transferred(); + + case WINDOW_TO_COMPLETE_LATENCIES_MS: + return get_window_to_complete_latencies_ms(); + + case WINDOW_TO_ACKED: + return get_window_to_acked(); + + case WINDOW_TO_FAILED: + return get_window_to_failed(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case WINDOW_TO_EMITTED: + return is_set_window_to_emitted(); + case WINDOW_TO_TRANSFERRED: + return is_set_window_to_transferred(); + case WINDOW_TO_COMPLETE_LATENCIES_MS: + return is_set_window_to_complete_latencies_ms(); + case WINDOW_TO_ACKED: + return is_set_window_to_acked(); + case WINDOW_TO_FAILED: + return is_set_window_to_failed(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TopologyStats) + return this.equals((TopologyStats)that); + return false; + } + + public boolean equals(TopologyStats that) { + if (that == null) + return false; + + boolean this_present_window_to_emitted = true && this.is_set_window_to_emitted(); + boolean that_present_window_to_emitted = true && that.is_set_window_to_emitted(); + if (this_present_window_to_emitted || that_present_window_to_emitted) { + if (!(this_present_window_to_emitted && that_present_window_to_emitted)) + return false; + if (!this.window_to_emitted.equals(that.window_to_emitted)) + return false; + } + + boolean this_present_window_to_transferred = true && this.is_set_window_to_transferred(); + boolean that_present_window_to_transferred = true && that.is_set_window_to_transferred(); + if (this_present_window_to_transferred || that_present_window_to_transferred) { + if (!(this_present_window_to_transferred && that_present_window_to_transferred)) + return false; + if (!this.window_to_transferred.equals(that.window_to_transferred)) + return false; + } + + boolean this_present_window_to_complete_latencies_ms = true && this.is_set_window_to_complete_latencies_ms(); + boolean that_present_window_to_complete_latencies_ms = true && that.is_set_window_to_complete_latencies_ms(); + if (this_present_window_to_complete_latencies_ms || that_present_window_to_complete_latencies_ms) { + if (!(this_present_window_to_complete_latencies_ms && that_present_window_to_complete_latencies_ms)) + return false; + if (!this.window_to_complete_latencies_ms.equals(that.window_to_complete_latencies_ms)) + return false; + } + + boolean this_present_window_to_acked = true && this.is_set_window_to_acked(); + boolean that_present_window_to_acked = true && that.is_set_window_to_acked(); + if (this_present_window_to_acked || that_present_window_to_acked) { + if (!(this_present_window_to_acked && that_present_window_to_acked)) + return false; + if (!this.window_to_acked.equals(that.window_to_acked)) + return false; + } + + boolean this_present_window_to_failed = true && this.is_set_window_to_failed(); + boolean that_present_window_to_failed = true && that.is_set_window_to_failed(); + if (this_present_window_to_failed || that_present_window_to_failed) { + if (!(this_present_window_to_failed && that_present_window_to_failed)) + return false; + if (!this.window_to_failed.equals(that.window_to_failed)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_window_to_emitted = true && (is_set_window_to_emitted()); + list.add(present_window_to_emitted); + if (present_window_to_emitted) + list.add(window_to_emitted); + + boolean present_window_to_transferred = true && (is_set_window_to_transferred()); + list.add(present_window_to_transferred); + if (present_window_to_transferred) + list.add(window_to_transferred); + + boolean present_window_to_complete_latencies_ms = true && (is_set_window_to_complete_latencies_ms()); + list.add(present_window_to_complete_latencies_ms); + if (present_window_to_complete_latencies_ms) + list.add(window_to_complete_latencies_ms); + + boolean present_window_to_acked = true && (is_set_window_to_acked()); + list.add(present_window_to_acked); + if (present_window_to_acked) + list.add(window_to_acked); + + boolean present_window_to_failed = true && (is_set_window_to_failed()); + list.add(present_window_to_failed); + if (present_window_to_failed) + list.add(window_to_failed); + + return list.hashCode(); + } + + @Override + public int compareTo(TopologyStats other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_window_to_emitted()).compareTo(other.is_set_window_to_emitted()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window_to_emitted()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_emitted, other.window_to_emitted); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_window_to_transferred()).compareTo(other.is_set_window_to_transferred()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window_to_transferred()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_transferred, other.window_to_transferred); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_window_to_complete_latencies_ms()).compareTo(other.is_set_window_to_complete_latencies_ms()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window_to_complete_latencies_ms()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_complete_latencies_ms, other.window_to_complete_latencies_ms); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_window_to_acked()).compareTo(other.is_set_window_to_acked()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window_to_acked()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_acked, other.window_to_acked); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_window_to_failed()).compareTo(other.is_set_window_to_failed()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_window_to_failed()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_failed, other.window_to_failed); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TopologyStats("); + boolean first = true; + + if (is_set_window_to_emitted()) { + sb.append("window_to_emitted:"); + if (this.window_to_emitted == null) { + sb.append("null"); + } else { + sb.append(this.window_to_emitted); + } + first = false; + } + if (is_set_window_to_transferred()) { + if (!first) sb.append(", "); + sb.append("window_to_transferred:"); + if (this.window_to_transferred == null) { + sb.append("null"); + } else { + sb.append(this.window_to_transferred); + } + first = false; + } + if (is_set_window_to_complete_latencies_ms()) { + if (!first) sb.append(", "); + sb.append("window_to_complete_latencies_ms:"); + if (this.window_to_complete_latencies_ms == null) { + sb.append("null"); + } else { + sb.append(this.window_to_complete_latencies_ms); + } + first = false; + } + if (is_set_window_to_acked()) { + if (!first) sb.append(", "); + sb.append("window_to_acked:"); + if (this.window_to_acked == null) { + sb.append("null"); + } else { + sb.append(this.window_to_acked); + } + first = false; + } + if (is_set_window_to_failed()) { + if (!first) sb.append(", "); + sb.append("window_to_failed:"); + if (this.window_to_failed == null) { + sb.append("null"); + } else { + sb.append(this.window_to_failed); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TopologyStatsStandardSchemeFactory implements SchemeFactory { + public TopologyStatsStandardScheme getScheme() { + return new TopologyStatsStandardScheme(); + } + } + + private static class TopologyStatsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyStats struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // WINDOW_TO_EMITTED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map334 = iprot.readMapBegin(); + struct.window_to_emitted = new HashMap(2*_map334.size); + String _key335; + long _val336; + for (int _i337 = 0; _i337 < _map334.size; ++_i337) + { + _key335 = iprot.readString(); + _val336 = iprot.readI64(); + struct.window_to_emitted.put(_key335, _val336); + } + iprot.readMapEnd(); + } + struct.set_window_to_emitted_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // WINDOW_TO_TRANSFERRED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map338 = iprot.readMapBegin(); + struct.window_to_transferred = new HashMap(2*_map338.size); + String _key339; + long _val340; + for (int _i341 = 0; _i341 < _map338.size; ++_i341) + { + _key339 = iprot.readString(); + _val340 = iprot.readI64(); + struct.window_to_transferred.put(_key339, _val340); + } + iprot.readMapEnd(); + } + struct.set_window_to_transferred_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // WINDOW_TO_COMPLETE_LATENCIES_MS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map342 = iprot.readMapBegin(); + struct.window_to_complete_latencies_ms = new HashMap(2*_map342.size); + String _key343; + double _val344; + for (int _i345 = 0; _i345 < _map342.size; ++_i345) + { + _key343 = iprot.readString(); + _val344 = iprot.readDouble(); + struct.window_to_complete_latencies_ms.put(_key343, _val344); + } + iprot.readMapEnd(); + } + struct.set_window_to_complete_latencies_ms_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // WINDOW_TO_ACKED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map346 = iprot.readMapBegin(); + struct.window_to_acked = new HashMap(2*_map346.size); + String _key347; + long _val348; + for (int _i349 = 0; _i349 < _map346.size; ++_i349) + { + _key347 = iprot.readString(); + _val348 = iprot.readI64(); + struct.window_to_acked.put(_key347, _val348); + } + iprot.readMapEnd(); + } + struct.set_window_to_acked_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // WINDOW_TO_FAILED + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map350 = iprot.readMapBegin(); + struct.window_to_failed = new HashMap(2*_map350.size); + String _key351; + long _val352; + for (int _i353 = 0; _i353 < _map350.size; ++_i353) + { + _key351 = iprot.readString(); + _val352 = iprot.readI64(); + struct.window_to_failed.put(_key351, _val352); + } + iprot.readMapEnd(); + } + struct.set_window_to_failed_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyStats struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.window_to_emitted != null) { + if (struct.is_set_window_to_emitted()) { + oprot.writeFieldBegin(WINDOW_TO_EMITTED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_emitted.size())); + for (Map.Entry _iter354 : struct.window_to_emitted.entrySet()) + { + oprot.writeString(_iter354.getKey()); + oprot.writeI64(_iter354.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.window_to_transferred != null) { + if (struct.is_set_window_to_transferred()) { + oprot.writeFieldBegin(WINDOW_TO_TRANSFERRED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_transferred.size())); + for (Map.Entry _iter355 : struct.window_to_transferred.entrySet()) + { + oprot.writeString(_iter355.getKey()); + oprot.writeI64(_iter355.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.window_to_complete_latencies_ms != null) { + if (struct.is_set_window_to_complete_latencies_ms()) { + oprot.writeFieldBegin(WINDOW_TO_COMPLETE_LATENCIES_MS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.window_to_complete_latencies_ms.size())); + for (Map.Entry _iter356 : struct.window_to_complete_latencies_ms.entrySet()) + { + oprot.writeString(_iter356.getKey()); + oprot.writeDouble(_iter356.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.window_to_acked != null) { + if (struct.is_set_window_to_acked()) { + oprot.writeFieldBegin(WINDOW_TO_ACKED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_acked.size())); + for (Map.Entry _iter357 : struct.window_to_acked.entrySet()) + { + oprot.writeString(_iter357.getKey()); + oprot.writeI64(_iter357.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.window_to_failed != null) { + if (struct.is_set_window_to_failed()) { + oprot.writeFieldBegin(WINDOW_TO_FAILED_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_failed.size())); + for (Map.Entry _iter358 : struct.window_to_failed.entrySet()) + { + oprot.writeString(_iter358.getKey()); + oprot.writeI64(_iter358.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TopologyStatsTupleSchemeFactory implements SchemeFactory { + public TopologyStatsTupleScheme getScheme() { + return new TopologyStatsTupleScheme(); + } + } + + private static class TopologyStatsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TopologyStats struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_window_to_emitted()) { + optionals.set(0); + } + if (struct.is_set_window_to_transferred()) { + optionals.set(1); + } + if (struct.is_set_window_to_complete_latencies_ms()) { + optionals.set(2); + } + if (struct.is_set_window_to_acked()) { + optionals.set(3); + } + if (struct.is_set_window_to_failed()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); + if (struct.is_set_window_to_emitted()) { + { + oprot.writeI32(struct.window_to_emitted.size()); + for (Map.Entry _iter359 : struct.window_to_emitted.entrySet()) + { + oprot.writeString(_iter359.getKey()); + oprot.writeI64(_iter359.getValue()); + } + } + } + if (struct.is_set_window_to_transferred()) { + { + oprot.writeI32(struct.window_to_transferred.size()); + for (Map.Entry _iter360 : struct.window_to_transferred.entrySet()) + { + oprot.writeString(_iter360.getKey()); + oprot.writeI64(_iter360.getValue()); + } + } + } + if (struct.is_set_window_to_complete_latencies_ms()) { + { + oprot.writeI32(struct.window_to_complete_latencies_ms.size()); + for (Map.Entry _iter361 : struct.window_to_complete_latencies_ms.entrySet()) + { + oprot.writeString(_iter361.getKey()); + oprot.writeDouble(_iter361.getValue()); + } + } + } + if (struct.is_set_window_to_acked()) { + { + oprot.writeI32(struct.window_to_acked.size()); + for (Map.Entry _iter362 : struct.window_to_acked.entrySet()) + { + oprot.writeString(_iter362.getKey()); + oprot.writeI64(_iter362.getValue()); + } + } + } + if (struct.is_set_window_to_failed()) { + { + oprot.writeI32(struct.window_to_failed.size()); + for (Map.Entry _iter363 : struct.window_to_failed.entrySet()) + { + oprot.writeString(_iter363.getKey()); + oprot.writeI64(_iter363.getValue()); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TopologyStats struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(5); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TMap _map364 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.window_to_emitted = new HashMap(2*_map364.size); + String _key365; + long _val366; + for (int _i367 = 0; _i367 < _map364.size; ++_i367) + { + _key365 = iprot.readString(); + _val366 = iprot.readI64(); + struct.window_to_emitted.put(_key365, _val366); + } + } + struct.set_window_to_emitted_isSet(true); + } + if (incoming.get(1)) { + { + org.apache.thrift.protocol.TMap _map368 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.window_to_transferred = new HashMap(2*_map368.size); + String _key369; + long _val370; + for (int _i371 = 0; _i371 < _map368.size; ++_i371) + { + _key369 = iprot.readString(); + _val370 = iprot.readI64(); + struct.window_to_transferred.put(_key369, _val370); + } + } + struct.set_window_to_transferred_isSet(true); + } + if (incoming.get(2)) { + { + org.apache.thrift.protocol.TMap _map372 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + struct.window_to_complete_latencies_ms = new HashMap(2*_map372.size); + String _key373; + double _val374; + for (int _i375 = 0; _i375 < _map372.size; ++_i375) + { + _key373 = iprot.readString(); + _val374 = iprot.readDouble(); + struct.window_to_complete_latencies_ms.put(_key373, _val374); + } + } + struct.set_window_to_complete_latencies_ms_isSet(true); + } + if (incoming.get(3)) { + { + org.apache.thrift.protocol.TMap _map376 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.window_to_acked = new HashMap(2*_map376.size); + String _key377; + long _val378; + for (int _i379 = 0; _i379 < _map376.size; ++_i379) + { + _key377 = iprot.readString(); + _val378 = iprot.readI64(); + struct.window_to_acked.put(_key377, _val378); + } + } + struct.set_window_to_acked_isSet(true); + } + if (incoming.get(4)) { + { + org.apache.thrift.protocol.TMap _map380 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.window_to_failed = new HashMap(2*_map380.size); + String _key381; + long _val382; + for (int _i383 = 0; _i383 < _map380.size; ++_i383) + { + _key381 = iprot.readString(); + _val382 = iprot.readI64(); + struct.window_to_failed.put(_key381, _val382); + } + } + struct.set_window_to_failed_isSet(true); + } + } + } + +} + diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java index e332a133d9f..c77d2015819 100644 --- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java +++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java @@ -44,7 +44,20 @@ public class SimpleACLAuthorizer implements IAuthorizer { protected Set _userCommands = new HashSet(Arrays.asList("submitTopology", "fileUpload", "getNimbusConf", "getClusterInfo")); protected Set _supervisorCommands = new HashSet(Arrays.asList("fileDownload")); - protected Set _topoCommands = new HashSet(Arrays.asList("killTopology","rebalance","activate","deactivate","getTopologyConf","getTopology","getUserTopology","getTopologyInfo","uploadNewCredentials","setLogConfig","getLogConfig")); + protected Set _topoCommands = new HashSet(Arrays.asList( + "killTopology", + "rebalance", + "activate", + "deactivate", + "getTopologyConf", + "getTopology", + "getUserTopology", + "getTopologyInfo", + "getTopologyPageInfo", + "getComponentPageInfo", + "uploadNewCredentials", + "setLogConfig", + "getLogConfig")); protected Set _admins; protected Set _supervisors; diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote index c5b4b0c5d54..d05a4b2b5d1 100644 --- a/storm-core/src/py/storm/Nimbus-remote +++ b/storm-core/src/py/storm/Nimbus-remote @@ -62,6 +62,8 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help': print(' ClusterSummary getClusterInfo()') print(' TopologyInfo getTopologyInfo(string id)') print(' TopologyInfo getTopologyInfoWithOpts(string id, GetInfoOptions options)') + print(' TopologyPageInfo getTopologyPageInfo(string id, string window, bool is_include_sys)') + print(' ComponentPageInfo getComponentPageInfo(string topology_id, string component_id, string window, bool is_include_sys)') print(' string getTopologyConf(string id)') print(' StormTopology getTopology(string id)') print(' StormTopology getUserTopology(string id)') @@ -241,6 +243,18 @@ elif cmd == 'getTopologyInfoWithOpts': sys.exit(1) pp.pprint(client.getTopologyInfoWithOpts(args[0],eval(args[1]),)) +elif cmd == 'getTopologyPageInfo': + if len(args) != 3: + print('getTopologyPageInfo requires 3 args') + sys.exit(1) + pp.pprint(client.getTopologyPageInfo(args[0],args[1],eval(args[2]),)) + +elif cmd == 'getComponentPageInfo': + if len(args) != 4: + print('getComponentPageInfo requires 4 args') + sys.exit(1) + pp.pprint(client.getComponentPageInfo(args[0],args[1],args[2],eval(args[3]),)) + elif cmd == 'getTopologyConf': if len(args) != 1: print('getTopologyConf requires 1 args') diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py index cf7adbf47c5..3c26a1673e1 100644 --- a/storm-core/src/py/storm/Nimbus.py +++ b/storm-core/src/py/storm/Nimbus.py @@ -186,6 +186,25 @@ def getTopologyInfoWithOpts(self, id, options): """ pass + def getTopologyPageInfo(self, id, window, is_include_sys): + """ + Parameters: + - id + - window + - is_include_sys + """ + pass + + def getComponentPageInfo(self, topology_id, component_id, window, is_include_sys): + """ + Parameters: + - topology_id + - component_id + - window + - is_include_sys + """ + pass + def getTopologyConf(self, id): """ Parameters: @@ -904,6 +923,86 @@ def recv_getTopologyInfoWithOpts(self): raise result.aze raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result"); + def getTopologyPageInfo(self, id, window, is_include_sys): + """ + Parameters: + - id + - window + - is_include_sys + """ + self.send_getTopologyPageInfo(id, window, is_include_sys) + return self.recv_getTopologyPageInfo() + + def send_getTopologyPageInfo(self, id, window, is_include_sys): + self._oprot.writeMessageBegin('getTopologyPageInfo', TMessageType.CALL, self._seqid) + args = getTopologyPageInfo_args() + args.id = id + args.window = window + args.is_include_sys = is_include_sys + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getTopologyPageInfo(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getTopologyPageInfo_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + if result.aze is not None: + raise result.aze + raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyPageInfo failed: unknown result"); + + def getComponentPageInfo(self, topology_id, component_id, window, is_include_sys): + """ + Parameters: + - topology_id + - component_id + - window + - is_include_sys + """ + self.send_getComponentPageInfo(topology_id, component_id, window, is_include_sys) + return self.recv_getComponentPageInfo() + + def send_getComponentPageInfo(self, topology_id, component_id, window, is_include_sys): + self._oprot.writeMessageBegin('getComponentPageInfo', TMessageType.CALL, self._seqid) + args = getComponentPageInfo_args() + args.topology_id = topology_id + args.component_id = component_id + args.window = window + args.is_include_sys = is_include_sys + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getComponentPageInfo(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getComponentPageInfo_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + if result.aze is not None: + raise result.aze + raise TApplicationException(TApplicationException.MISSING_RESULT, "getComponentPageInfo failed: unknown result"); + def getTopologyConf(self, id): """ Parameters: @@ -1038,6 +1137,8 @@ def __init__(self, handler): self._processMap["getClusterInfo"] = Processor.process_getClusterInfo self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo self._processMap["getTopologyInfoWithOpts"] = Processor.process_getTopologyInfoWithOpts + self._processMap["getTopologyPageInfo"] = Processor.process_getTopologyPageInfo + self._processMap["getComponentPageInfo"] = Processor.process_getComponentPageInfo self._processMap["getTopologyConf"] = Processor.process_getTopologyConf self._processMap["getTopology"] = Processor.process_getTopology self._processMap["getUserTopology"] = Processor.process_getUserTopology @@ -1361,6 +1462,38 @@ def process_getTopologyInfoWithOpts(self, seqid, iprot, oprot): oprot.writeMessageEnd() oprot.trans.flush() + def process_getTopologyPageInfo(self, seqid, iprot, oprot): + args = getTopologyPageInfo_args() + args.read(iprot) + iprot.readMessageEnd() + result = getTopologyPageInfo_result() + try: + result.success = self._handler.getTopologyPageInfo(args.id, args.window, args.is_include_sys) + except NotAliveException, e: + result.e = e + except AuthorizationException, aze: + result.aze = aze + oprot.writeMessageBegin("getTopologyPageInfo", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getComponentPageInfo(self, seqid, iprot, oprot): + args = getComponentPageInfo_args() + args.read(iprot) + iprot.readMessageEnd() + result = getComponentPageInfo_result() + try: + result.success = self._handler.getComponentPageInfo(args.topology_id, args.component_id, args.window, args.is_include_sys) + except NotAliveException, e: + result.e = e + except AuthorizationException, aze: + result.aze = aze + oprot.writeMessageBegin("getComponentPageInfo", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + def process_getTopologyConf(self, seqid, iprot, oprot): args = getTopologyConf_args() args.read(iprot) @@ -4467,6 +4600,387 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) +class getTopologyPageInfo_args: + """ + Attributes: + - id + - window + - is_include_sys + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + (2, TType.STRING, 'window', None, None, ), # 2 + (3, TType.BOOL, 'is_include_sys', None, None, ), # 3 + ) + + def __init__(self, id=None, window=None, is_include_sys=None,): + self.id = id + self.window = window + self.is_include_sys = is_include_sys + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.window = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.BOOL: + self.is_include_sys = iprot.readBool(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyPageInfo_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + if self.window is not None: + oprot.writeFieldBegin('window', TType.STRING, 2) + oprot.writeString(self.window.encode('utf-8')) + oprot.writeFieldEnd() + if self.is_include_sys is not None: + oprot.writeFieldBegin('is_include_sys', TType.BOOL, 3) + oprot.writeBool(self.is_include_sys) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + value = (value * 31) ^ hash(self.window) + value = (value * 31) ^ hash(self.is_include_sys) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyPageInfo_result: + """ + Attributes: + - success + - e + - aze + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (TopologyPageInfo, TopologyPageInfo.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, e=None, aze=None,): + self.success = success + self.e = e + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = TopologyPageInfo() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyPageInfo_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getComponentPageInfo_args: + """ + Attributes: + - topology_id + - component_id + - window + - is_include_sys + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'topology_id', None, None, ), # 1 + (2, TType.STRING, 'component_id', None, None, ), # 2 + (3, TType.STRING, 'window', None, None, ), # 3 + (4, TType.BOOL, 'is_include_sys', None, None, ), # 4 + ) + + def __init__(self, topology_id=None, component_id=None, window=None, is_include_sys=None,): + self.topology_id = topology_id + self.component_id = component_id + self.window = window + self.is_include_sys = is_include_sys + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.topology_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.component_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.window = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.is_include_sys = iprot.readBool(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getComponentPageInfo_args') + if self.topology_id is not None: + oprot.writeFieldBegin('topology_id', TType.STRING, 1) + oprot.writeString(self.topology_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.component_id is not None: + oprot.writeFieldBegin('component_id', TType.STRING, 2) + oprot.writeString(self.component_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.window is not None: + oprot.writeFieldBegin('window', TType.STRING, 3) + oprot.writeString(self.window.encode('utf-8')) + oprot.writeFieldEnd() + if self.is_include_sys is not None: + oprot.writeFieldBegin('is_include_sys', TType.BOOL, 4) + oprot.writeBool(self.is_include_sys) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topology_id) + value = (value * 31) ^ hash(self.component_id) + value = (value * 31) ^ hash(self.window) + value = (value * 31) ^ hash(self.is_include_sys) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getComponentPageInfo_result: + """ + Attributes: + - success + - e + - aze + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (ComponentPageInfo, ComponentPageInfo.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2 + ) + + def __init__(self, success=None, e=None, aze=None,): + self.success = success + self.e = e + self.aze = aze + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = ComponentPageInfo() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.aze = AuthorizationException() + self.aze.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getComponentPageInfo_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.aze is not None: + oprot.writeFieldBegin('aze', TType.STRUCT, 2) + self.aze.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.aze) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + class getTopologyConf_args: """ Attributes: diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py index 18818583d08..d3655669955 100644 --- a/storm-core/src/py/storm/ttypes.py +++ b/storm-core/src/py/storm/ttypes.py @@ -34,6 +34,20 @@ fastbinary = None +class ComponentType: + BOLT = 1 + SPOUT = 2 + + _VALUES_TO_NAMES = { + 1: "BOLT", + 2: "SPOUT", + } + + _NAMES_TO_VALUES = { + "BOLT": 1, + "SPOUT": 2, + } + class TopologyInitialStatus: ACTIVE = 1 INACTIVE = 2 @@ -4540,19 +4554,34 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class KillOptions: +class CommonAggregateStats: """ Attributes: - - wait_secs + - num_executors + - num_tasks + - emitted + - transferred + - acked + - failed """ thrift_spec = ( None, # 0 - (1, TType.I32, 'wait_secs', None, None, ), # 1 + (1, TType.I32, 'num_executors', None, None, ), # 1 + (2, TType.I32, 'num_tasks', None, None, ), # 2 + (3, TType.I64, 'emitted', None, None, ), # 3 + (4, TType.I64, 'transferred', None, None, ), # 4 + (5, TType.I64, 'acked', None, None, ), # 5 + (6, TType.I64, 'failed', None, None, ), # 6 ) - def __init__(self, wait_secs=None,): - self.wait_secs = wait_secs + def __init__(self, num_executors=None, num_tasks=None, emitted=None, transferred=None, acked=None, failed=None,): + self.num_executors = num_executors + self.num_tasks = num_tasks + self.emitted = emitted + self.transferred = transferred + self.acked = acked + self.failed = failed def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4565,7 +4594,32 @@ def read(self, iprot): break if fid == 1: if ftype == TType.I32: - self.wait_secs = iprot.readI32(); + self.num_executors = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.num_tasks = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I64: + self.emitted = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I64: + self.transferred = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I64: + self.acked = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.I64: + self.failed = iprot.readI64(); else: iprot.skip(ftype) else: @@ -4577,10 +4631,30 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('KillOptions') - if self.wait_secs is not None: - oprot.writeFieldBegin('wait_secs', TType.I32, 1) - oprot.writeI32(self.wait_secs) + oprot.writeStructBegin('CommonAggregateStats') + if self.num_executors is not None: + oprot.writeFieldBegin('num_executors', TType.I32, 1) + oprot.writeI32(self.num_executors) + oprot.writeFieldEnd() + if self.num_tasks is not None: + oprot.writeFieldBegin('num_tasks', TType.I32, 2) + oprot.writeI32(self.num_tasks) + oprot.writeFieldEnd() + if self.emitted is not None: + oprot.writeFieldBegin('emitted', TType.I64, 3) + oprot.writeI64(self.emitted) + oprot.writeFieldEnd() + if self.transferred is not None: + oprot.writeFieldBegin('transferred', TType.I64, 4) + oprot.writeI64(self.transferred) + oprot.writeFieldEnd() + if self.acked is not None: + oprot.writeFieldBegin('acked', TType.I64, 5) + oprot.writeI64(self.acked) + oprot.writeFieldEnd() + if self.failed is not None: + oprot.writeFieldBegin('failed', TType.I64, 6) + oprot.writeI64(self.failed) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -4591,7 +4665,12 @@ def validate(self): def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.wait_secs) + value = (value * 31) ^ hash(self.num_executors) + value = (value * 31) ^ hash(self.num_tasks) + value = (value * 31) ^ hash(self.emitted) + value = (value * 31) ^ hash(self.transferred) + value = (value * 31) ^ hash(self.acked) + value = (value * 31) ^ hash(self.failed) return value def __repr__(self): @@ -4605,25 +4684,19 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class RebalanceOptions: +class SpoutAggregateStats: """ Attributes: - - wait_secs - - num_workers - - num_executors + - complete_latency_ms """ thrift_spec = ( None, # 0 - (1, TType.I32, 'wait_secs', None, None, ), # 1 - (2, TType.I32, 'num_workers', None, None, ), # 2 - (3, TType.MAP, 'num_executors', (TType.STRING,None,TType.I32,None), None, ), # 3 + (1, TType.DOUBLE, 'complete_latency_ms', None, None, ), # 1 ) - def __init__(self, wait_secs=None, num_workers=None, num_executors=None,): - self.wait_secs = wait_secs - self.num_workers = num_workers - self.num_executors = num_executors + def __init__(self, complete_latency_ms=None,): + self.complete_latency_ms = complete_latency_ms def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4635,24 +4708,8 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.I32: - self.wait_secs = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.num_workers = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.num_executors = {} - (_ktype300, _vtype301, _size299 ) = iprot.readMapBegin() - for _i303 in xrange(_size299): - _key304 = iprot.readString().decode('utf-8') - _val305 = iprot.readI32(); - self.num_executors[_key304] = _val305 - iprot.readMapEnd() + if ftype == TType.DOUBLE: + self.complete_latency_ms = iprot.readDouble(); else: iprot.skip(ftype) else: @@ -4664,22 +4721,10 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('RebalanceOptions') - if self.wait_secs is not None: - oprot.writeFieldBegin('wait_secs', TType.I32, 1) - oprot.writeI32(self.wait_secs) - oprot.writeFieldEnd() - if self.num_workers is not None: - oprot.writeFieldBegin('num_workers', TType.I32, 2) - oprot.writeI32(self.num_workers) - oprot.writeFieldEnd() - if self.num_executors is not None: - oprot.writeFieldBegin('num_executors', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.I32, len(self.num_executors)) - for kiter306,viter307 in self.num_executors.items(): - oprot.writeString(kiter306.encode('utf-8')) - oprot.writeI32(viter307) - oprot.writeMapEnd() + oprot.writeStructBegin('SpoutAggregateStats') + if self.complete_latency_ms is not None: + oprot.writeFieldBegin('complete_latency_ms', TType.DOUBLE, 1) + oprot.writeDouble(self.complete_latency_ms) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -4690,9 +4735,7 @@ def validate(self): def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.wait_secs) - value = (value * 31) ^ hash(self.num_workers) - value = (value * 31) ^ hash(self.num_executors) + value = (value * 31) ^ hash(self.complete_latency_ms) return value def __repr__(self): @@ -4706,19 +4749,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class Credentials: +class BoltAggregateStats: """ Attributes: - - creds + - execute_latency_ms + - process_latency_ms + - executed + - capacity """ thrift_spec = ( None, # 0 - (1, TType.MAP, 'creds', (TType.STRING,None,TType.STRING,None), None, ), # 1 + (1, TType.DOUBLE, 'execute_latency_ms', None, None, ), # 1 + (2, TType.DOUBLE, 'process_latency_ms', None, None, ), # 2 + (3, TType.I64, 'executed', None, None, ), # 3 + (4, TType.DOUBLE, 'capacity', None, None, ), # 4 ) - def __init__(self, creds=None,): - self.creds = creds + def __init__(self, execute_latency_ms=None, process_latency_ms=None, executed=None, capacity=None,): + self.execute_latency_ms = execute_latency_ms + self.process_latency_ms = process_latency_ms + self.executed = executed + self.capacity = capacity def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4730,14 +4782,23 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.MAP: - self.creds = {} - (_ktype309, _vtype310, _size308 ) = iprot.readMapBegin() - for _i312 in xrange(_size308): - _key313 = iprot.readString().decode('utf-8') - _val314 = iprot.readString().decode('utf-8') - self.creds[_key313] = _val314 - iprot.readMapEnd() + if ftype == TType.DOUBLE: + self.execute_latency_ms = iprot.readDouble(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.DOUBLE: + self.process_latency_ms = iprot.readDouble(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I64: + self.executed = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.DOUBLE: + self.capacity = iprot.readDouble(); else: iprot.skip(ftype) else: @@ -4749,27 +4810,36 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('Credentials') - if self.creds is not None: - oprot.writeFieldBegin('creds', TType.MAP, 1) - oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds)) - for kiter315,viter316 in self.creds.items(): - oprot.writeString(kiter315.encode('utf-8')) - oprot.writeString(viter316.encode('utf-8')) - oprot.writeMapEnd() + oprot.writeStructBegin('BoltAggregateStats') + if self.execute_latency_ms is not None: + oprot.writeFieldBegin('execute_latency_ms', TType.DOUBLE, 1) + oprot.writeDouble(self.execute_latency_ms) + oprot.writeFieldEnd() + if self.process_latency_ms is not None: + oprot.writeFieldBegin('process_latency_ms', TType.DOUBLE, 2) + oprot.writeDouble(self.process_latency_ms) + oprot.writeFieldEnd() + if self.executed is not None: + oprot.writeFieldBegin('executed', TType.I64, 3) + oprot.writeI64(self.executed) + oprot.writeFieldEnd() + if self.capacity is not None: + oprot.writeFieldBegin('capacity', TType.DOUBLE, 4) + oprot.writeDouble(self.capacity) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): - if self.creds is None: - raise TProtocol.TProtocolException(message='Required field creds is unset!') return def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.creds) + value = (value * 31) ^ hash(self.execute_latency_ms) + value = (value * 31) ^ hash(self.process_latency_ms) + value = (value * 31) ^ hash(self.executed) + value = (value * 31) ^ hash(self.capacity) return value def __repr__(self): @@ -4783,22 +4853,22 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class SubmitOptions: +class SpecificAggregateStats: """ Attributes: - - initial_status - - creds + - bolt + - spout """ thrift_spec = ( None, # 0 - (1, TType.I32, 'initial_status', None, None, ), # 1 - (2, TType.STRUCT, 'creds', (Credentials, Credentials.thrift_spec), None, ), # 2 + (1, TType.STRUCT, 'bolt', (BoltAggregateStats, BoltAggregateStats.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'spout', (SpoutAggregateStats, SpoutAggregateStats.thrift_spec), None, ), # 2 ) - def __init__(self, initial_status=None, creds=None,): - self.initial_status = initial_status - self.creds = creds + def __init__(self, bolt=None, spout=None,): + self.bolt = bolt + self.spout = spout def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4810,14 +4880,15 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.I32: - self.initial_status = iprot.readI32(); + if ftype == TType.STRUCT: + self.bolt = BoltAggregateStats() + self.bolt.read(iprot) else: iprot.skip(ftype) elif fid == 2: if ftype == TType.STRUCT: - self.creds = Credentials() - self.creds.read(iprot) + self.spout = SpoutAggregateStats() + self.spout.read(iprot) else: iprot.skip(ftype) else: @@ -4829,28 +4900,26 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('SubmitOptions') - if self.initial_status is not None: - oprot.writeFieldBegin('initial_status', TType.I32, 1) - oprot.writeI32(self.initial_status) + oprot.writeStructBegin('SpecificAggregateStats') + if self.bolt is not None: + oprot.writeFieldBegin('bolt', TType.STRUCT, 1) + self.bolt.write(oprot) oprot.writeFieldEnd() - if self.creds is not None: - oprot.writeFieldBegin('creds', TType.STRUCT, 2) - self.creds.write(oprot) + if self.spout is not None: + oprot.writeFieldBegin('spout', TType.STRUCT, 2) + self.spout.write(oprot) oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() def validate(self): - if self.initial_status is None: - raise TProtocol.TProtocolException(message='Required field initial_status is unset!') return def __hash__(self): value = 17 - value = (value * 31) ^ hash(self.initial_status) - value = (value * 31) ^ hash(self.creds) + value = (value * 31) ^ hash(self.bolt) + value = (value * 31) ^ hash(self.spout) return value def __repr__(self): @@ -4864,40 +4933,28 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) -class SupervisorInfo: +class ComponentAggregateStats: """ Attributes: - - time_secs - - hostname - - assignment_id - - used_ports - - meta - - scheduler_meta - - uptime_secs - - version + - type + - common_stats + - specific_stats + - last_error """ thrift_spec = ( None, # 0 - (1, TType.I64, 'time_secs', None, None, ), # 1 - (2, TType.STRING, 'hostname', None, None, ), # 2 - (3, TType.STRING, 'assignment_id', None, None, ), # 3 - (4, TType.LIST, 'used_ports', (TType.I64,None), None, ), # 4 - (5, TType.LIST, 'meta', (TType.I64,None), None, ), # 5 - (6, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 6 - (7, TType.I64, 'uptime_secs', None, None, ), # 7 - (8, TType.STRING, 'version', None, None, ), # 8 + (1, TType.I32, 'type', None, None, ), # 1 + (2, TType.STRUCT, 'common_stats', (CommonAggregateStats, CommonAggregateStats.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'specific_stats', (SpecificAggregateStats, SpecificAggregateStats.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'last_error', (ErrorInfo, ErrorInfo.thrift_spec), None, ), # 4 ) - def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None, version=None,): - self.time_secs = time_secs - self.hostname = hostname - self.assignment_id = assignment_id - self.used_ports = used_ports - self.meta = meta - self.scheduler_meta = scheduler_meta - self.uptime_secs = uptime_secs - self.version = version + def __init__(self, type=None, common_stats=None, specific_stats=None, last_error=None,): + self.type = type + self.common_stats = common_stats + self.specific_stats = specific_stats + self.last_error = last_error def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -4909,59 +4966,26 @@ def read(self, iprot): if ftype == TType.STOP: break if fid == 1: - if ftype == TType.I64: - self.time_secs = iprot.readI64(); + if ftype == TType.I32: + self.type = iprot.readI32(); else: iprot.skip(ftype) elif fid == 2: - if ftype == TType.STRING: - self.hostname = iprot.readString().decode('utf-8') + if ftype == TType.STRUCT: + self.common_stats = CommonAggregateStats() + self.common_stats.read(iprot) else: iprot.skip(ftype) elif fid == 3: - if ftype == TType.STRING: - self.assignment_id = iprot.readString().decode('utf-8') + if ftype == TType.STRUCT: + self.specific_stats = SpecificAggregateStats() + self.specific_stats.read(iprot) else: iprot.skip(ftype) elif fid == 4: - if ftype == TType.LIST: - self.used_ports = [] - (_etype320, _size317) = iprot.readListBegin() - for _i321 in xrange(_size317): - _elem322 = iprot.readI64(); - self.used_ports.append(_elem322) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.LIST: - self.meta = [] - (_etype326, _size323) = iprot.readListBegin() - for _i327 in xrange(_size323): - _elem328 = iprot.readI64(); - self.meta.append(_elem328) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.MAP: - self.scheduler_meta = {} - (_ktype330, _vtype331, _size329 ) = iprot.readMapBegin() - for _i333 in xrange(_size329): - _key334 = iprot.readString().decode('utf-8') - _val335 = iprot.readString().decode('utf-8') - self.scheduler_meta[_key334] = _val335 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 7: - if ftype == TType.I64: - self.uptime_secs = iprot.readI64(); - else: - iprot.skip(ftype) - elif fid == 8: - if ftype == TType.STRING: - self.version = iprot.readString().decode('utf-8') + if ftype == TType.STRUCT: + self.last_error = ErrorInfo() + self.last_error.read(iprot) else: iprot.skip(ftype) else: @@ -4973,12 +4997,1312 @@ def write(self, oprot): if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) return - oprot.writeStructBegin('SupervisorInfo') - if self.time_secs is not None: - oprot.writeFieldBegin('time_secs', TType.I64, 1) - oprot.writeI64(self.time_secs) + oprot.writeStructBegin('ComponentAggregateStats') + if self.type is not None: + oprot.writeFieldBegin('type', TType.I32, 1) + oprot.writeI32(self.type) oprot.writeFieldEnd() - if self.hostname is not None: + if self.common_stats is not None: + oprot.writeFieldBegin('common_stats', TType.STRUCT, 2) + self.common_stats.write(oprot) + oprot.writeFieldEnd() + if self.specific_stats is not None: + oprot.writeFieldBegin('specific_stats', TType.STRUCT, 3) + self.specific_stats.write(oprot) + oprot.writeFieldEnd() + if self.last_error is not None: + oprot.writeFieldBegin('last_error', TType.STRUCT, 4) + self.last_error.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.type) + value = (value * 31) ^ hash(self.common_stats) + value = (value * 31) ^ hash(self.specific_stats) + value = (value * 31) ^ hash(self.last_error) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TopologyStats: + """ + Attributes: + - window_to_emitted + - window_to_transferred + - window_to_complete_latencies_ms + - window_to_acked + - window_to_failed + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'window_to_emitted', (TType.STRING,None,TType.I64,None), None, ), # 1 + (2, TType.MAP, 'window_to_transferred', (TType.STRING,None,TType.I64,None), None, ), # 2 + (3, TType.MAP, 'window_to_complete_latencies_ms', (TType.STRING,None,TType.DOUBLE,None), None, ), # 3 + (4, TType.MAP, 'window_to_acked', (TType.STRING,None,TType.I64,None), None, ), # 4 + (5, TType.MAP, 'window_to_failed', (TType.STRING,None,TType.I64,None), None, ), # 5 + ) + + def __init__(self, window_to_emitted=None, window_to_transferred=None, window_to_complete_latencies_ms=None, window_to_acked=None, window_to_failed=None,): + self.window_to_emitted = window_to_emitted + self.window_to_transferred = window_to_transferred + self.window_to_complete_latencies_ms = window_to_complete_latencies_ms + self.window_to_acked = window_to_acked + self.window_to_failed = window_to_failed + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.window_to_emitted = {} + (_ktype300, _vtype301, _size299 ) = iprot.readMapBegin() + for _i303 in xrange(_size299): + _key304 = iprot.readString().decode('utf-8') + _val305 = iprot.readI64(); + self.window_to_emitted[_key304] = _val305 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.MAP: + self.window_to_transferred = {} + (_ktype307, _vtype308, _size306 ) = iprot.readMapBegin() + for _i310 in xrange(_size306): + _key311 = iprot.readString().decode('utf-8') + _val312 = iprot.readI64(); + self.window_to_transferred[_key311] = _val312 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.MAP: + self.window_to_complete_latencies_ms = {} + (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin() + for _i317 in xrange(_size313): + _key318 = iprot.readString().decode('utf-8') + _val319 = iprot.readDouble(); + self.window_to_complete_latencies_ms[_key318] = _val319 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.MAP: + self.window_to_acked = {} + (_ktype321, _vtype322, _size320 ) = iprot.readMapBegin() + for _i324 in xrange(_size320): + _key325 = iprot.readString().decode('utf-8') + _val326 = iprot.readI64(); + self.window_to_acked[_key325] = _val326 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.MAP: + self.window_to_failed = {} + (_ktype328, _vtype329, _size327 ) = iprot.readMapBegin() + for _i331 in xrange(_size327): + _key332 = iprot.readString().decode('utf-8') + _val333 = iprot.readI64(); + self.window_to_failed[_key332] = _val333 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TopologyStats') + if self.window_to_emitted is not None: + oprot.writeFieldBegin('window_to_emitted', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_emitted)) + for kiter334,viter335 in self.window_to_emitted.items(): + oprot.writeString(kiter334.encode('utf-8')) + oprot.writeI64(viter335) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.window_to_transferred is not None: + oprot.writeFieldBegin('window_to_transferred', TType.MAP, 2) + oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_transferred)) + for kiter336,viter337 in self.window_to_transferred.items(): + oprot.writeString(kiter336.encode('utf-8')) + oprot.writeI64(viter337) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.window_to_complete_latencies_ms is not None: + oprot.writeFieldBegin('window_to_complete_latencies_ms', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.window_to_complete_latencies_ms)) + for kiter338,viter339 in self.window_to_complete_latencies_ms.items(): + oprot.writeString(kiter338.encode('utf-8')) + oprot.writeDouble(viter339) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.window_to_acked is not None: + oprot.writeFieldBegin('window_to_acked', TType.MAP, 4) + oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_acked)) + for kiter340,viter341 in self.window_to_acked.items(): + oprot.writeString(kiter340.encode('utf-8')) + oprot.writeI64(viter341) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.window_to_failed is not None: + oprot.writeFieldBegin('window_to_failed', TType.MAP, 5) + oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_failed)) + for kiter342,viter343 in self.window_to_failed.items(): + oprot.writeString(kiter342.encode('utf-8')) + oprot.writeI64(viter343) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.window_to_emitted) + value = (value * 31) ^ hash(self.window_to_transferred) + value = (value * 31) ^ hash(self.window_to_complete_latencies_ms) + value = (value * 31) ^ hash(self.window_to_acked) + value = (value * 31) ^ hash(self.window_to_failed) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TopologyPageInfo: + """ + Attributes: + - id + - name + - uptime_secs + - status + - num_tasks + - num_workers + - num_executors + - topology_conf + - id_to_spout_agg_stats + - id_to_bolt_agg_stats + - sched_status + - topology_stats + - owner + - debug_options + - replication_count + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + (2, TType.STRING, 'name', None, None, ), # 2 + (3, TType.I32, 'uptime_secs', None, None, ), # 3 + (4, TType.STRING, 'status', None, None, ), # 4 + (5, TType.I32, 'num_tasks', None, None, ), # 5 + (6, TType.I32, 'num_workers', None, None, ), # 6 + (7, TType.I32, 'num_executors', None, None, ), # 7 + (8, TType.STRING, 'topology_conf', None, None, ), # 8 + (9, TType.MAP, 'id_to_spout_agg_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 9 + (10, TType.MAP, 'id_to_bolt_agg_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 10 + (11, TType.STRING, 'sched_status', None, None, ), # 11 + (12, TType.STRUCT, 'topology_stats', (TopologyStats, TopologyStats.thrift_spec), None, ), # 12 + (13, TType.STRING, 'owner', None, None, ), # 13 + (14, TType.STRUCT, 'debug_options', (DebugOptions, DebugOptions.thrift_spec), None, ), # 14 + (15, TType.I32, 'replication_count', None, None, ), # 15 + ) + + def __init__(self, id=None, name=None, uptime_secs=None, status=None, num_tasks=None, num_workers=None, num_executors=None, topology_conf=None, id_to_spout_agg_stats=None, id_to_bolt_agg_stats=None, sched_status=None, topology_stats=None, owner=None, debug_options=None, replication_count=None,): + self.id = id + self.name = name + self.uptime_secs = uptime_secs + self.status = status + self.num_tasks = num_tasks + self.num_workers = num_workers + self.num_executors = num_executors + self.topology_conf = topology_conf + self.id_to_spout_agg_stats = id_to_spout_agg_stats + self.id_to_bolt_agg_stats = id_to_bolt_agg_stats + self.sched_status = sched_status + self.topology_stats = topology_stats + self.owner = owner + self.debug_options = debug_options + self.replication_count = replication_count + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.uptime_secs = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.status = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.num_tasks = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.I32: + self.num_workers = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.I32: + self.num_executors = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 8: + if ftype == TType.STRING: + self.topology_conf = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 9: + if ftype == TType.MAP: + self.id_to_spout_agg_stats = {} + (_ktype345, _vtype346, _size344 ) = iprot.readMapBegin() + for _i348 in xrange(_size344): + _key349 = iprot.readString().decode('utf-8') + _val350 = ComponentAggregateStats() + _val350.read(iprot) + self.id_to_spout_agg_stats[_key349] = _val350 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 10: + if ftype == TType.MAP: + self.id_to_bolt_agg_stats = {} + (_ktype352, _vtype353, _size351 ) = iprot.readMapBegin() + for _i355 in xrange(_size351): + _key356 = iprot.readString().decode('utf-8') + _val357 = ComponentAggregateStats() + _val357.read(iprot) + self.id_to_bolt_agg_stats[_key356] = _val357 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 11: + if ftype == TType.STRING: + self.sched_status = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 12: + if ftype == TType.STRUCT: + self.topology_stats = TopologyStats() + self.topology_stats.read(iprot) + else: + iprot.skip(ftype) + elif fid == 13: + if ftype == TType.STRING: + self.owner = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 14: + if ftype == TType.STRUCT: + self.debug_options = DebugOptions() + self.debug_options.read(iprot) + else: + iprot.skip(ftype) + elif fid == 15: + if ftype == TType.I32: + self.replication_count = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TopologyPageInfo') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 2) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.uptime_secs is not None: + oprot.writeFieldBegin('uptime_secs', TType.I32, 3) + oprot.writeI32(self.uptime_secs) + oprot.writeFieldEnd() + if self.status is not None: + oprot.writeFieldBegin('status', TType.STRING, 4) + oprot.writeString(self.status.encode('utf-8')) + oprot.writeFieldEnd() + if self.num_tasks is not None: + oprot.writeFieldBegin('num_tasks', TType.I32, 5) + oprot.writeI32(self.num_tasks) + oprot.writeFieldEnd() + if self.num_workers is not None: + oprot.writeFieldBegin('num_workers', TType.I32, 6) + oprot.writeI32(self.num_workers) + oprot.writeFieldEnd() + if self.num_executors is not None: + oprot.writeFieldBegin('num_executors', TType.I32, 7) + oprot.writeI32(self.num_executors) + oprot.writeFieldEnd() + if self.topology_conf is not None: + oprot.writeFieldBegin('topology_conf', TType.STRING, 8) + oprot.writeString(self.topology_conf.encode('utf-8')) + oprot.writeFieldEnd() + if self.id_to_spout_agg_stats is not None: + oprot.writeFieldBegin('id_to_spout_agg_stats', TType.MAP, 9) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_spout_agg_stats)) + for kiter358,viter359 in self.id_to_spout_agg_stats.items(): + oprot.writeString(kiter358.encode('utf-8')) + viter359.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.id_to_bolt_agg_stats is not None: + oprot.writeFieldBegin('id_to_bolt_agg_stats', TType.MAP, 10) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_bolt_agg_stats)) + for kiter360,viter361 in self.id_to_bolt_agg_stats.items(): + oprot.writeString(kiter360.encode('utf-8')) + viter361.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.sched_status is not None: + oprot.writeFieldBegin('sched_status', TType.STRING, 11) + oprot.writeString(self.sched_status.encode('utf-8')) + oprot.writeFieldEnd() + if self.topology_stats is not None: + oprot.writeFieldBegin('topology_stats', TType.STRUCT, 12) + self.topology_stats.write(oprot) + oprot.writeFieldEnd() + if self.owner is not None: + oprot.writeFieldBegin('owner', TType.STRING, 13) + oprot.writeString(self.owner.encode('utf-8')) + oprot.writeFieldEnd() + if self.debug_options is not None: + oprot.writeFieldBegin('debug_options', TType.STRUCT, 14) + self.debug_options.write(oprot) + oprot.writeFieldEnd() + if self.replication_count is not None: + oprot.writeFieldBegin('replication_count', TType.I32, 15) + oprot.writeI32(self.replication_count) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.id is None: + raise TProtocol.TProtocolException(message='Required field id is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.uptime_secs) + value = (value * 31) ^ hash(self.status) + value = (value * 31) ^ hash(self.num_tasks) + value = (value * 31) ^ hash(self.num_workers) + value = (value * 31) ^ hash(self.num_executors) + value = (value * 31) ^ hash(self.topology_conf) + value = (value * 31) ^ hash(self.id_to_spout_agg_stats) + value = (value * 31) ^ hash(self.id_to_bolt_agg_stats) + value = (value * 31) ^ hash(self.sched_status) + value = (value * 31) ^ hash(self.topology_stats) + value = (value * 31) ^ hash(self.owner) + value = (value * 31) ^ hash(self.debug_options) + value = (value * 31) ^ hash(self.replication_count) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ExecutorAggregateStats: + """ + Attributes: + - exec_summary + - stats + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'exec_summary', (ExecutorSummary, ExecutorSummary.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'stats', (ComponentAggregateStats, ComponentAggregateStats.thrift_spec), None, ), # 2 + ) + + def __init__(self, exec_summary=None, stats=None,): + self.exec_summary = exec_summary + self.stats = stats + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.exec_summary = ExecutorSummary() + self.exec_summary.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.stats = ComponentAggregateStats() + self.stats.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ExecutorAggregateStats') + if self.exec_summary is not None: + oprot.writeFieldBegin('exec_summary', TType.STRUCT, 1) + self.exec_summary.write(oprot) + oprot.writeFieldEnd() + if self.stats is not None: + oprot.writeFieldBegin('stats', TType.STRUCT, 2) + self.stats.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.exec_summary) + value = (value * 31) ^ hash(self.stats) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ComponentPageInfo: + """ + Attributes: + - component_id + - component_type + - topology_id + - topology_name + - num_executors + - num_tasks + - window_to_stats + - gsid_to_input_stats + - sid_to_output_stats + - exec_stats + - errors + - eventlog_host + - eventlog_port + - debug_options + - topology_status + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'component_id', None, None, ), # 1 + (2, TType.I32, 'component_type', None, None, ), # 2 + (3, TType.STRING, 'topology_id', None, None, ), # 3 + (4, TType.STRING, 'topology_name', None, None, ), # 4 + (5, TType.I32, 'num_executors', None, None, ), # 5 + (6, TType.I32, 'num_tasks', None, None, ), # 6 + (7, TType.MAP, 'window_to_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 7 + (8, TType.MAP, 'gsid_to_input_stats', (TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 8 + (9, TType.MAP, 'sid_to_output_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 9 + (10, TType.LIST, 'exec_stats', (TType.STRUCT,(ExecutorAggregateStats, ExecutorAggregateStats.thrift_spec)), None, ), # 10 + (11, TType.LIST, 'errors', (TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec)), None, ), # 11 + (12, TType.STRING, 'eventlog_host', None, None, ), # 12 + (13, TType.I32, 'eventlog_port', None, None, ), # 13 + (14, TType.STRUCT, 'debug_options', (DebugOptions, DebugOptions.thrift_spec), None, ), # 14 + (15, TType.STRING, 'topology_status', None, None, ), # 15 + ) + + def __init__(self, component_id=None, component_type=None, topology_id=None, topology_name=None, num_executors=None, num_tasks=None, window_to_stats=None, gsid_to_input_stats=None, sid_to_output_stats=None, exec_stats=None, errors=None, eventlog_host=None, eventlog_port=None, debug_options=None, topology_status=None,): + self.component_id = component_id + self.component_type = component_type + self.topology_id = topology_id + self.topology_name = topology_name + self.num_executors = num_executors + self.num_tasks = num_tasks + self.window_to_stats = window_to_stats + self.gsid_to_input_stats = gsid_to_input_stats + self.sid_to_output_stats = sid_to_output_stats + self.exec_stats = exec_stats + self.errors = errors + self.eventlog_host = eventlog_host + self.eventlog_port = eventlog_port + self.debug_options = debug_options + self.topology_status = topology_status + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.component_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.component_type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.topology_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.topology_name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.num_executors = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.I32: + self.num_tasks = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.MAP: + self.window_to_stats = {} + (_ktype363, _vtype364, _size362 ) = iprot.readMapBegin() + for _i366 in xrange(_size362): + _key367 = iprot.readString().decode('utf-8') + _val368 = ComponentAggregateStats() + _val368.read(iprot) + self.window_to_stats[_key367] = _val368 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 8: + if ftype == TType.MAP: + self.gsid_to_input_stats = {} + (_ktype370, _vtype371, _size369 ) = iprot.readMapBegin() + for _i373 in xrange(_size369): + _key374 = GlobalStreamId() + _key374.read(iprot) + _val375 = ComponentAggregateStats() + _val375.read(iprot) + self.gsid_to_input_stats[_key374] = _val375 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 9: + if ftype == TType.MAP: + self.sid_to_output_stats = {} + (_ktype377, _vtype378, _size376 ) = iprot.readMapBegin() + for _i380 in xrange(_size376): + _key381 = iprot.readString().decode('utf-8') + _val382 = ComponentAggregateStats() + _val382.read(iprot) + self.sid_to_output_stats[_key381] = _val382 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 10: + if ftype == TType.LIST: + self.exec_stats = [] + (_etype386, _size383) = iprot.readListBegin() + for _i387 in xrange(_size383): + _elem388 = ExecutorAggregateStats() + _elem388.read(iprot) + self.exec_stats.append(_elem388) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 11: + if ftype == TType.LIST: + self.errors = [] + (_etype392, _size389) = iprot.readListBegin() + for _i393 in xrange(_size389): + _elem394 = ErrorInfo() + _elem394.read(iprot) + self.errors.append(_elem394) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 12: + if ftype == TType.STRING: + self.eventlog_host = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 13: + if ftype == TType.I32: + self.eventlog_port = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 14: + if ftype == TType.STRUCT: + self.debug_options = DebugOptions() + self.debug_options.read(iprot) + else: + iprot.skip(ftype) + elif fid == 15: + if ftype == TType.STRING: + self.topology_status = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ComponentPageInfo') + if self.component_id is not None: + oprot.writeFieldBegin('component_id', TType.STRING, 1) + oprot.writeString(self.component_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.component_type is not None: + oprot.writeFieldBegin('component_type', TType.I32, 2) + oprot.writeI32(self.component_type) + oprot.writeFieldEnd() + if self.topology_id is not None: + oprot.writeFieldBegin('topology_id', TType.STRING, 3) + oprot.writeString(self.topology_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.topology_name is not None: + oprot.writeFieldBegin('topology_name', TType.STRING, 4) + oprot.writeString(self.topology_name.encode('utf-8')) + oprot.writeFieldEnd() + if self.num_executors is not None: + oprot.writeFieldBegin('num_executors', TType.I32, 5) + oprot.writeI32(self.num_executors) + oprot.writeFieldEnd() + if self.num_tasks is not None: + oprot.writeFieldBegin('num_tasks', TType.I32, 6) + oprot.writeI32(self.num_tasks) + oprot.writeFieldEnd() + if self.window_to_stats is not None: + oprot.writeFieldBegin('window_to_stats', TType.MAP, 7) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.window_to_stats)) + for kiter395,viter396 in self.window_to_stats.items(): + oprot.writeString(kiter395.encode('utf-8')) + viter396.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.gsid_to_input_stats is not None: + oprot.writeFieldBegin('gsid_to_input_stats', TType.MAP, 8) + oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.gsid_to_input_stats)) + for kiter397,viter398 in self.gsid_to_input_stats.items(): + kiter397.write(oprot) + viter398.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.sid_to_output_stats is not None: + oprot.writeFieldBegin('sid_to_output_stats', TType.MAP, 9) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.sid_to_output_stats)) + for kiter399,viter400 in self.sid_to_output_stats.items(): + oprot.writeString(kiter399.encode('utf-8')) + viter400.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.exec_stats is not None: + oprot.writeFieldBegin('exec_stats', TType.LIST, 10) + oprot.writeListBegin(TType.STRUCT, len(self.exec_stats)) + for iter401 in self.exec_stats: + iter401.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.errors is not None: + oprot.writeFieldBegin('errors', TType.LIST, 11) + oprot.writeListBegin(TType.STRUCT, len(self.errors)) + for iter402 in self.errors: + iter402.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.eventlog_host is not None: + oprot.writeFieldBegin('eventlog_host', TType.STRING, 12) + oprot.writeString(self.eventlog_host.encode('utf-8')) + oprot.writeFieldEnd() + if self.eventlog_port is not None: + oprot.writeFieldBegin('eventlog_port', TType.I32, 13) + oprot.writeI32(self.eventlog_port) + oprot.writeFieldEnd() + if self.debug_options is not None: + oprot.writeFieldBegin('debug_options', TType.STRUCT, 14) + self.debug_options.write(oprot) + oprot.writeFieldEnd() + if self.topology_status is not None: + oprot.writeFieldBegin('topology_status', TType.STRING, 15) + oprot.writeString(self.topology_status.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.component_id is None: + raise TProtocol.TProtocolException(message='Required field component_id is unset!') + if self.component_type is None: + raise TProtocol.TProtocolException(message='Required field component_type is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.component_id) + value = (value * 31) ^ hash(self.component_type) + value = (value * 31) ^ hash(self.topology_id) + value = (value * 31) ^ hash(self.topology_name) + value = (value * 31) ^ hash(self.num_executors) + value = (value * 31) ^ hash(self.num_tasks) + value = (value * 31) ^ hash(self.window_to_stats) + value = (value * 31) ^ hash(self.gsid_to_input_stats) + value = (value * 31) ^ hash(self.sid_to_output_stats) + value = (value * 31) ^ hash(self.exec_stats) + value = (value * 31) ^ hash(self.errors) + value = (value * 31) ^ hash(self.eventlog_host) + value = (value * 31) ^ hash(self.eventlog_port) + value = (value * 31) ^ hash(self.debug_options) + value = (value * 31) ^ hash(self.topology_status) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class KillOptions: + """ + Attributes: + - wait_secs + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'wait_secs', None, None, ), # 1 + ) + + def __init__(self, wait_secs=None,): + self.wait_secs = wait_secs + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.wait_secs = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('KillOptions') + if self.wait_secs is not None: + oprot.writeFieldBegin('wait_secs', TType.I32, 1) + oprot.writeI32(self.wait_secs) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.wait_secs) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class RebalanceOptions: + """ + Attributes: + - wait_secs + - num_workers + - num_executors + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'wait_secs', None, None, ), # 1 + (2, TType.I32, 'num_workers', None, None, ), # 2 + (3, TType.MAP, 'num_executors', (TType.STRING,None,TType.I32,None), None, ), # 3 + ) + + def __init__(self, wait_secs=None, num_workers=None, num_executors=None,): + self.wait_secs = wait_secs + self.num_workers = num_workers + self.num_executors = num_executors + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.wait_secs = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.num_workers = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.MAP: + self.num_executors = {} + (_ktype404, _vtype405, _size403 ) = iprot.readMapBegin() + for _i407 in xrange(_size403): + _key408 = iprot.readString().decode('utf-8') + _val409 = iprot.readI32(); + self.num_executors[_key408] = _val409 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('RebalanceOptions') + if self.wait_secs is not None: + oprot.writeFieldBegin('wait_secs', TType.I32, 1) + oprot.writeI32(self.wait_secs) + oprot.writeFieldEnd() + if self.num_workers is not None: + oprot.writeFieldBegin('num_workers', TType.I32, 2) + oprot.writeI32(self.num_workers) + oprot.writeFieldEnd() + if self.num_executors is not None: + oprot.writeFieldBegin('num_executors', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.I32, len(self.num_executors)) + for kiter410,viter411 in self.num_executors.items(): + oprot.writeString(kiter410.encode('utf-8')) + oprot.writeI32(viter411) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.wait_secs) + value = (value * 31) ^ hash(self.num_workers) + value = (value * 31) ^ hash(self.num_executors) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class Credentials: + """ + Attributes: + - creds + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'creds', (TType.STRING,None,TType.STRING,None), None, ), # 1 + ) + + def __init__(self, creds=None,): + self.creds = creds + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.creds = {} + (_ktype413, _vtype414, _size412 ) = iprot.readMapBegin() + for _i416 in xrange(_size412): + _key417 = iprot.readString().decode('utf-8') + _val418 = iprot.readString().decode('utf-8') + self.creds[_key417] = _val418 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('Credentials') + if self.creds is not None: + oprot.writeFieldBegin('creds', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds)) + for kiter419,viter420 in self.creds.items(): + oprot.writeString(kiter419.encode('utf-8')) + oprot.writeString(viter420.encode('utf-8')) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.creds is None: + raise TProtocol.TProtocolException(message='Required field creds is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.creds) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SubmitOptions: + """ + Attributes: + - initial_status + - creds + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'initial_status', None, None, ), # 1 + (2, TType.STRUCT, 'creds', (Credentials, Credentials.thrift_spec), None, ), # 2 + ) + + def __init__(self, initial_status=None, creds=None,): + self.initial_status = initial_status + self.creds = creds + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.initial_status = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.creds = Credentials() + self.creds.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SubmitOptions') + if self.initial_status is not None: + oprot.writeFieldBegin('initial_status', TType.I32, 1) + oprot.writeI32(self.initial_status) + oprot.writeFieldEnd() + if self.creds is not None: + oprot.writeFieldBegin('creds', TType.STRUCT, 2) + self.creds.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.initial_status is None: + raise TProtocol.TProtocolException(message='Required field initial_status is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.initial_status) + value = (value * 31) ^ hash(self.creds) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SupervisorInfo: + """ + Attributes: + - time_secs + - hostname + - assignment_id + - used_ports + - meta + - scheduler_meta + - uptime_secs + - version + """ + + thrift_spec = ( + None, # 0 + (1, TType.I64, 'time_secs', None, None, ), # 1 + (2, TType.STRING, 'hostname', None, None, ), # 2 + (3, TType.STRING, 'assignment_id', None, None, ), # 3 + (4, TType.LIST, 'used_ports', (TType.I64,None), None, ), # 4 + (5, TType.LIST, 'meta', (TType.I64,None), None, ), # 5 + (6, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 6 + (7, TType.I64, 'uptime_secs', None, None, ), # 7 + (8, TType.STRING, 'version', None, None, ), # 8 + ) + + def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None, version=None,): + self.time_secs = time_secs + self.hostname = hostname + self.assignment_id = assignment_id + self.used_ports = used_ports + self.meta = meta + self.scheduler_meta = scheduler_meta + self.uptime_secs = uptime_secs + self.version = version + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I64: + self.time_secs = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.hostname = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.assignment_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.LIST: + self.used_ports = [] + (_etype424, _size421) = iprot.readListBegin() + for _i425 in xrange(_size421): + _elem426 = iprot.readI64(); + self.used_ports.append(_elem426) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.LIST: + self.meta = [] + (_etype430, _size427) = iprot.readListBegin() + for _i431 in xrange(_size427): + _elem432 = iprot.readI64(); + self.meta.append(_elem432) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.MAP: + self.scheduler_meta = {} + (_ktype434, _vtype435, _size433 ) = iprot.readMapBegin() + for _i437 in xrange(_size433): + _key438 = iprot.readString().decode('utf-8') + _val439 = iprot.readString().decode('utf-8') + self.scheduler_meta[_key438] = _val439 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.I64: + self.uptime_secs = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 8: + if ftype == TType.STRING: + self.version = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SupervisorInfo') + if self.time_secs is not None: + oprot.writeFieldBegin('time_secs', TType.I64, 1) + oprot.writeI64(self.time_secs) + oprot.writeFieldEnd() + if self.hostname is not None: oprot.writeFieldBegin('hostname', TType.STRING, 2) oprot.writeString(self.hostname.encode('utf-8')) oprot.writeFieldEnd() @@ -4989,23 +6313,23 @@ def write(self, oprot): if self.used_ports is not None: oprot.writeFieldBegin('used_ports', TType.LIST, 4) oprot.writeListBegin(TType.I64, len(self.used_ports)) - for iter336 in self.used_ports: - oprot.writeI64(iter336) + for iter440 in self.used_ports: + oprot.writeI64(iter440) oprot.writeListEnd() oprot.writeFieldEnd() if self.meta is not None: oprot.writeFieldBegin('meta', TType.LIST, 5) oprot.writeListBegin(TType.I64, len(self.meta)) - for iter337 in self.meta: - oprot.writeI64(iter337) + for iter441 in self.meta: + oprot.writeI64(iter441) oprot.writeListEnd() oprot.writeFieldEnd() if self.scheduler_meta is not None: oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta)) - for kiter338,viter339 in self.scheduler_meta.items(): - oprot.writeString(kiter338.encode('utf-8')) - oprot.writeString(viter339.encode('utf-8')) + for kiter442,viter443 in self.scheduler_meta.items(): + oprot.writeString(kiter442.encode('utf-8')) + oprot.writeString(viter443.encode('utf-8')) oprot.writeMapEnd() oprot.writeFieldEnd() if self.uptime_secs is not None: @@ -5084,10 +6408,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.SET: self.port = set() - (_etype343, _size340) = iprot.readSetBegin() - for _i344 in xrange(_size340): - _elem345 = iprot.readI64(); - self.port.add(_elem345) + (_etype447, _size444) = iprot.readSetBegin() + for _i448 in xrange(_size444): + _elem449 = iprot.readI64(); + self.port.add(_elem449) iprot.readSetEnd() else: iprot.skip(ftype) @@ -5108,8 +6432,8 @@ def write(self, oprot): if self.port is not None: oprot.writeFieldBegin('port', TType.SET, 2) oprot.writeSetBegin(TType.I64, len(self.port)) - for iter346 in self.port: - oprot.writeI64(iter346) + for iter450 in self.port: + oprot.writeI64(iter450) oprot.writeSetEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -5192,44 +6516,44 @@ def read(self, iprot): elif fid == 2: if ftype == TType.MAP: self.node_host = {} - (_ktype348, _vtype349, _size347 ) = iprot.readMapBegin() - for _i351 in xrange(_size347): - _key352 = iprot.readString().decode('utf-8') - _val353 = iprot.readString().decode('utf-8') - self.node_host[_key352] = _val353 + (_ktype452, _vtype453, _size451 ) = iprot.readMapBegin() + for _i455 in xrange(_size451): + _key456 = iprot.readString().decode('utf-8') + _val457 = iprot.readString().decode('utf-8') + self.node_host[_key456] = _val457 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 3: if ftype == TType.MAP: self.executor_node_port = {} - (_ktype355, _vtype356, _size354 ) = iprot.readMapBegin() - for _i358 in xrange(_size354): - _key359 = [] - (_etype364, _size361) = iprot.readListBegin() - for _i365 in xrange(_size361): - _elem366 = iprot.readI64(); - _key359.append(_elem366) + (_ktype459, _vtype460, _size458 ) = iprot.readMapBegin() + for _i462 in xrange(_size458): + _key463 = [] + (_etype468, _size465) = iprot.readListBegin() + for _i469 in xrange(_size465): + _elem470 = iprot.readI64(); + _key463.append(_elem470) iprot.readListEnd() - _val360 = NodeInfo() - _val360.read(iprot) - self.executor_node_port[_key359] = _val360 + _val464 = NodeInfo() + _val464.read(iprot) + self.executor_node_port[_key463] = _val464 iprot.readMapEnd() else: iprot.skip(ftype) elif fid == 4: if ftype == TType.MAP: self.executor_start_time_secs = {} - (_ktype368, _vtype369, _size367 ) = iprot.readMapBegin() - for _i371 in xrange(_size367): - _key372 = [] - (_etype377, _size374) = iprot.readListBegin() - for _i378 in xrange(_size374): - _elem379 = iprot.readI64(); - _key372.append(_elem379) + (_ktype472, _vtype473, _size471 ) = iprot.readMapBegin() + for _i475 in xrange(_size471): + _key476 = [] + (_etype481, _size478) = iprot.readListBegin() + for _i482 in xrange(_size478): + _elem483 = iprot.readI64(); + _key476.append(_elem483) iprot.readListEnd() - _val373 = iprot.readI64(); - self.executor_start_time_secs[_key372] = _val373 + _val477 = iprot.readI64(); + self.executor_start_time_secs[_key476] = _val477 iprot.readMapEnd() else: iprot.skip(ftype) @@ -5250,31 +6574,31 @@ def write(self, oprot): if self.node_host is not None: oprot.writeFieldBegin('node_host', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.node_host)) - for kiter380,viter381 in self.node_host.items(): - oprot.writeString(kiter380.encode('utf-8')) - oprot.writeString(viter381.encode('utf-8')) + for kiter484,viter485 in self.node_host.items(): + oprot.writeString(kiter484.encode('utf-8')) + oprot.writeString(viter485.encode('utf-8')) oprot.writeMapEnd() oprot.writeFieldEnd() if self.executor_node_port is not None: oprot.writeFieldBegin('executor_node_port', TType.MAP, 3) oprot.writeMapBegin(TType.LIST, TType.STRUCT, len(self.executor_node_port)) - for kiter382,viter383 in self.executor_node_port.items(): - oprot.writeListBegin(TType.I64, len(kiter382)) - for iter384 in kiter382: - oprot.writeI64(iter384) + for kiter486,viter487 in self.executor_node_port.items(): + oprot.writeListBegin(TType.I64, len(kiter486)) + for iter488 in kiter486: + oprot.writeI64(iter488) oprot.writeListEnd() - viter383.write(oprot) + viter487.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.executor_start_time_secs is not None: oprot.writeFieldBegin('executor_start_time_secs', TType.MAP, 4) oprot.writeMapBegin(TType.LIST, TType.I64, len(self.executor_start_time_secs)) - for kiter385,viter386 in self.executor_start_time_secs.items(): - oprot.writeListBegin(TType.I64, len(kiter385)) - for iter387 in kiter385: - oprot.writeI64(iter387) + for kiter489,viter490 in self.executor_start_time_secs.items(): + oprot.writeListBegin(TType.I64, len(kiter489)) + for iter491 in kiter489: + oprot.writeI64(iter491) oprot.writeListEnd() - oprot.writeI64(viter386) + oprot.writeI64(viter490) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -5450,11 +6774,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.MAP: self.component_executors = {} - (_ktype389, _vtype390, _size388 ) = iprot.readMapBegin() - for _i392 in xrange(_size388): - _key393 = iprot.readString().decode('utf-8') - _val394 = iprot.readI32(); - self.component_executors[_key393] = _val394 + (_ktype493, _vtype494, _size492 ) = iprot.readMapBegin() + for _i496 in xrange(_size492): + _key497 = iprot.readString().decode('utf-8') + _val498 = iprot.readI32(); + self.component_executors[_key497] = _val498 iprot.readMapEnd() else: iprot.skip(ftype) @@ -5482,12 +6806,12 @@ def read(self, iprot): elif fid == 9: if ftype == TType.MAP: self.component_debug = {} - (_ktype396, _vtype397, _size395 ) = iprot.readMapBegin() - for _i399 in xrange(_size395): - _key400 = iprot.readString().decode('utf-8') - _val401 = DebugOptions() - _val401.read(iprot) - self.component_debug[_key400] = _val401 + (_ktype500, _vtype501, _size499 ) = iprot.readMapBegin() + for _i503 in xrange(_size499): + _key504 = iprot.readString().decode('utf-8') + _val505 = DebugOptions() + _val505.read(iprot) + self.component_debug[_key504] = _val505 iprot.readMapEnd() else: iprot.skip(ftype) @@ -5516,9 +6840,9 @@ def write(self, oprot): if self.component_executors is not None: oprot.writeFieldBegin('component_executors', TType.MAP, 4) oprot.writeMapBegin(TType.STRING, TType.I32, len(self.component_executors)) - for kiter402,viter403 in self.component_executors.items(): - oprot.writeString(kiter402.encode('utf-8')) - oprot.writeI32(viter403) + for kiter506,viter507 in self.component_executors.items(): + oprot.writeString(kiter506.encode('utf-8')) + oprot.writeI32(viter507) oprot.writeMapEnd() oprot.writeFieldEnd() if self.launch_time_secs is not None: @@ -5540,9 +6864,9 @@ def write(self, oprot): if self.component_debug is not None: oprot.writeFieldBegin('component_debug', TType.MAP, 9) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug)) - for kiter404,viter405 in self.component_debug.items(): - oprot.writeString(kiter404.encode('utf-8')) - viter405.write(oprot) + for kiter508,viter509 in self.component_debug.items(): + oprot.writeString(kiter508.encode('utf-8')) + viter509.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -5622,13 +6946,13 @@ def read(self, iprot): elif fid == 2: if ftype == TType.MAP: self.executor_stats = {} - (_ktype407, _vtype408, _size406 ) = iprot.readMapBegin() - for _i410 in xrange(_size406): - _key411 = ExecutorInfo() - _key411.read(iprot) - _val412 = ExecutorStats() - _val412.read(iprot) - self.executor_stats[_key411] = _val412 + (_ktype511, _vtype512, _size510 ) = iprot.readMapBegin() + for _i514 in xrange(_size510): + _key515 = ExecutorInfo() + _key515.read(iprot) + _val516 = ExecutorStats() + _val516.read(iprot) + self.executor_stats[_key515] = _val516 iprot.readMapEnd() else: iprot.skip(ftype) @@ -5659,9 +6983,9 @@ def write(self, oprot): if self.executor_stats is not None: oprot.writeFieldBegin('executor_stats', TType.MAP, 2) oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.executor_stats)) - for kiter413,viter414 in self.executor_stats.items(): - kiter413.write(oprot) - viter414.write(oprot) + for kiter517,viter518 in self.executor_stats.items(): + kiter517.write(oprot) + viter518.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() if self.time_secs is not None: @@ -5814,12 +7138,12 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.serialized_parts = {} - (_ktype416, _vtype417, _size415 ) = iprot.readMapBegin() - for _i419 in xrange(_size415): - _key420 = iprot.readString().decode('utf-8') - _val421 = ThriftSerializedObject() - _val421.read(iprot) - self.serialized_parts[_key420] = _val421 + (_ktype520, _vtype521, _size519 ) = iprot.readMapBegin() + for _i523 in xrange(_size519): + _key524 = iprot.readString().decode('utf-8') + _val525 = ThriftSerializedObject() + _val525.read(iprot) + self.serialized_parts[_key524] = _val525 iprot.readMapEnd() else: iprot.skip(ftype) @@ -5836,9 +7160,9 @@ def write(self, oprot): if self.serialized_parts is not None: oprot.writeFieldBegin('serialized_parts', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.serialized_parts)) - for kiter422,viter423 in self.serialized_parts.items(): - oprot.writeString(kiter422.encode('utf-8')) - viter423.write(oprot) + for kiter526,viter527 in self.serialized_parts.items(): + oprot.writeString(kiter526.encode('utf-8')) + viter527.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -5900,11 +7224,11 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.executors = [] - (_etype427, _size424) = iprot.readListBegin() - for _i428 in xrange(_size424): - _elem429 = ExecutorInfo() - _elem429.read(iprot) - self.executors.append(_elem429) + (_etype531, _size528) = iprot.readListBegin() + for _i532 in xrange(_size528): + _elem533 = ExecutorInfo() + _elem533.read(iprot) + self.executors.append(_elem533) iprot.readListEnd() else: iprot.skip(ftype) @@ -5925,8 +7249,8 @@ def write(self, oprot): if self.executors is not None: oprot.writeFieldBegin('executors', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.executors)) - for iter430 in self.executors: - iter430.write(oprot) + for iter534 in self.executors: + iter534.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -6050,11 +7374,11 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.approved_workers = {} - (_ktype432, _vtype433, _size431 ) = iprot.readMapBegin() - for _i435 in xrange(_size431): - _key436 = iprot.readString().decode('utf-8') - _val437 = iprot.readI32(); - self.approved_workers[_key436] = _val437 + (_ktype536, _vtype537, _size535 ) = iprot.readMapBegin() + for _i539 in xrange(_size535): + _key540 = iprot.readString().decode('utf-8') + _val541 = iprot.readI32(); + self.approved_workers[_key540] = _val541 iprot.readMapEnd() else: iprot.skip(ftype) @@ -6071,9 +7395,9 @@ def write(self, oprot): if self.approved_workers is not None: oprot.writeFieldBegin('approved_workers', TType.MAP, 1) oprot.writeMapBegin(TType.STRING, TType.I32, len(self.approved_workers)) - for kiter438,viter439 in self.approved_workers.items(): - oprot.writeString(kiter438.encode('utf-8')) - oprot.writeI32(viter439) + for kiter542,viter543 in self.approved_workers.items(): + oprot.writeString(kiter542.encode('utf-8')) + oprot.writeI32(viter543) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -6127,12 +7451,12 @@ def read(self, iprot): if fid == 1: if ftype == TType.MAP: self.assignments = {} - (_ktype441, _vtype442, _size440 ) = iprot.readMapBegin() - for _i444 in xrange(_size440): - _key445 = iprot.readI32(); - _val446 = LocalAssignment() - _val446.read(iprot) - self.assignments[_key445] = _val446 + (_ktype545, _vtype546, _size544 ) = iprot.readMapBegin() + for _i548 in xrange(_size544): + _key549 = iprot.readI32(); + _val550 = LocalAssignment() + _val550.read(iprot) + self.assignments[_key549] = _val550 iprot.readMapEnd() else: iprot.skip(ftype) @@ -6149,9 +7473,9 @@ def write(self, oprot): if self.assignments is not None: oprot.writeFieldBegin('assignments', TType.MAP, 1) oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments)) - for kiter447,viter448 in self.assignments.items(): - oprot.writeI32(kiter447) - viter448.write(oprot) + for kiter551,viter552 in self.assignments.items(): + oprot.writeI32(kiter551) + viter552.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -6224,11 +7548,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.LIST: self.executors = [] - (_etype452, _size449) = iprot.readListBegin() - for _i453 in xrange(_size449): - _elem454 = ExecutorInfo() - _elem454.read(iprot) - self.executors.append(_elem454) + (_etype556, _size553) = iprot.readListBegin() + for _i557 in xrange(_size553): + _elem558 = ExecutorInfo() + _elem558.read(iprot) + self.executors.append(_elem558) iprot.readListEnd() else: iprot.skip(ftype) @@ -6258,8 +7582,8 @@ def write(self, oprot): if self.executors is not None: oprot.writeFieldBegin('executors', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.executors)) - for iter455 in self.executors: - iter455.write(oprot) + for iter559 in self.executors: + iter559.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.port is not None: @@ -6511,12 +7835,12 @@ def read(self, iprot): if fid == 2: if ftype == TType.MAP: self.named_logger_level = {} - (_ktype457, _vtype458, _size456 ) = iprot.readMapBegin() - for _i460 in xrange(_size456): - _key461 = iprot.readString().decode('utf-8') - _val462 = LogLevel() - _val462.read(iprot) - self.named_logger_level[_key461] = _val462 + (_ktype561, _vtype562, _size560 ) = iprot.readMapBegin() + for _i564 in xrange(_size560): + _key565 = iprot.readString().decode('utf-8') + _val566 = LogLevel() + _val566.read(iprot) + self.named_logger_level[_key565] = _val566 iprot.readMapEnd() else: iprot.skip(ftype) @@ -6533,9 +7857,9 @@ def write(self, oprot): if self.named_logger_level is not None: oprot.writeFieldBegin('named_logger_level', TType.MAP, 2) oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.named_logger_level)) - for kiter463,viter464 in self.named_logger_level.items(): - oprot.writeString(kiter463.encode('utf-8')) - viter464.write(oprot) + for kiter567,viter568 in self.named_logger_level.items(): + oprot.writeString(kiter567.encode('utf-8')) + viter568.write(oprot) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift index eba9fcda505..296c3f09b03 100644 --- a/storm-core/src/storm.thrift +++ b/storm-core/src/storm.thrift @@ -238,6 +238,92 @@ struct DebugOptions { 2: optional double samplingpct } +struct CommonAggregateStats { +1: optional i32 num_executors; +2: optional i32 num_tasks; +3: optional i64 emitted; +4: optional i64 transferred; +5: optional i64 acked; +6: optional i64 failed; +} + +struct SpoutAggregateStats { +1: optional double complete_latency_ms; +} + +struct BoltAggregateStats { +1: optional double execute_latency_ms; +2: optional double process_latency_ms; +3: optional i64 executed; +4: optional double capacity; +} + +union SpecificAggregateStats { +1: BoltAggregateStats bolt; +2: SpoutAggregateStats spout; +} + +enum ComponentType { + BOLT = 1, + SPOUT = 2 +} + +struct ComponentAggregateStats { +1: optional ComponentType type; +2: optional CommonAggregateStats common_stats; +3: optional SpecificAggregateStats specific_stats; +4: optional ErrorInfo last_error; +} + +struct TopologyStats { +1: optional map window_to_emitted; +2: optional map window_to_transferred; +3: optional map window_to_complete_latencies_ms; +4: optional map window_to_acked; +5: optional map window_to_failed; +} + +struct TopologyPageInfo { + 1: required string id; + 2: optional string name; + 3: optional i32 uptime_secs; + 4: optional string status; + 5: optional i32 num_tasks; + 6: optional i32 num_workers; + 7: optional i32 num_executors; + 8: optional string topology_conf; + 9: optional map id_to_spout_agg_stats; +10: optional map id_to_bolt_agg_stats; +11: optional string sched_status; +12: optional TopologyStats topology_stats; +13: optional string owner; +14: optional DebugOptions debug_options; +15: optional i32 replication_count; +} + +struct ExecutorAggregateStats { +1: optional ExecutorSummary exec_summary; +2: optional ComponentAggregateStats stats; +} + +struct ComponentPageInfo { + 1: required string component_id; + 2: required ComponentType component_type; + 3: optional string topology_id; + 4: optional string topology_name; + 5: optional i32 num_executors; + 6: optional i32 num_tasks; + 7: optional map window_to_stats; + 8: optional map gsid_to_input_stats; + 9: optional map sid_to_output_stats; +10: optional list exec_stats; +11: optional list errors; +12: optional string eventlog_host; +13: optional i32 eventlog_port; +14: optional DebugOptions debug_options; +15: optional string topology_status; +} + struct KillOptions { 1: optional i32 wait_secs; } @@ -428,6 +514,8 @@ service Nimbus { ClusterSummary getClusterInfo() throws (1: AuthorizationException aze); TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); TopologyInfo getTopologyInfoWithOpts(1: string id, 2: GetInfoOptions options) throws (1: NotAliveException e, 2: AuthorizationException aze); + TopologyPageInfo getTopologyPageInfo(1: string id, 2: string window, 3: bool is_include_sys) throws (1: NotAliveException e, 2: AuthorizationException aze); + ComponentPageInfo getComponentPageInfo(1: string topology_id, 2: string component_id, 3: string window, 4: bool is_include_sys) throws (1: NotAliveException e, 2: AuthorizationException aze); //returns json string getTopologyConf(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze); /** diff --git a/storm-core/src/ui/public/js/visualization.js b/storm-core/src/ui/public/js/visualization.js index 1e1c4d65983..9aab0875f59 100644 --- a/storm-core/src/ui/public/js/visualization.js +++ b/storm-core/src/ui/public/js/visualization.js @@ -375,45 +375,67 @@ function update_data(jdat, sys) { }); } +function jsError(other) { + try { + other(); + } catch (err) { + $.get("/templates/json-error-template.html", function(template) { + $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),{error: "JS Error", errorMessage: err})); + }); + } +} + var should_update; function show_visualization(sys) { + $.getJSON("/api/v1/topology/"+$.url("?id")+"/visualization-init",function(response,status,jqXHR) { + $.get("/templates/topology-page-template.html", function(template) { + jsError(function() { + var topologyVisualization = $("#visualization-container"); + topologyVisualization.append( + Mustache.render($(template) + .filter("#topology-visualization-container-template") + .html(), + response)); + }); + + if(sys == null) + { + sys = arbor.ParticleSystem(20, 1000, 0.15, true, 55, 0.02, 0.6); + sys.renderer = renderGraph("#topoGraph"); + sys.stop(); - if(sys == null) - { - sys = arbor.ParticleSystem(20, 1000, 0.15, true, 55, 0.02, 0.6); - sys.renderer = renderGraph("#topoGraph"); - sys.stop(); - - $(".stream-box").click(function () { rechoose(topology_data, sys, this) }); - } - - should_update = true; - var update_freq_ms = 10000; - var update = function(should_rechoose) { - if(should_update) { - $.ajax({ - url: "/api/v1/topology/"+$.url("?id")+"/visualization", - success: function (data, status, jqXHR) { - topology_data = data; - update_data(topology_data, sys); - sys.renderer.signal_update(); - sys.renderer.redraw(); - if (should_update) - setTimeout(update, update_freq_ms); - if (should_rechoose) - $(".stream-box").each(function () { - rechoose(topology_data, sys, this) - }); + $(".stream-box").click(function () { rechoose(topology_data, sys, this) }); } - }); - } - }; - - update(true); - $("#visualization-container").show(500); - $("#show-hide-visualization").attr('value', 'Hide Visualization'); - $("#show-hide-visualization").unbind("click"); - $("#show-hide-visualization").click(function () { hide_visualization(sys) }); + + should_update = true; + var update_freq_ms = 10000; + var update = function(should_rechoose){ + if(should_update) { + $.ajax({ + url: "/api/v1/topology/"+$.url("?id")+"/visualization", + success: function(data, status, jqXHR) { + topology_data = data; + update_data(topology_data, sys); + sys.renderer.signal_update(); + sys.renderer.redraw(); + if(should_update) + setTimeout(update, update_freq_ms); + if(should_rechoose) + $(".stream-box").each(function () { + rechoose(topology_data, sys, this) + }); + } + }); + } + }; + + update(true); + $("#visualization-container").show(500); + $("#show-hide-visualization").attr('value', 'Hide Visualization'); + $("#show-hide-visualization").unbind("click"); + $("#show-hide-visualization").click(function () { hide_visualization(sys) }); + }) + }); } function hide_visualization(sys) { diff --git a/storm-core/src/ui/public/templates/component-page-template.html b/storm-core/src/ui/public/templates/component-page-template.html index ccb4d5a2938..7a4c8948fa5 100644 --- a/storm-core/src/ui/public/templates/component-page-template.html +++ b/storm-core/src/ui/public/templates/component-page-template.html @@ -481,7 +481,7 @@

Errors

{{#componentErrors}} - {{time}} + {{errorTime}} {{errorHost}} {{errorPort}} diff --git a/storm-core/src/ui/public/templates/topology-page-template.html b/storm-core/src/ui/public/templates/topology-page-template.html index f0cc82299cb..817a7ab294f 100644 --- a/storm-core/src/ui/public/templates/topology-page-template.html +++ b/storm-core/src/ui/public/templates/topology-page-template.html @@ -137,33 +137,37 @@

Topology stats

+ + + @@ -253,7 +257,7 @@

Spouts ({{windowHint}})

{{lastError}} - {{time}} + {{errorTime}} {{/spouts}} @@ -347,7 +351,7 @@

Bolts ({{windowHint}})

{{lastError}} - {{time}} + {{errorTime}} {{/bolts}} @@ -431,4 +435,4 @@

Change Log Level

{{/loggers}} - + \ No newline at end of file