You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by sr...@apache.org on 2015/09/21 19:40:46 UTC

[01/24] storm git commit: Event (tuple) logging and viewing support.

Repository: storm
Updated Branches:
  refs/heads/master 3ec512055 -> 206e7283a


Event (tuple) logging and viewing support.

1. Added EventLogger internal bolt
2. Forward tuples + metadata from spouts/bolts to the EventLogger.
3. Added UI link in component summary to view the logged events.


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

Branch: refs/heads/master
Commit: a924e9efb461cd86cee7ff59d867c8cf2118f1e0
Parents: f72beb0
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Mon Aug 3 12:34:30 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 10 14:20:47 2015 +0530

----------------------------------------------------------------------
 conf/defaults.yaml                              |   1 +
 .../src/clj/backtype/storm/daemon/common.clj    |  42 +++++++-
 .../src/clj/backtype/storm/daemon/executor.clj  |  19 +++-
 .../src/clj/backtype/storm/daemon/logviewer.clj |   1 +
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   1 +
 storm-core/src/clj/backtype/storm/ui/core.clj   |  60 ++++++++---
 storm-core/src/clj/backtype/storm/util.clj      |  12 ++-
 storm-core/src/jvm/backtype/storm/Config.java   |  18 ++++
 .../backtype/storm/metric/EventLoggerBolt.java  |  56 ++++++++++
 .../storm/metric/FileBasedEventLogger.java      | 105 +++++++++++++++++++
 .../jvm/backtype/storm/metric/IEventLogger.java |  59 +++++++++++
 .../templates/component-page-template.html      |   6 ++
 12 files changed, 361 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index c3fa372..d357da7 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -169,6 +169,7 @@ topology.enable.message.timeouts: true
 topology.debug: false
 topology.workers: 1
 topology.acker.executors: null
+topology.eventlogger.executors: null
 topology.tasks: null
 # maximum amount of time a message has to complete before it's considered failed
 topology.message.timeout.secs: 30

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/clj/backtype/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index e3a10ef..7868eb1 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -21,6 +21,7 @@
   (:import [backtype.storm.task WorkerTopologyContext])
   (:import [backtype.storm Constants])
   (:import [backtype.storm.metric SystemBolt])
+  (:import [backtype.storm.metric EventLoggerBolt])
   (:import [backtype.storm.security.auth IAuthorizer]) 
   (:import [java.io InterruptedIOException])
   (:require [clojure.set :as set])  
@@ -38,6 +39,9 @@
 
 (def SYSTEM-STREAM-ID "__system")
 
+(def EVENTLOGGER-COMPONENT-ID "__eventlogger")
+(def EVENTLOGGER-STREAM-ID "__eventlog")
+
 (def SYSTEM-COMPONENT-ID Constants/SYSTEM_COMPONENT_ID)
 (def SYSTEM-TICK-STREAM-ID Constants/SYSTEM_TICK_STREAM_ID)
 (def METRICS-STREAM-ID Constants/METRICS_STREAM_ID)
@@ -182,6 +186,19 @@
                              ))]
     (merge spout-inputs bolt-inputs)))
 
+(defn eventlogger-inputs [^StormTopology topology]
+  (let [bolt-ids (.. topology get_bolts keySet)
+        spout-ids (.. topology get_spouts keySet)
+        spout-inputs (apply merge
+                       (for [id spout-ids]
+                         {[id EVENTLOGGER-STREAM-ID] ["component-id"]} ;grouping on component id
+                         ))
+        bolt-inputs (apply merge
+                      (for [id bolt-ids]
+                        {[id EVENTLOGGER-STREAM-ID] ["component-id"]}
+                        ))]
+    (merge spout-inputs bolt-inputs)))
+
 (defn add-acker! [storm-conf ^StormTopology ret]
   (let [num-executors (if (nil? (storm-conf TOPOLOGY-ACKER-EXECUTORS)) (storm-conf TOPOLOGY-WORKERS) (storm-conf TOPOLOGY-ACKER-EXECUTORS))
         acker-bolt (thrift/mk-bolt-spec* (acker-inputs ret)
@@ -273,6 +290,26 @@
      (metrics-consumer-register-ids storm-conf)
      (get storm-conf TOPOLOGY-METRICS-CONSUMER-REGISTER))))
 
+; return the fields that event logger bolt expects
+(defn eventlogger-bolt-fields []
+  [(EventLoggerBolt/FIELD_COMPONENT_ID)  (EventLoggerBolt/FIELD_TS) (EventLoggerBolt/FIELD_VALUES)]
+  )
+
+(defn add-eventlogger! [storm-conf ^StormTopology ret]
+  (let [num-executors (if (nil? (storm-conf TOPOLOGY-EVENTLOGGER-EXECUTORS)) (storm-conf TOPOLOGY-WORKERS) (storm-conf TOPOLOGY-EVENTLOGGER-EXECUTORS))
+        eventlogger-bolt (thrift/mk-bolt-spec* (eventlogger-inputs ret)
+                     (EventLoggerBolt.)
+                     {}
+                     :p num-executors
+                     :conf {TOPOLOGY-TASKS num-executors
+                            TOPOLOGY-TICK-TUPLE-FREQ-SECS (storm-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)})]
+
+    (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)
+    ))
+
 (defn add-metric-components! [storm-conf ^StormTopology topology]  
   (doseq [[comp-id bolt-spec] (metrics-consumer-bolt-specs storm-conf topology)]
     (.put_to_bolts topology comp-id bolt-spec)))
@@ -292,7 +329,8 @@
   (validate-basic! topology)
   (let [ret (.deepCopy topology)]
     (add-acker! storm-conf ret)
-    (add-metric-components! storm-conf ret)    
+    (add-eventlogger! storm-conf ret)
+    (add-metric-components! storm-conf ret)
     (add-system-components! storm-conf ret)
     (add-metric-streams! ret)
     (add-system-streams! ret)
@@ -303,6 +341,8 @@
 (defn has-ackers? [storm-conf]
   (or (nil? (storm-conf TOPOLOGY-ACKER-EXECUTORS)) (> (storm-conf TOPOLOGY-ACKER-EXECUTORS) 0)))
 
+(defn has-eventloggers? [storm-conf]
+  (or (nil? (storm-conf TOPOLOGY-EVENTLOGGER-EXECUTORS)) (> (storm-conf TOPOLOGY-EVENTLOGGER-EXECUTORS) 0)))
 
 (defn num-start-executors [component]
   (thrift/parallelism-hint (.get_common component)))

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index e9c7a2e..757ee48 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -488,6 +488,7 @@
         receive-queue (:receive-queue executor-data)
         event-handler (mk-task-receiver executor-data tuple-action-fn)
         has-ackers? (has-ackers? storm-conf)
+        has-eventloggers? (has-eventloggers? storm-conf)
         emitted-count (MutableLong. 0)
         empty-emit-streak (MutableLong. 0)
         
@@ -531,6 +532,13 @@
                                                                         out-tuple
                                                                         overflow-buffer)
                                                            ))
+                                         ; Send data to the eventlogger.
+                                         (if has-eventloggers?
+                                           (task/send-unanchored
+                                             task-data
+                                             EVENTLOGGER-STREAM-ID
+                                             [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
+                                             overflow-buffer))
                                          (if (and rooted?
                                                   (not (.isEmpty out-ids)))
                                            (do
@@ -700,7 +708,8 @@
         ;; buffers filled up)
         ;; the overflow buffer is might gradually fill degrading the performance gradually
         ;; eventually running out of memory, but at least prevent live-locks/deadlocks.
-        overflow-buffer (if (storm-conf TOPOLOGY-BOLTS-OUTGOING-OVERFLOW-BUFFER-ENABLE) (ConcurrentLinkedQueue.) nil)]
+        overflow-buffer (if (storm-conf TOPOLOGY-BOLTS-OUTGOING-OVERFLOW-BUFFER-ENABLE) (ConcurrentLinkedQueue.) nil)
+        has-eventloggers? (has-eventloggers? storm-conf)]
     
     ;; TODO: can get any SubscribedState objects out of the context now
 
@@ -729,13 +738,19 @@
                                                                             (fast-list-iter [root-id root-ids]
                                                                                             (put-xor! anchors-to-ids root-id edge-id))
                                                                             ))))
-                                                      (transfer-fn t
+                                                        (transfer-fn t
                                                                    (TupleImpl. worker-context
                                                                                values
                                                                                task-id
                                                                                stream
                                                                                (MessageId/makeId anchors-to-ids))
                                                                    overflow-buffer)))
+                                    ; send the data to the eventlogger
+                                    (if has-eventloggers?
+                                      (task/send-unanchored task-data
+                                        EVENTLOGGER-STREAM-ID
+                                        [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
+                                        overflow-buffer))
                                     (or out-tasks [])))]]
           (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
           (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials)) 

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
index a981d94..2a176ef 100644
--- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
@@ -46,6 +46,7 @@
 (defn cleanup-cutoff-age-millis [conf now-millis]
   (- now-millis (* (conf LOGVIEWER-CLEANUP-AGE-MINS) 60 1000)))
 
+;TODO: handle cleanup of old event log files
 (defn mk-FileFilter-for-log-cleanup [conf now-millis]
   (let [cutoff-age-millis (cleanup-cutoff-age-millis conf now-millis)]
     (reify FileFilter (^boolean accept [this ^File file]

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index c88e36b..fd11046 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -868,6 +868,7 @@
            {TOPOLOGY-KRYO-DECORATORS (get-merged-conf-val TOPOLOGY-KRYO-DECORATORS distinct)
             TOPOLOGY-KRYO-REGISTER (get-merged-conf-val TOPOLOGY-KRYO-REGISTER mapify-serializations)
             TOPOLOGY-ACKER-EXECUTORS (total-conf TOPOLOGY-ACKER-EXECUTORS)
+            TOPOLOGY-EVENTLOGGER-EXECUTORS (total-conf TOPOLOGY-EVENTLOGGER-EXECUTORS)
             TOPOLOGY-MAX-TASK-PARALLELISM (total-conf TOPOLOGY-MAX-TASK-PARALLELISM)})))
 
 (defn do-cleanup [nimbus]

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index b0e266c..0541c3a 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -21,14 +21,14 @@
         ring.middleware.multipart-params)
   (:use [ring.middleware.json :only [wrap-json-params]])
   (:use [hiccup core page-helpers])
-  (:use [backtype.storm config util log])
+  (:use [backtype.storm config util log tuple])
   (: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]]])
   (:use [clojure.string :only [blank? lower-case trim]])
   (:import [backtype.storm.utils Utils])
   (:import [backtype.storm.generated ExecutorSpecificStats
-            ExecutorStats ExecutorSummary TopologyInfo SpoutStats BoltStats
+            ExecutorStats ExecutorSummary ExecutorInfo TopologyInfo SpoutStats BoltStats
             ErrorInfo ClusterSummary SupervisorSummary TopologySummary
             Nimbus$Client StormTopology GlobalStreamId RebalanceOptions
             KillOptions GetInfoOptions NumErrorsChoice])
@@ -293,17 +293,52 @@
               (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"
+      host
+      (*STORM-CONF* LOGVIEWER-HTTPS-PORT)
+      fname)
+    (url-format "http://%s:%s/log?file=%s"
+      host
+      (*STORM-CONF* LOGVIEWER-PORT)
+      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?)))
+
 (defn worker-log-link [host port topology-id secure?]
   (let [fname (logs-filename topology-id port)]
-    (if (and secure? (*STORM-CONF* LOGVIEWER-HTTPS-PORT))
-      (url-format "https://%s:%s/log?file=%s"
-                  host
-                  (*STORM-CONF* LOGVIEWER-HTTPS-PORT)
-                  fname)
-      (url-format "http://%s:%s/log?file=%s"
-                  host
-                  (*STORM-CONF* LOGVIEWER-PORT)
-                  fname))))
+    (logviewer-link host fname secure?)))
 
 (defn compute-executor-capacity
   [^ExecutorSummary e]
@@ -910,7 +945,8 @@
          "encodedTopologyId" (url-encode topology-id)
          "window" window
          "componentType" (name type)
-         "windowHint" (window-hint window)}
+         "windowHint" (window-hint window)
+         "eventLogLink" (event-log-link topology-id summ topology component secure?)}
        spec errors))))
 
 (defn topology-config [topology-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 3fd9f2a..1e02054 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -1017,11 +1017,15 @@
   (.getCanonicalPath 
                 (clojure.java.io/file (System/getProperty "storm.home") "logs")))
 
-(defn- logs-rootname [storm-id port]
-  (str storm-id "-worker-" port))
+(defn- logs-rootname
+  ([storm-id port] (logs-rootname storm-id port "-worker-"))
+  ([storm-id port type] (str storm-id type port)))
 
-(defn logs-filename [storm-id port]
-  (str (logs-rootname storm-id port) ".log"))
+(defn logs-filename
+  ([storm-id port] (str (logs-rootname storm-id port) ".log"))
+  ([storm-id port type] (str (logs-rootname storm-id port type) ".log")))
+
+(defn event-logs-filename [storm-id port] (logs-filename storm-id port "-events-"))
 
 (defn logs-metadata-filename [storm-id port]
   (str (logs-rootname storm-id port) ".yaml"))

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 58e2a4a..1069056 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1087,6 +1087,15 @@ public class Config extends HashMap<String, Object> {
     public static final String TOPOLOGY_ACKER_EXECUTORS = "topology.acker.executors";
     public static final Object TOPOLOGY_ACKER_EXECUTORS_SCHEMA = ConfigValidation.IntegerValidator;
 
+    /**
+     * How many executors to spawn for event logger.
+     *
+     * <p>By not setting this variable or setting it as null, Storm will set the number of eventlogger executors
+     * to be equal to the number of workers configured for this topology. If this variable is set to 0,
+     * event logging will be disabled.</p>
+     */
+    public static final String TOPOLOGY_EVENTLOGGER_EXECUTORS = "topology.eventlogger.executors";
+    public static final Object TOPOLOGY_EVENTLOGGER_EXECUTORS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The maximum amount of time given to the topology to fully process a message
@@ -1500,6 +1509,15 @@ public class Config extends HashMap<String, Object> {
         setNumAckers(this, numExecutors);
     }
 
+    public static void setNumEventLoggers(Map conf, int numExecutors) {
+        conf.put(Config.TOPOLOGY_EVENTLOGGER_EXECUTORS, numExecutors);
+    }
+
+    public void setNumEventLoggers(int numExecutors) {
+        setNumEventLoggers(this, numExecutors);
+    }
+
+
     public static void setMessageTimeoutSecs(Map conf, int secs) {
         conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs);
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java b/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
new file mode 100644
index 0000000..19ff8c5
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
@@ -0,0 +1,56 @@
+package backtype.storm.metric;
+
+import backtype.storm.task.IBolt;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.Map;
+import static backtype.storm.metric.IEventLogger.EventInfo;
+
+public class EventLoggerBolt implements IBolt {
+
+    private static final Logger LOG = LoggerFactory.getLogger(EventLoggerBolt.class);
+
+    /*
+     The below field declarations are also used in common.clj to define the event logger output fields
+      */
+    public static final String FIELD_TS = "ts";
+    public static final String FIELD_VALUES = "values";
+    public static final String FIELD_COMPONENT_ID = "component-id";
+
+    private IEventLogger eventLogger;
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        LOG.info("EventLoggerBolt prepare called");
+        eventLogger = new FileBasedEventLogger();
+        eventLogger.prepare(stormConf, context);
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        LOG.debug("** EventLoggerBolt got tuple from sourceComponent {}, with values {}", input.getSourceComponent(), input.getValues());
+
+        EventInfo eventInfo = new EventInfo(input.getValueByField(FIELD_TS).toString(), input.getSourceComponent(),
+                                            String.valueOf(input.getSourceTask()), input.getMessageId().toString(),
+                                            input.getValueByField(FIELD_VALUES).toString());
+
+        eventLogger.log(eventInfo);
+    }
+
+    @Override
+    public void cleanup() {
+        eventLogger.close();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java b/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java
new file mode 100644
index 0000000..3834c55
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java
@@ -0,0 +1,105 @@
+package backtype.storm.metric;
+
+import backtype.storm.task.TopologyContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+public class FileBasedEventLogger implements IEventLogger {
+    private static Logger LOG = LoggerFactory.getLogger(FileBasedEventLogger.class);
+
+    private static final int FLUSH_INTERVAL_MILLIS = 1000;
+
+    private Path eventLogPath;
+    private BufferedWriter eventLogWriter;
+    private volatile boolean dirty = false;
+
+    private void initLogWriter(Path logFilePath) {
+        try {
+            LOG.info("logFilePath {}", logFilePath);
+            eventLogPath = logFilePath;
+            eventLogWriter = Files.newBufferedWriter(eventLogPath, StandardCharsets.UTF_8, StandardOpenOption.CREATE,
+                                                     StandardOpenOption.WRITE, StandardOpenOption.APPEND);
+        } catch (IOException e) {
+            LOG.error("Error setting up FileBasedEventLogger.", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+
+    private void setUpFlushTask() {
+        ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor();
+        Runnable task = new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    if(dirty) {
+                        eventLogWriter.flush();
+                        dirty = false;
+                    }
+                } catch (IOException ex) {
+                    LOG.error("Error flushing " + eventLogPath, ex);
+                    throw new RuntimeException(ex);
+                }
+            }
+        };
+
+        scheduler.scheduleAtFixedRate(task, FLUSH_INTERVAL_MILLIS, FLUSH_INTERVAL_MILLIS, TimeUnit.MILLISECONDS);
+    }
+
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context) {
+        String logDir;
+        String stormId = context.getStormId();
+        int port = context.getThisWorkerPort();
+        if((logDir = System.getProperty("storm.log.dir")) == null
+                && (logDir = System.getProperty("java.io.tmpdir")) == null) {
+            String msg = "Could not determine the directory to log events.";
+            LOG.error(msg);
+            throw new RuntimeException(msg);
+        } else {
+            LOG.info("FileBasedEventLogger log directory {}.", logDir);
+        }
+
+        /*
+         * Include the topology name & worker port in the file name so that
+         * multiple event loggers can log independently.
+         */
+        initLogWriter(Paths.get(logDir, String.format("%s-events-%d.log", stormId, port)));
+        setUpFlushTask();
+    }
+
+    @Override
+    public void log(EventInfo event) {
+        try {
+            //TODO: file rotation
+            eventLogWriter.write(event.toString());
+            eventLogWriter.newLine();
+            dirty = true;
+        } catch (IOException ex) {
+            LOG.error("Error logging event {}", event, ex);
+            throw new RuntimeException(ex);
+        }
+    }
+
+    @Override
+    public void close() {
+        try {
+            eventLogWriter.close();
+        } catch (IOException ex) {
+            LOG.error("Error closing event log.", ex);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java b/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java
new file mode 100644
index 0000000..9d0e7e9
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java
@@ -0,0 +1,59 @@
+package backtype.storm.metric;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+
+import java.util.Date;
+import java.util.Map;
+
+/**
+ * EventLogger interface for logging the event info to a sink like log file or db
+ * for inspecting the events via UI for debugging.
+ */
+public interface IEventLogger {
+
+    /**
+     * A wrapper for the fields that we would log.
+     */
+    public static class EventInfo {
+        String ts;
+        String component;
+        String task;
+        String messageId;
+        String values;
+        EventInfo(String ts, String component, String task, String messageId, String values) {
+            this.ts = ts;
+            this.component = component;
+            this.task = task;
+            this.messageId = messageId;
+            this.values = values;
+        }
+
+        /**
+         * Returns a default formatted string with fields separated by ","
+         *
+         * @return a default formatted string with fields separated by ","
+         */
+        @Override
+        public String toString() {
+            return new StringBuilder(new Date(Long.parseLong(ts)).toString()).append(",")
+                    .append(component).append(",")
+                    .append(task).append(",")
+                    .append(messageId).append(",")
+                    .append(values).toString();
+        }
+    }
+
+    void prepare(Map stormConf, TopologyContext context);
+
+    /**
+     * This method would be invoked when the {@link EventLoggerBolt} receives a tuple from the spouts or bolts that has
+     * event logging enabled.
+     *
+     * @param e the event
+     */
+    void log(EventInfo e);
+
+    void close();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/a924e9ef/storm-core/src/ui/public/templates/component-page-template.html
----------------------------------------------------------------------
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 5abc775..7bf9026 100644
--- a/storm-core/src/ui/public/templates/component-page-template.html
+++ b/storm-core/src/ui/public/templates/component-page-template.html
@@ -39,6 +39,11 @@
             Tasks
           </span>
         </th>
+        <th>
+          <span data-toggle="tooltip" data-placement="above" title="Click on the link below to open the logviewer and view the events emitted by this component.">
+            Debug
+          </span>
+        </th>
       </tr>
     </thead>
     <tbody>
@@ -47,6 +52,7 @@
         <td><a href="/topology.html?id={{encodedTopologyId}}">{{name}}</a></td>
         <td>{{executors}}</td>
         <td>{{tasks}}</td>
+        <td><a href="{{eventLogLink}}">events</a></td>
     </tbody>
   </table>
 </script>


[24/24] storm git commit: Added STORM-954 to CHANGELOG.

Posted by sr...@apache.org.
Added STORM-954 to CHANGELOG.


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

Branch: refs/heads/master
Commit: 206e7283a7752f6a57154e70e369733a7df37fa1
Parents: 05de790
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Mon Sep 21 10:34:06 2015 -0700
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Mon Sep 21 10:34:06 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/storm/blob/206e7283/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index e24dd5c..1933243 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-954: Toplogy Event Inspector
  * STORM-862: Pluggable System Metrics
  * STORM-1032: Add generics to component configuration methods
  * STORM-886: Automatic Back Pressure


[23/24] storm git commit: Merge branch 'STORM-954-PR' of https://github.com/arunmahadevan/storm into STORM-954

Posted by sr...@apache.org.
Merge branch 'STORM-954-PR' of https://github.com/arunmahadevan/storm into STORM-954


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

Branch: refs/heads/master
Commit: 05de790097452a257a46873b63f8a6a55a1fcba3
Parents: 3ec5120 0e680b3
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Mon Sep 21 10:13:27 2015 -0700
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Mon Sep 21 10:13:27 2015 -0700

----------------------------------------------------------------------
 conf/defaults.yaml                              |    1 +
 storm-core/src/clj/backtype/storm/cluster.clj   |    4 +-
 storm-core/src/clj/backtype/storm/converter.clj |   23 +-
 .../src/clj/backtype/storm/daemon/common.clj    |   47 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |   26 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |    1 +
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   27 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |   39 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  108 +-
 storm-core/src/clj/backtype/storm/util.clj      |   12 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   18 +
 .../backtype/storm/generated/Assignment.java    |  194 +--
 .../jvm/backtype/storm/generated/BoltStats.java |  342 ++---
 .../storm/generated/ClusterWorkerHeartbeat.java |   55 +-
 .../backtype/storm/generated/Credentials.java   |   46 +-
 .../backtype/storm/generated/DebugOptions.java  |  506 +++++++
 .../backtype/storm/generated/ExecutorStats.java |  162 +--
 .../storm/generated/LSApprovedWorkers.java      |   46 +-
 .../generated/LSSupervisorAssignments.java      |   50 +-
 .../storm/generated/LSWorkerHeartbeat.java      |   38 +-
 .../storm/generated/LocalAssignment.java        |   38 +-
 .../storm/generated/LocalStateData.java         |   50 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 1305 +++++++++++++++++-
 .../jvm/backtype/storm/generated/NodeInfo.java  |   34 +-
 .../storm/generated/RebalanceOptions.java       |   46 +-
 .../backtype/storm/generated/SpoutStats.java    |  226 +--
 .../jvm/backtype/storm/generated/StormBase.java |  225 ++-
 .../storm/generated/SupervisorInfo.java         |  110 +-
 .../backtype/storm/generated/TopologyInfo.java  |  259 +++-
 .../backtype/storm/metric/EventLoggerBolt.java  |   58 +
 .../storm/metric/FileBasedEventLogger.java      |  105 ++
 .../jvm/backtype/storm/metric/IEventLogger.java |   59 +
 storm-core/src/py/storm/Nimbus-remote           |    7 +
 storm-core/src/py/storm/Nimbus.py               |  263 ++++
 storm-core/src/py/storm/ttypes.py               |  425 ++++--
 storm-core/src/storm.thrift                     |   14 +
 storm-core/src/ui/public/component.html         |    8 +
 storm-core/src/ui/public/js/script.js           |   78 +-
 .../templates/component-page-template.html      |   13 +
 .../templates/topology-page-template.html       |   10 +-
 storm-core/src/ui/public/topology.html          |    3 +-
 .../test/clj/backtype/storm/cluster_test.clj    |    4 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |   62 +-
 43 files changed, 4123 insertions(+), 1024 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/05de7900/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/05de7900/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------


[08/24] storm git commit: nimbus api and stormbase changes to support component level debug

Posted by sr...@apache.org.
nimbus api and stormbase changes to support component level debug


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

Branch: refs/heads/master
Commit: 306ec57d9cebaaa8f5b9caa0c6fcfb256702bc77
Parents: 5914944
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Sun Aug 9 18:27:47 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 10 14:22:35 2015 +0530

----------------------------------------------------------------------
 .../backtype/storm/generated/Assignment.java    | 194 +++++-----
 .../storm/generated/ClusterWorkerHeartbeat.java |  55 ++-
 .../backtype/storm/generated/Credentials.java   |  46 +--
 .../storm/generated/LSApprovedWorkers.java      |  46 +--
 .../generated/LSSupervisorAssignments.java      |  50 +--
 .../storm/generated/LSWorkerHeartbeat.java      |  38 +-
 .../storm/generated/LocalAssignment.java        |  38 +-
 .../storm/generated/LocalStateData.java         |  50 +--
 .../jvm/backtype/storm/generated/Nimbus.java    | 144 +++++++-
 .../jvm/backtype/storm/generated/NodeInfo.java  |  34 +-
 .../storm/generated/RebalanceOptions.java       |  46 +--
 .../jvm/backtype/storm/generated/StormBase.java | 223 +++++++-----
 .../storm/generated/SupervisorInfo.java         | 110 +++---
 .../backtype/storm/generated/TopologyInfo.java  | 248 ++++++++-----
 storm-core/src/py/storm/Nimbus-remote           |   8 +-
 storm-core/src/py/storm/Nimbus.py               |  32 +-
 storm-core/src/py/storm/ttypes.py               | 352 ++++++++++---------
 storm-core/src/storm.thrift                     |   6 +-
 18 files changed, 990 insertions(+), 730 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/Assignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Assignment.java b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
index 2e6e8a3..933bbe1 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable, Comparable<Assignment> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment");
 
@@ -678,15 +678,15 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 2: // NODE_HOST
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map370 = iprot.readMapBegin();
-                struct.node_host = new HashMap<String,String>(2*_map370.size);
-                String _key371;
-                String _val372;
-                for (int _i373 = 0; _i373 < _map370.size; ++_i373)
+                org.apache.thrift.protocol.TMap _map380 = iprot.readMapBegin();
+                struct.node_host = new HashMap<String,String>(2*_map380.size);
+                String _key381;
+                String _val382;
+                for (int _i383 = 0; _i383 < _map380.size; ++_i383)
                 {
-                  _key371 = iprot.readString();
-                  _val372 = iprot.readString();
-                  struct.node_host.put(_key371, _val372);
+                  _key381 = iprot.readString();
+                  _val382 = iprot.readString();
+                  struct.node_host.put(_key381, _val382);
                 }
                 iprot.readMapEnd();
               }
@@ -698,26 +698,26 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 3: // EXECUTOR_NODE_PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map374 = iprot.readMapBegin();
-                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map374.size);
-                List<Long> _key375;
-                NodeInfo _val376;
-                for (int _i377 = 0; _i377 < _map374.size; ++_i377)
+                org.apache.thrift.protocol.TMap _map384 = iprot.readMapBegin();
+                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map384.size);
+                List<Long> _key385;
+                NodeInfo _val386;
+                for (int _i387 = 0; _i387 < _map384.size; ++_i387)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list378 = iprot.readListBegin();
-                    _key375 = new ArrayList<Long>(_list378.size);
-                    long _elem379;
-                    for (int _i380 = 0; _i380 < _list378.size; ++_i380)
+                    org.apache.thrift.protocol.TList _list388 = iprot.readListBegin();
+                    _key385 = new ArrayList<Long>(_list388.size);
+                    long _elem389;
+                    for (int _i390 = 0; _i390 < _list388.size; ++_i390)
                     {
-                      _elem379 = iprot.readI64();
-                      _key375.add(_elem379);
+                      _elem389 = iprot.readI64();
+                      _key385.add(_elem389);
                     }
                     iprot.readListEnd();
                   }
-                  _val376 = new NodeInfo();
-                  _val376.read(iprot);
-                  struct.executor_node_port.put(_key375, _val376);
+                  _val386 = new NodeInfo();
+                  _val386.read(iprot);
+                  struct.executor_node_port.put(_key385, _val386);
                 }
                 iprot.readMapEnd();
               }
@@ -729,25 +729,25 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 4: // EXECUTOR_START_TIME_SECS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map381 = iprot.readMapBegin();
-                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map381.size);
-                List<Long> _key382;
-                long _val383;
-                for (int _i384 = 0; _i384 < _map381.size; ++_i384)
+                org.apache.thrift.protocol.TMap _map391 = iprot.readMapBegin();
+                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map391.size);
+                List<Long> _key392;
+                long _val393;
+                for (int _i394 = 0; _i394 < _map391.size; ++_i394)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list385 = iprot.readListBegin();
-                    _key382 = new ArrayList<Long>(_list385.size);
-                    long _elem386;
-                    for (int _i387 = 0; _i387 < _list385.size; ++_i387)
+                    org.apache.thrift.protocol.TList _list395 = iprot.readListBegin();
+                    _key392 = new ArrayList<Long>(_list395.size);
+                    long _elem396;
+                    for (int _i397 = 0; _i397 < _list395.size; ++_i397)
                     {
-                      _elem386 = iprot.readI64();
-                      _key382.add(_elem386);
+                      _elem396 = iprot.readI64();
+                      _key392.add(_elem396);
                     }
                     iprot.readListEnd();
                   }
-                  _val383 = iprot.readI64();
-                  struct.executor_start_time_secs.put(_key382, _val383);
+                  _val393 = iprot.readI64();
+                  struct.executor_start_time_secs.put(_key392, _val393);
                 }
                 iprot.readMapEnd();
               }
@@ -779,10 +779,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           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<String, String> _iter388 : struct.node_host.entrySet())
+            for (Map.Entry<String, String> _iter398 : struct.node_host.entrySet())
             {
-              oprot.writeString(_iter388.getKey());
-              oprot.writeString(_iter388.getValue());
+              oprot.writeString(_iter398.getKey());
+              oprot.writeString(_iter398.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -794,17 +794,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           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<List<Long>, NodeInfo> _iter389 : struct.executor_node_port.entrySet())
+            for (Map.Entry<List<Long>, NodeInfo> _iter399 : struct.executor_node_port.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter389.getKey().size()));
-                for (long _iter390 : _iter389.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter399.getKey().size()));
+                for (long _iter400 : _iter399.getKey())
                 {
-                  oprot.writeI64(_iter390);
+                  oprot.writeI64(_iter400);
                 }
                 oprot.writeListEnd();
               }
-              _iter389.getValue().write(oprot);
+              _iter399.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -816,17 +816,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           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<List<Long>, Long> _iter391 : struct.executor_start_time_secs.entrySet())
+            for (Map.Entry<List<Long>, Long> _iter401 : struct.executor_start_time_secs.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter391.getKey().size()));
-                for (long _iter392 : _iter391.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter401.getKey().size()));
+                for (long _iter402 : _iter401.getKey())
                 {
-                  oprot.writeI64(_iter392);
+                  oprot.writeI64(_iter402);
                 }
                 oprot.writeListEnd();
               }
-              oprot.writeI64(_iter391.getValue());
+              oprot.writeI64(_iter401.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -865,42 +865,42 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       if (struct.is_set_node_host()) {
         {
           oprot.writeI32(struct.node_host.size());
-          for (Map.Entry<String, String> _iter393 : struct.node_host.entrySet())
+          for (Map.Entry<String, String> _iter403 : struct.node_host.entrySet())
           {
-            oprot.writeString(_iter393.getKey());
-            oprot.writeString(_iter393.getValue());
+            oprot.writeString(_iter403.getKey());
+            oprot.writeString(_iter403.getValue());
           }
         }
       }
       if (struct.is_set_executor_node_port()) {
         {
           oprot.writeI32(struct.executor_node_port.size());
-          for (Map.Entry<List<Long>, NodeInfo> _iter394 : struct.executor_node_port.entrySet())
+          for (Map.Entry<List<Long>, NodeInfo> _iter404 : struct.executor_node_port.entrySet())
           {
             {
-              oprot.writeI32(_iter394.getKey().size());
-              for (long _iter395 : _iter394.getKey())
+              oprot.writeI32(_iter404.getKey().size());
+              for (long _iter405 : _iter404.getKey())
               {
-                oprot.writeI64(_iter395);
+                oprot.writeI64(_iter405);
               }
             }
-            _iter394.getValue().write(oprot);
+            _iter404.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_executor_start_time_secs()) {
         {
           oprot.writeI32(struct.executor_start_time_secs.size());
-          for (Map.Entry<List<Long>, Long> _iter396 : struct.executor_start_time_secs.entrySet())
+          for (Map.Entry<List<Long>, Long> _iter406 : struct.executor_start_time_secs.entrySet())
           {
             {
-              oprot.writeI32(_iter396.getKey().size());
-              for (long _iter397 : _iter396.getKey())
+              oprot.writeI32(_iter406.getKey().size());
+              for (long _iter407 : _iter406.getKey())
               {
-                oprot.writeI64(_iter397);
+                oprot.writeI64(_iter407);
               }
             }
-            oprot.writeI64(_iter396.getValue());
+            oprot.writeI64(_iter406.getValue());
           }
         }
       }
@@ -914,64 +914,64 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map398 = 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<String,String>(2*_map398.size);
-          String _key399;
-          String _val400;
-          for (int _i401 = 0; _i401 < _map398.size; ++_i401)
+          org.apache.thrift.protocol.TMap _map408 = 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<String,String>(2*_map408.size);
+          String _key409;
+          String _val410;
+          for (int _i411 = 0; _i411 < _map408.size; ++_i411)
           {
-            _key399 = iprot.readString();
-            _val400 = iprot.readString();
-            struct.node_host.put(_key399, _val400);
+            _key409 = iprot.readString();
+            _val410 = iprot.readString();
+            struct.node_host.put(_key409, _val410);
           }
         }
         struct.set_node_host_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map402 = 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<List<Long>,NodeInfo>(2*_map402.size);
-          List<Long> _key403;
-          NodeInfo _val404;
-          for (int _i405 = 0; _i405 < _map402.size; ++_i405)
+          org.apache.thrift.protocol.TMap _map412 = 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<List<Long>,NodeInfo>(2*_map412.size);
+          List<Long> _key413;
+          NodeInfo _val414;
+          for (int _i415 = 0; _i415 < _map412.size; ++_i415)
           {
             {
-              org.apache.thrift.protocol.TList _list406 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key403 = new ArrayList<Long>(_list406.size);
-              long _elem407;
-              for (int _i408 = 0; _i408 < _list406.size; ++_i408)
+              org.apache.thrift.protocol.TList _list416 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key413 = new ArrayList<Long>(_list416.size);
+              long _elem417;
+              for (int _i418 = 0; _i418 < _list416.size; ++_i418)
               {
-                _elem407 = iprot.readI64();
-                _key403.add(_elem407);
+                _elem417 = iprot.readI64();
+                _key413.add(_elem417);
               }
             }
-            _val404 = new NodeInfo();
-            _val404.read(iprot);
-            struct.executor_node_port.put(_key403, _val404);
+            _val414 = new NodeInfo();
+            _val414.read(iprot);
+            struct.executor_node_port.put(_key413, _val414);
           }
         }
         struct.set_executor_node_port_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map409 = 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<List<Long>,Long>(2*_map409.size);
-          List<Long> _key410;
-          long _val411;
-          for (int _i412 = 0; _i412 < _map409.size; ++_i412)
+          org.apache.thrift.protocol.TMap _map419 = 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<List<Long>,Long>(2*_map419.size);
+          List<Long> _key420;
+          long _val421;
+          for (int _i422 = 0; _i422 < _map419.size; ++_i422)
           {
             {
-              org.apache.thrift.protocol.TList _list413 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key410 = new ArrayList<Long>(_list413.size);
-              long _elem414;
-              for (int _i415 = 0; _i415 < _list413.size; ++_i415)
+              org.apache.thrift.protocol.TList _list423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key420 = new ArrayList<Long>(_list423.size);
+              long _elem424;
+              for (int _i425 = 0; _i425 < _list423.size; ++_i425)
               {
-                _elem414 = iprot.readI64();
-                _key410.add(_elem414);
+                _elem424 = iprot.readI64();
+                _key420.add(_elem424);
               }
             }
-            _val411 = iprot.readI64();
-            struct.executor_start_time_secs.put(_key410, _val411);
+            _val421 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key420, _val421);
           }
         }
         struct.set_executor_start_time_secs_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
index f60540b..424ff41 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
@@ -51,8 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-8")
-
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWorkerHeartbeat, ClusterWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterWorkerHeartbeat> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterWorkerHeartbeat");
 
@@ -636,17 +635,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
           case 2: // EXECUTOR_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
-                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map426.size);
-                ExecutorInfo _key427;
-                ExecutorStats _val428;
-                for (int _i429 = 0; _i429 < _map426.size; ++_i429)
+                org.apache.thrift.protocol.TMap _map446 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map446.size);
+                ExecutorInfo _key447;
+                ExecutorStats _val448;
+                for (int _i449 = 0; _i449 < _map446.size; ++_i449)
                 {
-                  _key427 = new ExecutorInfo();
-                  _key427.read(iprot);
-                  _val428 = new ExecutorStats();
-                  _val428.read(iprot);
-                  struct.executor_stats.put(_key427, _val428);
+                  _key447 = new ExecutorInfo();
+                  _key447.read(iprot);
+                  _val448 = new ExecutorStats();
+                  _val448.read(iprot);
+                  struct.executor_stats.put(_key447, _val448);
                 }
                 iprot.readMapEnd();
               }
@@ -693,10 +692,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
         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<ExecutorInfo, ExecutorStats> _iter430 : struct.executor_stats.entrySet())
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter450 : struct.executor_stats.entrySet())
           {
-            _iter430.getKey().write(oprot);
-            _iter430.getValue().write(oprot);
+            _iter450.getKey().write(oprot);
+            _iter450.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -728,10 +727,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       oprot.writeString(struct.storm_id);
       {
         oprot.writeI32(struct.executor_stats.size());
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter431 : struct.executor_stats.entrySet())
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter451 : struct.executor_stats.entrySet())
         {
-          _iter431.getKey().write(oprot);
-          _iter431.getValue().write(oprot);
+          _iter451.getKey().write(oprot);
+          _iter451.getValue().write(oprot);
         }
       }
       oprot.writeI32(struct.time_secs);
@@ -744,17 +743,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       struct.storm_id = iprot.readString();
       struct.set_storm_id_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map432 = 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<ExecutorInfo,ExecutorStats>(2*_map432.size);
-        ExecutorInfo _key433;
-        ExecutorStats _val434;
-        for (int _i435 = 0; _i435 < _map432.size; ++_i435)
+        org.apache.thrift.protocol.TMap _map452 = 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<ExecutorInfo,ExecutorStats>(2*_map452.size);
+        ExecutorInfo _key453;
+        ExecutorStats _val454;
+        for (int _i455 = 0; _i455 < _map452.size; ++_i455)
         {
-          _key433 = new ExecutorInfo();
-          _key433.read(iprot);
-          _val434 = new ExecutorStats();
-          _val434.read(iprot);
-          struct.executor_stats.put(_key433, _val434);
+          _key453 = new ExecutorInfo();
+          _key453.read(iprot);
+          _val454 = new ExecutorStats();
+          _val454.read(iprot);
+          struct.executor_stats.put(_key453, _val454);
         }
       }
       struct.set_executor_stats_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/Credentials.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Credentials.java b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
index c1ef293..896d227 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class Credentials implements org.apache.thrift.TBase<Credentials, Credentials._Fields>, java.io.Serializable, Cloneable, Comparable<Credentials> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials");
 
@@ -365,15 +365,15 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
           case 1: // CREDS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map326 = iprot.readMapBegin();
-                struct.creds = new HashMap<String,String>(2*_map326.size);
-                String _key327;
-                String _val328;
-                for (int _i329 = 0; _i329 < _map326.size; ++_i329)
+                org.apache.thrift.protocol.TMap _map336 = iprot.readMapBegin();
+                struct.creds = new HashMap<String,String>(2*_map336.size);
+                String _key337;
+                String _val338;
+                for (int _i339 = 0; _i339 < _map336.size; ++_i339)
                 {
-                  _key327 = iprot.readString();
-                  _val328 = iprot.readString();
-                  struct.creds.put(_key327, _val328);
+                  _key337 = iprot.readString();
+                  _val338 = iprot.readString();
+                  struct.creds.put(_key337, _val338);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
         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<String, String> _iter330 : struct.creds.entrySet())
+          for (Map.Entry<String, String> _iter340 : struct.creds.entrySet())
           {
-            oprot.writeString(_iter330.getKey());
-            oprot.writeString(_iter330.getValue());
+            oprot.writeString(_iter340.getKey());
+            oprot.writeString(_iter340.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.creds.size());
-        for (Map.Entry<String, String> _iter331 : struct.creds.entrySet())
+        for (Map.Entry<String, String> _iter341 : struct.creds.entrySet())
         {
-          oprot.writeString(_iter331.getKey());
-          oprot.writeString(_iter331.getValue());
+          oprot.writeString(_iter341.getKey());
+          oprot.writeString(_iter341.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
     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 _map332 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.creds = new HashMap<String,String>(2*_map332.size);
-        String _key333;
-        String _val334;
-        for (int _i335 = 0; _i335 < _map332.size; ++_i335)
+        org.apache.thrift.protocol.TMap _map342 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.creds = new HashMap<String,String>(2*_map342.size);
+        String _key343;
+        String _val344;
+        for (int _i345 = 0; _i345 < _map342.size; ++_i345)
         {
-          _key333 = iprot.readString();
-          _val334 = iprot.readString();
-          struct.creds.put(_key333, _val334);
+          _key343 = iprot.readString();
+          _val344 = iprot.readString();
+          struct.creds.put(_key343, _val344);
         }
       }
       struct.set_creds_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
index 1d97dca..be950cb 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWorkers, LSApprovedWorkers._Fields>, java.io.Serializable, Cloneable, Comparable<LSApprovedWorkers> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSApprovedWorkers");
 
@@ -365,15 +365,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
           case 1: // APPROVED_WORKERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map454 = iprot.readMapBegin();
-                struct.approved_workers = new HashMap<String,Integer>(2*_map454.size);
-                String _key455;
-                int _val456;
-                for (int _i457 = 0; _i457 < _map454.size; ++_i457)
+                org.apache.thrift.protocol.TMap _map474 = iprot.readMapBegin();
+                struct.approved_workers = new HashMap<String,Integer>(2*_map474.size);
+                String _key475;
+                int _val476;
+                for (int _i477 = 0; _i477 < _map474.size; ++_i477)
                 {
-                  _key455 = iprot.readString();
-                  _val456 = iprot.readI32();
-                  struct.approved_workers.put(_key455, _val456);
+                  _key475 = iprot.readString();
+                  _val476 = iprot.readI32();
+                  struct.approved_workers.put(_key475, _val476);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
         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<String, Integer> _iter458 : struct.approved_workers.entrySet())
+          for (Map.Entry<String, Integer> _iter478 : struct.approved_workers.entrySet())
           {
-            oprot.writeString(_iter458.getKey());
-            oprot.writeI32(_iter458.getValue());
+            oprot.writeString(_iter478.getKey());
+            oprot.writeI32(_iter478.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.approved_workers.size());
-        for (Map.Entry<String, Integer> _iter459 : struct.approved_workers.entrySet())
+        for (Map.Entry<String, Integer> _iter479 : struct.approved_workers.entrySet())
         {
-          oprot.writeString(_iter459.getKey());
-          oprot.writeI32(_iter459.getValue());
+          oprot.writeString(_iter479.getKey());
+          oprot.writeI32(_iter479.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
     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 _map460 = 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<String,Integer>(2*_map460.size);
-        String _key461;
-        int _val462;
-        for (int _i463 = 0; _i463 < _map460.size; ++_i463)
+        org.apache.thrift.protocol.TMap _map480 = 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<String,Integer>(2*_map480.size);
+        String _key481;
+        int _val482;
+        for (int _i483 = 0; _i483 < _map480.size; ++_i483)
         {
-          _key461 = iprot.readString();
-          _val462 = iprot.readI32();
-          struct.approved_workers.put(_key461, _val462);
+          _key481 = iprot.readString();
+          _val482 = iprot.readI32();
+          struct.approved_workers.put(_key481, _val482);
         }
       }
       struct.set_approved_workers_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
index 4667287..4ad9aaf 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSupervisorAssignments, LSSupervisorAssignments._Fields>, java.io.Serializable, Cloneable, Comparable<LSSupervisorAssignments> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorAssignments");
 
@@ -376,16 +376,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
           case 1: // ASSIGNMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map464 = iprot.readMapBegin();
-                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map464.size);
-                int _key465;
-                LocalAssignment _val466;
-                for (int _i467 = 0; _i467 < _map464.size; ++_i467)
+                org.apache.thrift.protocol.TMap _map484 = iprot.readMapBegin();
+                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map484.size);
+                int _key485;
+                LocalAssignment _val486;
+                for (int _i487 = 0; _i487 < _map484.size; ++_i487)
                 {
-                  _key465 = iprot.readI32();
-                  _val466 = new LocalAssignment();
-                  _val466.read(iprot);
-                  struct.assignments.put(_key465, _val466);
+                  _key485 = iprot.readI32();
+                  _val486 = new LocalAssignment();
+                  _val486.read(iprot);
+                  struct.assignments.put(_key485, _val486);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
         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<Integer, LocalAssignment> _iter468 : struct.assignments.entrySet())
+          for (Map.Entry<Integer, LocalAssignment> _iter488 : struct.assignments.entrySet())
           {
-            oprot.writeI32(_iter468.getKey());
-            _iter468.getValue().write(oprot);
+            oprot.writeI32(_iter488.getKey());
+            _iter488.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.assignments.size());
-        for (Map.Entry<Integer, LocalAssignment> _iter469 : struct.assignments.entrySet())
+        for (Map.Entry<Integer, LocalAssignment> _iter489 : struct.assignments.entrySet())
         {
-          oprot.writeI32(_iter469.getKey());
-          _iter469.getValue().write(oprot);
+          oprot.writeI32(_iter489.getKey());
+          _iter489.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
     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 _map470 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map470.size);
-        int _key471;
-        LocalAssignment _val472;
-        for (int _i473 = 0; _i473 < _map470.size; ++_i473)
+        org.apache.thrift.protocol.TMap _map490 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map490.size);
+        int _key491;
+        LocalAssignment _val492;
+        for (int _i493 = 0; _i493 < _map490.size; ++_i493)
         {
-          _key471 = iprot.readI32();
-          _val472 = new LocalAssignment();
-          _val472.read(iprot);
-          struct.assignments.put(_key471, _val472);
+          _key491 = iprot.readI32();
+          _val492 = new LocalAssignment();
+          _val492.read(iprot);
+          struct.assignments.put(_key491, _val492);
         }
       }
       struct.set_assignments_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
index 80f917b..5384063 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartbeat, LSWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<LSWorkerHeartbeat> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSWorkerHeartbeat");
 
@@ -638,14 +638,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
           case 3: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list474 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list474.size);
-                ExecutorInfo _elem475;
-                for (int _i476 = 0; _i476 < _list474.size; ++_i476)
+                org.apache.thrift.protocol.TList _list494 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list494.size);
+                ExecutorInfo _elem495;
+                for (int _i496 = 0; _i496 < _list494.size; ++_i496)
                 {
-                  _elem475 = new ExecutorInfo();
-                  _elem475.read(iprot);
-                  struct.executors.add(_elem475);
+                  _elem495 = new ExecutorInfo();
+                  _elem495.read(iprot);
+                  struct.executors.add(_elem495);
                 }
                 iprot.readListEnd();
               }
@@ -687,9 +687,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
         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 _iter497 : struct.executors)
           {
-            _iter477.write(oprot);
+            _iter497.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter478 : struct.executors)
+        for (ExecutorInfo _iter498 : struct.executors)
         {
-          _iter478.write(oprot);
+          _iter498.write(oprot);
         }
       }
       oprot.writeI32(struct.port);
@@ -735,14 +735,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       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<ExecutorInfo>(_list479.size);
-        ExecutorInfo _elem480;
-        for (int _i481 = 0; _i481 < _list479.size; ++_i481)
+        org.apache.thrift.protocol.TList _list499 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list499.size);
+        ExecutorInfo _elem500;
+        for (int _i501 = 0; _i501 < _list499.size; ++_i501)
         {
-          _elem480 = new ExecutorInfo();
-          _elem480.read(iprot);
-          struct.executors.add(_elem480);
+          _elem500 = new ExecutorInfo();
+          _elem500.read(iprot);
+          struct.executors.add(_elem500);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
index 3c34891..695c2cb 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment, LocalAssignment._Fields>, java.io.Serializable, Cloneable, Comparable<LocalAssignment> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalAssignment");
 
@@ -464,14 +464,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
           case 2: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list446 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list446.size);
-                ExecutorInfo _elem447;
-                for (int _i448 = 0; _i448 < _list446.size; ++_i448)
+                org.apache.thrift.protocol.TList _list466 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list466.size);
+                ExecutorInfo _elem467;
+                for (int _i468 = 0; _i468 < _list466.size; ++_i468)
                 {
-                  _elem447 = new ExecutorInfo();
-                  _elem447.read(iprot);
-                  struct.executors.add(_elem447);
+                  _elem467 = new ExecutorInfo();
+                  _elem467.read(iprot);
+                  struct.executors.add(_elem467);
                 }
                 iprot.readListEnd();
               }
@@ -502,9 +502,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter449 : struct.executors)
+          for (ExecutorInfo _iter469 : struct.executors)
           {
-            _iter449.write(oprot);
+            _iter469.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -530,9 +530,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter450 : struct.executors)
+        for (ExecutorInfo _iter470 : struct.executors)
         {
-          _iter450.write(oprot);
+          _iter470.write(oprot);
         }
       }
     }
@@ -543,14 +543,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list451 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list451.size);
-        ExecutorInfo _elem452;
-        for (int _i453 = 0; _i453 < _list451.size; ++_i453)
+        org.apache.thrift.protocol.TList _list471 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list471.size);
+        ExecutorInfo _elem472;
+        for (int _i473 = 0; _i473 < _list471.size; ++_i473)
         {
-          _elem452 = new ExecutorInfo();
-          _elem452.read(iprot);
-          struct.executors.add(_elem452);
+          _elem472 = new ExecutorInfo();
+          _elem472.read(iprot);
+          struct.executors.add(_elem472);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
index 448711d..69eb2a9 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, LocalStateData._Fields>, java.io.Serializable, Cloneable, Comparable<LocalStateData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalStateData");
 
@@ -376,16 +376,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
           case 1: // SERIALIZED_PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map436 = iprot.readMapBegin();
-                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map436.size);
-                String _key437;
-                ThriftSerializedObject _val438;
-                for (int _i439 = 0; _i439 < _map436.size; ++_i439)
+                org.apache.thrift.protocol.TMap _map456 = iprot.readMapBegin();
+                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map456.size);
+                String _key457;
+                ThriftSerializedObject _val458;
+                for (int _i459 = 0; _i459 < _map456.size; ++_i459)
                 {
-                  _key437 = iprot.readString();
-                  _val438 = new ThriftSerializedObject();
-                  _val438.read(iprot);
-                  struct.serialized_parts.put(_key437, _val438);
+                  _key457 = iprot.readString();
+                  _val458 = new ThriftSerializedObject();
+                  _val458.read(iprot);
+                  struct.serialized_parts.put(_key457, _val458);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
         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<String, ThriftSerializedObject> _iter440 : struct.serialized_parts.entrySet())
+          for (Map.Entry<String, ThriftSerializedObject> _iter460 : struct.serialized_parts.entrySet())
           {
-            oprot.writeString(_iter440.getKey());
-            _iter440.getValue().write(oprot);
+            oprot.writeString(_iter460.getKey());
+            _iter460.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.serialized_parts.size());
-        for (Map.Entry<String, ThriftSerializedObject> _iter441 : struct.serialized_parts.entrySet())
+        for (Map.Entry<String, ThriftSerializedObject> _iter461 : struct.serialized_parts.entrySet())
         {
-          oprot.writeString(_iter441.getKey());
-          _iter441.getValue().write(oprot);
+          oprot.writeString(_iter461.getKey());
+          _iter461.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
     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 _map442 = 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<String,ThriftSerializedObject>(2*_map442.size);
-        String _key443;
-        ThriftSerializedObject _val444;
-        for (int _i445 = 0; _i445 < _map442.size; ++_i445)
+        org.apache.thrift.protocol.TMap _map462 = 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<String,ThriftSerializedObject>(2*_map462.size);
+        String _key463;
+        ThriftSerializedObject _val464;
+        for (int _i465 = 0; _i465 < _map462.size; ++_i465)
         {
-          _key443 = iprot.readString();
-          _val444 = new ThriftSerializedObject();
-          _val444.read(iprot);
-          struct.serialized_parts.put(_key443, _val444);
+          _key463 = iprot.readString();
+          _val464 = new ThriftSerializedObject();
+          _val464.read(iprot);
+          struct.serialized_parts.put(_key463, _val464);
         }
       }
       struct.set_serialized_parts_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index b535e69..1c2b2f0 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class Nimbus {
 
   public interface Iface {
@@ -70,7 +70,7 @@ public class Nimbus {
 
     public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
-    public void debug(String name, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+    public void debug(String name, String component, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
@@ -126,7 +126,7 @@ public class Nimbus {
 
     public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void debug(String name, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void debug(String name, String component, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -376,16 +376,17 @@ public class Nimbus {
       return;
     }
 
-    public void debug(String name, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    public void debug(String name, String component, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
-      send_debug(name, enable);
+      send_debug(name, component, enable);
       recv_debug();
     }
 
-    public void send_debug(String name, boolean enable) throws org.apache.thrift.TException
+    public void send_debug(String name, String component, boolean enable) throws org.apache.thrift.TException
     {
       debug_args args = new debug_args();
       args.set_name(name);
+      args.set_component(component);
       args.set_enable(enable);
       sendBase("debug", args);
     }
@@ -1022,19 +1023,21 @@ public class Nimbus {
       }
     }
 
-    public void debug(String name, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void debug(String name, String component, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      debug_call method_call = new debug_call(name, enable, resultHandler, this, ___protocolFactory, ___transport);
+      debug_call method_call = new debug_call(name, component, enable, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
 
     public static class debug_call extends org.apache.thrift.async.TAsyncMethodCall {
       private String name;
+      private String component;
       private boolean enable;
-      public debug_call(String name, boolean enable, 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 {
+      public debug_call(String name, String component, boolean enable, 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.name = name;
+        this.component = component;
         this.enable = enable;
       }
 
@@ -1042,6 +1045,7 @@ public class Nimbus {
         prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("debug", org.apache.thrift.protocol.TMessageType.CALL, 0));
         debug_args args = new debug_args();
         args.set_name(name);
+        args.set_component(component);
         args.set_enable(enable);
         args.write(prot);
         prot.writeMessageEnd();
@@ -1714,7 +1718,7 @@ public class Nimbus {
       public debug_result getResult(I iface, debug_args args) throws org.apache.thrift.TException {
         debug_result result = new debug_result();
         try {
-          iface.debug(args.name, args.enable);
+          iface.debug(args.name, args.component, args.enable);
         } catch (NotAliveException e) {
           result.e = e;
         } catch (AuthorizationException aze) {
@@ -2586,7 +2590,7 @@ public class Nimbus {
       }
 
       public void start(I iface, debug_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.debug(args.name, args.enable,resultHandler);
+        iface.debug(args.name, args.component, args.enable,resultHandler);
       }
     }
 
@@ -10416,7 +10420,8 @@ public class Nimbus {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("debug_args");
 
     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)1);
-    private static final org.apache.thrift.protocol.TField ENABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("enable", org.apache.thrift.protocol.TType.BOOL, (short)2);
+    private static final org.apache.thrift.protocol.TField COMPONENT_FIELD_DESC = new org.apache.thrift.protocol.TField("component", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField ENABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("enable", org.apache.thrift.protocol.TType.BOOL, (short)3);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -10425,12 +10430,14 @@ public class Nimbus {
     }
 
     private String name; // required
+    private String component; // required
     private boolean enable; // 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 {
       NAME((short)1, "name"),
-      ENABLE((short)2, "enable");
+      COMPONENT((short)2, "component"),
+      ENABLE((short)3, "enable");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -10447,7 +10454,9 @@ public class Nimbus {
         switch(fieldId) {
           case 1: // NAME
             return NAME;
-          case 2: // ENABLE
+          case 2: // COMPONENT
+            return COMPONENT;
+          case 3: // ENABLE
             return ENABLE;
           default:
             return null;
@@ -10496,6 +10505,8 @@ public class Nimbus {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.COMPONENT, new org.apache.thrift.meta_data.FieldMetaData("component", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       tmpMap.put(_Fields.ENABLE, new org.apache.thrift.meta_data.FieldMetaData("enable", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
@@ -10507,10 +10518,12 @@ public class Nimbus {
 
     public debug_args(
       String name,
+      String component,
       boolean enable)
     {
       this();
       this.name = name;
+      this.component = component;
       this.enable = enable;
       set_enable_isSet(true);
     }
@@ -10523,6 +10536,9 @@ public class Nimbus {
       if (other.is_set_name()) {
         this.name = other.name;
       }
+      if (other.is_set_component()) {
+        this.component = other.component;
+      }
       this.enable = other.enable;
     }
 
@@ -10533,6 +10549,7 @@ public class Nimbus {
     @Override
     public void clear() {
       this.name = null;
+      this.component = null;
       set_enable_isSet(false);
       this.enable = false;
     }
@@ -10560,6 +10577,29 @@ public class Nimbus {
       }
     }
 
+    public String get_component() {
+      return this.component;
+    }
+
+    public void set_component(String component) {
+      this.component = component;
+    }
+
+    public void unset_component() {
+      this.component = null;
+    }
+
+    /** Returns true if field component is set (has been assigned a value) and false otherwise */
+    public boolean is_set_component() {
+      return this.component != null;
+    }
+
+    public void set_component_isSet(boolean value) {
+      if (!value) {
+        this.component = null;
+      }
+    }
+
     public boolean is_enable() {
       return this.enable;
     }
@@ -10592,6 +10632,14 @@ public class Nimbus {
         }
         break;
 
+      case COMPONENT:
+        if (value == null) {
+          unset_component();
+        } else {
+          set_component((String)value);
+        }
+        break;
+
       case ENABLE:
         if (value == null) {
           unset_enable();
@@ -10608,6 +10656,9 @@ public class Nimbus {
       case NAME:
         return get_name();
 
+      case COMPONENT:
+        return get_component();
+
       case ENABLE:
         return Boolean.valueOf(is_enable());
 
@@ -10624,6 +10675,8 @@ public class Nimbus {
       switch (field) {
       case NAME:
         return is_set_name();
+      case COMPONENT:
+        return is_set_component();
       case ENABLE:
         return is_set_enable();
       }
@@ -10652,6 +10705,15 @@ public class Nimbus {
           return false;
       }
 
+      boolean this_present_component = true && this.is_set_component();
+      boolean that_present_component = true && that.is_set_component();
+      if (this_present_component || that_present_component) {
+        if (!(this_present_component && that_present_component))
+          return false;
+        if (!this.component.equals(that.component))
+          return false;
+      }
+
       boolean this_present_enable = true;
       boolean that_present_enable = true;
       if (this_present_enable || that_present_enable) {
@@ -10673,6 +10735,11 @@ public class Nimbus {
       if (present_name)
         list.add(name);
 
+      boolean present_component = true && (is_set_component());
+      list.add(present_component);
+      if (present_component)
+        list.add(component);
+
       boolean present_enable = true;
       list.add(present_enable);
       if (present_enable)
@@ -10699,6 +10766,16 @@ public class Nimbus {
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_component()).compareTo(other.is_set_component());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_component()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component, other.component);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       lastComparison = Boolean.valueOf(is_set_enable()).compareTo(other.is_set_enable());
       if (lastComparison != 0) {
         return lastComparison;
@@ -10737,6 +10814,14 @@ public class Nimbus {
       }
       first = false;
       if (!first) sb.append(", ");
+      sb.append("component:");
+      if (this.component == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component);
+      }
+      first = false;
+      if (!first) sb.append(", ");
       sb.append("enable:");
       sb.append(this.enable);
       first = false;
@@ -10793,7 +10878,15 @@ public class Nimbus {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
-            case 2: // ENABLE
+            case 2: // COMPONENT
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.component = iprot.readString();
+                struct.set_component_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // ENABLE
               if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
                 struct.enable = iprot.readBool();
                 struct.set_enable_isSet(true);
@@ -10819,6 +10912,11 @@ public class Nimbus {
           oprot.writeString(struct.name);
           oprot.writeFieldEnd();
         }
+        if (struct.component != null) {
+          oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
+          oprot.writeString(struct.component);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldBegin(ENABLE_FIELD_DESC);
         oprot.writeBool(struct.enable);
         oprot.writeFieldEnd();
@@ -10843,13 +10941,19 @@ public class Nimbus {
         if (struct.is_set_name()) {
           optionals.set(0);
         }
-        if (struct.is_set_enable()) {
+        if (struct.is_set_component()) {
           optionals.set(1);
         }
-        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_enable()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
         if (struct.is_set_name()) {
           oprot.writeString(struct.name);
         }
+        if (struct.is_set_component()) {
+          oprot.writeString(struct.component);
+        }
         if (struct.is_set_enable()) {
           oprot.writeBool(struct.enable);
         }
@@ -10858,12 +10962,16 @@ public class Nimbus {
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, debug_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(2);
+        BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
           struct.name = iprot.readString();
           struct.set_name_isSet(true);
         }
         if (incoming.get(1)) {
+          struct.component = iprot.readString();
+          struct.set_component_isSet(true);
+        }
+        if (incoming.get(2)) {
           struct.enable = iprot.readBool();
           struct.set_enable_isSet(true);
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
index a71d1b4..f844023 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<NodeInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo");
 
@@ -461,13 +461,13 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
           case 2: // PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set362 = iprot.readSetBegin();
-                struct.port = new HashSet<Long>(2*_set362.size);
-                long _elem363;
-                for (int _i364 = 0; _i364 < _set362.size; ++_i364)
+                org.apache.thrift.protocol.TSet _set372 = iprot.readSetBegin();
+                struct.port = new HashSet<Long>(2*_set372.size);
+                long _elem373;
+                for (int _i374 = 0; _i374 < _set372.size; ++_i374)
                 {
-                  _elem363 = iprot.readI64();
-                  struct.port.add(_elem363);
+                  _elem373 = iprot.readI64();
+                  struct.port.add(_elem373);
                 }
                 iprot.readSetEnd();
               }
@@ -498,9 +498,9 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
         oprot.writeFieldBegin(PORT_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size()));
-          for (long _iter365 : struct.port)
+          for (long _iter375 : struct.port)
           {
-            oprot.writeI64(_iter365);
+            oprot.writeI64(_iter375);
           }
           oprot.writeSetEnd();
         }
@@ -526,9 +526,9 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       oprot.writeString(struct.node);
       {
         oprot.writeI32(struct.port.size());
-        for (long _iter366 : struct.port)
+        for (long _iter376 : struct.port)
         {
-          oprot.writeI64(_iter366);
+          oprot.writeI64(_iter376);
         }
       }
     }
@@ -539,13 +539,13 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       struct.node = iprot.readString();
       struct.set_node_isSet(true);
       {
-        org.apache.thrift.protocol.TSet _set367 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.port = new HashSet<Long>(2*_set367.size);
-        long _elem368;
-        for (int _i369 = 0; _i369 < _set367.size; ++_i369)
+        org.apache.thrift.protocol.TSet _set377 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.port = new HashSet<Long>(2*_set377.size);
+        long _elem378;
+        for (int _i379 = 0; _i379 < _set377.size; ++_i379)
         {
-          _elem368 = iprot.readI64();
-          struct.port.add(_elem368);
+          _elem378 = iprot.readI64();
+          struct.port.add(_elem378);
         }
       }
       struct.set_port_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
index f973ffc..98d035a 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOptions, RebalanceOptions._Fields>, java.io.Serializable, Cloneable, Comparable<RebalanceOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RebalanceOptions");
 
@@ -529,15 +529,15 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
           case 3: // NUM_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map316 = iprot.readMapBegin();
-                struct.num_executors = new HashMap<String,Integer>(2*_map316.size);
-                String _key317;
-                int _val318;
-                for (int _i319 = 0; _i319 < _map316.size; ++_i319)
+                org.apache.thrift.protocol.TMap _map326 = iprot.readMapBegin();
+                struct.num_executors = new HashMap<String,Integer>(2*_map326.size);
+                String _key327;
+                int _val328;
+                for (int _i329 = 0; _i329 < _map326.size; ++_i329)
                 {
-                  _key317 = iprot.readString();
-                  _val318 = iprot.readI32();
-                  struct.num_executors.put(_key317, _val318);
+                  _key327 = iprot.readString();
+                  _val328 = iprot.readI32();
+                  struct.num_executors.put(_key327, _val328);
                 }
                 iprot.readMapEnd();
               }
@@ -574,10 +574,10 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
           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<String, Integer> _iter320 : struct.num_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter330 : struct.num_executors.entrySet())
             {
-              oprot.writeString(_iter320.getKey());
-              oprot.writeI32(_iter320.getValue());
+              oprot.writeString(_iter330.getKey());
+              oprot.writeI32(_iter330.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -621,10 +621,10 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
       if (struct.is_set_num_executors()) {
         {
           oprot.writeI32(struct.num_executors.size());
-          for (Map.Entry<String, Integer> _iter321 : struct.num_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter331 : struct.num_executors.entrySet())
           {
-            oprot.writeString(_iter321.getKey());
-            oprot.writeI32(_iter321.getValue());
+            oprot.writeString(_iter331.getKey());
+            oprot.writeI32(_iter331.getValue());
           }
         }
       }
@@ -644,15 +644,15 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map322 = 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<String,Integer>(2*_map322.size);
-          String _key323;
-          int _val324;
-          for (int _i325 = 0; _i325 < _map322.size; ++_i325)
+          org.apache.thrift.protocol.TMap _map332 = 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<String,Integer>(2*_map332.size);
+          String _key333;
+          int _val334;
+          for (int _i335 = 0; _i335 < _map332.size; ++_i335)
           {
-            _key323 = iprot.readString();
-            _val324 = iprot.readI32();
-            struct.num_executors.put(_key323, _val324);
+            _key333 = iprot.readString();
+            _val334 = iprot.readI32();
+            struct.num_executors.put(_key333, _val334);
           }
         }
         struct.set_num_executors_isSet(true);


[13/24] storm git commit: Add support for sampling percentage

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/ui/public/templates/component-page-template.html
----------------------------------------------------------------------
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 01b19ce..ccb4d5a 100644
--- a/storm-core/src/ui/public/templates/component-page-template.html
+++ b/storm-core/src/ui/public/templates/component-page-template.html
@@ -495,7 +495,7 @@
 <script id="component-actions-template" type="text/html">
   <h2>Component actions</h2>
   <p id="component-actions">
-    <input {{startDebugStatus}} onclick="confirmComponentAction('{{encodedTopologyId}}', '{{encodedId}}', '{{componentName}}', 'debug/enable', 'debug')" type="button" value="Debug" class="btn btn-default">
-    <input {{stopDebugStatus}} onclick="confirmComponentAction('{{encodedTopologyId}}', '{{encodedId}}', '{{componentName}}', 'debug/disable', 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
+    <input {{startDebugStatus}} onclick="confirmComponentAction('{{encodedTopologyId}}', '{{encodedId}}', '{{componentName}}', 'debug/enable', true, {{currentSamplingPct}}, 'sampling percentage', 'debug')" type="button" value="Debug" class="btn btn-default">
+    <input {{stopDebugStatus}} onclick="confirmComponentAction('{{encodedTopologyId}}', '{{encodedId}}', '{{componentName}}', 'debug/disable', false, 0, 'sampling percentage', 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
   </p>
 </script>

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------
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 cdf8ef5..628edfc 100644
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@ -340,11 +340,11 @@
 <script id="topology-actions-template" type="text/html">
   <h2>Topology actions</h2>
   <p id="topology-actions">
-    <input {{activateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'activate', false, 0)" type="button" value="Activate" class="btn btn-default">
-    <input {{deactivateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'deactivate', false, 0)" type="button" value="Deactivate" class="btn btn-default">
-    <input {{rebalanceStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'rebalance', true, {{msgTimeout}})" type="button" value="Rebalance" class="btn btn-default">
-    <input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30)" type="button" value="Kill" class="btn btn-default">
-    <input {{startDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/enable', false, 0, 'debug')" type="button" value="Debug" class="btn btn-default">
-    <input {{stopDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/disable', false, 0, 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
+    <input {{activateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'activate', false)" type="button" value="Activate" class="btn btn-default">
+    <input {{deactivateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'deactivate', false)" type="button" value="Deactivate" class="btn btn-default">
+    <input {{rebalanceStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'rebalance', true, {{msgTimeout}}, 'wait time in seconds')" type="button" value="Rebalance" class="btn btn-default">
+    <input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30, 'wait time in seconds')" type="button" value="Kill" class="btn btn-default">
+    <input {{startDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/enable', true, {{currentSamplingPct}}, 'sampling percentage', 'debug')" type="button" value="Debug" class="btn btn-default">
+    <input {{stopDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/disable', false, 0, 'sampling percentage', 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
   </p>
 </script>

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html
index 65cc107..22bcefd 100644
--- a/storm-core/src/ui/public/topology.html
+++ b/storm-core/src/ui/public/topology.html
@@ -132,7 +132,8 @@ $(document).ready(function() {
         var topologyActions = $("#topology-actions");
         var topologyVisualization = $("#topology-visualization")
         var formattedConfig = formatConfigData(response["configuration"]);
-        var buttonJsonData = topologyActionJson(response["id"],response["encodedId"],response["name"],response["status"],response["msgTimeout"],response["debug"]);
+        var buttonJsonData = topologyActionJson(response["id"],response["encodedId"],response["name"],response["status"]
+                                                ,response["msgTimeout"],response["debug"],response["samplingPct"]);
         $.get("/templates/topology-page-template.html", function(template) {
             topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));
             topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData));


[10/24] storm git commit: Fixed unit tests and a few issues

Posted by sr...@apache.org.
Fixed unit tests and a few issues


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

Branch: refs/heads/master
Commit: 35990fa28aa660b389afb00ca3ecfe38fdf519db
Parents: 7940d0c
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Tue Aug 11 12:26:45 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Tue Aug 11 12:26:45 2015 +0530

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/common.clj    |  7 ++-
 .../src/clj/backtype/storm/daemon/executor.clj  |  6 ++-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  8 ++--
 .../src/clj/backtype/storm/daemon/worker.clj    |  2 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  1 +
 storm-core/src/ui/public/component.html         |  3 +-
 storm-core/src/ui/public/js/script.js           |  6 +--
 .../test/clj/backtype/storm/cluster_test.clj    |  4 +-
 .../test/clj/backtype/storm/nimbus_test.clj     | 50 +++++++++++++-------
 9 files changed, 54 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/src/clj/backtype/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index 359c4b6..173cd5a 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -186,12 +186,15 @@
                              ))]
     (merge spout-inputs bolt-inputs)))
 
+;; the event logger receives inputs from all the spouts and bolts
+;; with a field grouping on component id so that all tuples from a component
+;; goes to same executor and can be viewed via logviewer.
 (defn eventlogger-inputs [^StormTopology topology]
   (let [bolt-ids (.. topology get_bolts keySet)
         spout-ids (.. topology get_spouts keySet)
         spout-inputs (apply merge
                        (for [id spout-ids]
-                         {[id EVENTLOGGER-STREAM-ID] ["component-id"]} ;grouping on component id
+                         {[id EVENTLOGGER-STREAM-ID] ["component-id"]}
                          ))
         bolt-inputs (apply merge
                       (for [id bolt-ids]
@@ -290,7 +293,7 @@
      (metrics-consumer-register-ids storm-conf)
      (get storm-conf TOPOLOGY-METRICS-CONSUMER-REGISTER))))
 
-; return the fields that event logger bolt expects
+;; return the fields that event logger bolt expects
 (defn eventlogger-bolt-fields []
   [(EventLoggerBolt/FIELD_COMPONENT_ID)  (EventLoggerBolt/FIELD_TS) (EventLoggerBolt/FIELD_VALUES)]
   )

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index e8a42f8..593ea0e 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -533,7 +533,8 @@
                                                                         out-tuple
                                                                         overflow-buffer)
                                                            ))
-                                         ; Send data to the eventlogger.
+                                         ;; Send data to the eventlogger if event logging is enabled in conf
+                                         ;; and the global or component level debug flag is set (via nimbus api).
                                          (if has-eventloggers?
                                            (let [c->d @(:storm-component->debug-atom executor-data)]
                                              (if (get c->d component-id (get c->d (:storm-id executor-data) false))
@@ -748,7 +749,8 @@
                                                                                stream
                                                                                (MessageId/makeId anchors-to-ids))
                                                                    overflow-buffer)))
-                                    ; send the data to the eventlogger
+                                    ;; Send data to the eventlogger if event logging is enabled in conf
+                                    ;; and the global or component level debug flag is set (via nimbus api).
                                     (if has-eventloggers?
                                       (let [c->d @(:storm-component->debug-atom executor-data)]
                                         (if (get c->d component-id (get c->d (:storm-id executor-data) false))

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index c9e80c7..c568e9e 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1161,14 +1161,14 @@
           (check-authorization! nimbus storm-name topology-conf "deactivate"))
         (transition-name! nimbus storm-name :inactivate true))
 
-      ;; TODO
       (debug [this storm-name component-id enable?]
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
+              topology-conf (try-read-storm-conf conf storm-id)
               storm-base-updates (assoc {} :component->debug (if (empty? component-id) {storm-id enable?} {component-id enable?}))]
-;;          (check-authorization! nimbus storm-name topology-conf "debug")
-;;          (when-not storm-id
-;;            (throw (NotAliveException. storm-name)))
+          (check-authorization! nimbus storm-name topology-conf "debug")
+          (when-not storm-id
+            (throw (NotAliveException. storm-name)))
           (log-message "Nimbus setting debug to " enable? " for storm-name " storm-name " storm-id " storm-id)
           (locking (:submit-lock nimbus)
             (.update-storm! storm-cluster-state storm-id storm-base-updates))))

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 69eddfb..b542348 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -329,7 +329,7 @@
         (:storm-active-atom worker)
         (and (= :active (-> base :status :type)) @(:worker-active-flag worker)))
       (reset! (:storm-component->debug-atom worker) (-> base :component->debug))
-      (log-message "debug options " @(:storm-component->debug-atom worker)))))
+      (log-debug "Event debug options " @(:storm-component->debug-atom worker)))))
 
 ;; TODO: consider having a max batch size besides what disruptor does automagically to prevent latency issues
 (defn mk-transfer-tuples-handler [worker]

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index a0e9171..d8fe30f 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -946,6 +946,7 @@
          "executors" (count summs)
          "tasks" (sum-tasks summs)
          "topologyId" topology-id
+         "topologyStatus" (.get_status summ)
          "encodedTopologyId" (url-encode topology-id)
          "window" window
          "componentType" (name type)

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/src/ui/public/component.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html
index 3b4b10d..5749a57 100644
--- a/storm-core/src/ui/public/component.html
+++ b/storm-core/src/ui/public/component.html
@@ -116,7 +116,8 @@ $(document).ready(function() {
 
         var componentSummary = $("#component-summary");
         var componentActions = $("#component-actions");
-        var buttonJsonData = componentActionJson(response["encodedTopologyId"], response["encodedId"], response["id"], response["debug"]);
+        var buttonJsonData = componentActionJson(response["encodedTopologyId"], response["encodedId"], response["id"],
+                                                 response["topologyStatus"], response["debug"]);
         var componentStatsDetail = $("#component-stats-detail")
         var inputStats = $("#component-input-stats");
         var outputStats = $("#component-output-stats");

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/src/ui/public/js/script.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js
index 2173b43..d707530 100644
--- a/storm-core/src/ui/public/js/script.js
+++ b/storm-core/src/ui/public/js/script.js
@@ -186,13 +186,13 @@ function topologyActionJson(id, encodedId, name, status, msgTimeout, debug) {
     return jsonData;
 }
 
-function componentActionJson(encodedTopologyId, encodedId, componentName, debug) {
+function componentActionJson(encodedTopologyId, encodedId, componentName, status, debug) {
     var jsonData = {};
     jsonData["encodedTopologyId"] = encodedTopologyId;
     jsonData["encodedId"] = encodedId;
     jsonData["componentName"] = componentName;
-    jsonData["startDebugStatus"] = (!debug) ? "enabled" : "disabled";
-    jsonData["stopDebugStatus"] = debug ? "enabled" : "disabled";
+    jsonData["startDebugStatus"] = (status === "ACTIVE" && !debug) ? "enabled" : "disabled";
+    jsonData["stopDebugStatus"] = (status === "ACTIVE" && debug) ? "enabled" : "disabled";
     return jsonData;
 }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/cluster_test.clj b/storm-core/test/clj/backtype/storm/cluster_test.clj
index 881c0d2..251c0c6 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -170,8 +170,8 @@
     (let [state (mk-storm-state zk-port)
           assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {})
           assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {})
-          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil)
-          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil)]
+          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil {})
+          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil {})]
       (is (= [] (.assignments state nil)))
       (.set-assignment! state "storm1" assignment1)
       (is (= assignment1 (.assignment-info state "storm1" nil)))

http://git-wip-us.apache.org/repos/asf/storm/blob/35990fa2/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index 10d135c..1f7258b 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -177,7 +177,8 @@
  	
 
 (deftest test-bogusId
-  (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
+  (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3
+                       :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
           nimbus (:nimbus cluster)]
        (is (thrown? NotAliveException (.getTopologyConf nimbus "bogus-id")))
@@ -188,7 +189,8 @@
       )))
 
 (deftest test-assignment
-  (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
+  (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3
+                       :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
           nimbus (:nimbus cluster)
           topology (thrift/mk-topology
@@ -246,6 +248,7 @@
                                       :ports-per-supervisor 3
                                       :daemon-conf {SUPERVISOR-ENABLE false
                                                     TOPOLOGY-ACKER-EXECUTORS 0
+                                                    TOPOLOGY-EVENTLOGGER-EXECUTORS 0
                                                     NIMBUS-CREDENTIAL-RENEW-FREQ-SECS 10
                                                     NIMBUS-CREDENTIAL-RENEWERS (list "backtype.storm.MockAutoCred")
                                                     NIMBUS-AUTO-CRED-PLUGINS (list "backtype.storm.MockAutoCred")
@@ -277,6 +280,7 @@
                                :inimbus (isolation-nimbus)
                                :daemon-conf {SUPERVISOR-ENABLE false
                                              TOPOLOGY-ACKER-EXECUTORS 0
+                                             TOPOLOGY-EVENTLOGGER-EXECUTORS 0
                                              STORM-SCHEDULER "backtype.storm.scheduler.IsolationScheduler"
                                              ISOLATION-SCHEDULER-MACHINES {"tester1" 3 "tester2" 2}
                                              NIMBUS-MONITOR-FREQ-SECS 10
@@ -326,7 +330,7 @@
       )))
 
 (deftest test-zero-executor-or-tasks
-  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
+  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
           nimbus (:nimbus cluster)
           topology (thrift/mk-topology
@@ -343,7 +347,7 @@
       )))
 
 (deftest test-executor-assignments
-  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
+  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [nimbus (:nimbus cluster)
           topology (thrift/mk-topology
                     {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3 :conf {TOPOLOGY-TASKS 5})}
@@ -365,7 +369,8 @@
       )))
 
 (deftest test-over-parallelism-assignment
-  (with-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
+  (with-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5
+                       :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
           nimbus (:nimbus cluster)
           topology (thrift/mk-topology
@@ -391,7 +396,8 @@
     :daemon-conf {SUPERVISOR-ENABLE false
                   NIMBUS-TASK-TIMEOUT-SECS 30
                   NIMBUS-MONITOR-FREQ-SECS 10
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (letlocals
       (bind conf (:daemon-conf cluster))
       (bind topology (thrift/mk-topology
@@ -480,7 +486,8 @@
                   NIMBUS-TASK-TIMEOUT-SECS 20
                   NIMBUS-MONITOR-FREQ-SECS 10
                   NIMBUS-SUPERVISOR-TIMEOUT-SECS 100
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (letlocals
       (bind conf (:daemon-conf cluster))
       (bind topology (thrift/mk-topology
@@ -575,7 +582,8 @@
                   NIMBUS-TASK-TIMEOUT-SECS 20
                   NIMBUS-MONITOR-FREQ-SECS 10
                   NIMBUS-SUPERVISOR-TIMEOUT-SECS 100
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (letlocals
       (add-supervisor cluster :ports 1 :id "a")
       (add-supervisor cluster :ports 1 :id "b")
@@ -632,7 +640,8 @@
                   NIMBUS-TASK-LAUNCH-SECS 60
                   NIMBUS-TASK-TIMEOUT-SECS 20
                   NIMBUS-MONITOR-FREQ-SECS 10
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (letlocals
       (bind topology (thrift/mk-topology
                         {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 9)}
@@ -678,7 +687,8 @@
     :daemon-conf {SUPERVISOR-ENABLE false
                   NIMBUS-MONITOR-FREQ-SECS 10
                   TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (letlocals
       (bind topology (thrift/mk-topology
                         {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -721,7 +731,8 @@
   (with-simulated-time-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3
     :daemon-conf {SUPERVISOR-ENABLE false
                   NIMBUS-MONITOR-FREQ-SECS 10
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (letlocals
       (bind topology (thrift/mk-topology
                         {"1" (thrift/mk-spout-spec (TestPlannerSpout. true)
@@ -797,7 +808,8 @@
     :daemon-conf {SUPERVISOR-ENABLE false
                   NIMBUS-MONITOR-FREQ-SECS 10
                   TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
-                  TOPOLOGY-ACKER-EXECUTORS 0}]
+                  TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (letlocals
       (bind topology (thrift/mk-topology
                         {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -842,6 +854,7 @@
   (with-simulated-time-local-cluster [cluster
     :daemon-conf {SUPERVISOR-ENABLE false
                   TOPOLOGY-ACKER-EXECUTORS 0
+                  TOPOLOGY-EVENTLOGGER-EXECUTORS 0
                   NIMBUS-EXECUTORS-PER-TOPOLOGY 8
                   NIMBUS-SLOTS-PER-TOPOLOGY 8}]
     (letlocals
@@ -1164,7 +1177,7 @@
                                             expected-acls)))))
 
 (deftest test-file-bogus-download
-  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0}]
+  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [nimbus (:nimbus cluster)]
       (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus nil)))
       (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus "")))
@@ -1213,9 +1226,10 @@
 
 (deftest test-debug-on
   (with-local-cluster [cluster]
-    (let [nimbus (:nimbus cluster)]
-      (stubbing [converter/clojurify-storm-base (backtype.storm.daemon.common.StormBase. "storm-name" 100
-                                                  {:type :active} 1 nil nil nil nil nil)
-                 cluster/maybe-deserialize nil]
-        (.debug nimbus "test" true)))))
+    (let [nimbus (:nimbus cluster)
+          topology (thrift/mk-topology
+                     {"spout" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                     {})]
+        (submit-local-topology nimbus "t1" {TOPOLOGY-WORKERS 1} topology)
+        (.debug nimbus "t1" "spout" true))))
 


[03/24] storm git commit: Nimbus debug api first cut

Posted by sr...@apache.org.
Nimbus debug api first cut

1. Added debug flag to storm base (currently boolean)
2. Added a new nimbus api for setting the flag
3. UI changes to enable and disable debugging at the topology level.


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

Branch: refs/heads/master
Commit: 26b02706201e3dc92b1dff8398795ae78d6753b6
Parents: a924e9e
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Thu Aug 6 18:26:38 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 10 14:21:43 2015 +0530

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   |    3 +-
 storm-core/src/clj/backtype/storm/converter.clj |    6 +-
 .../src/clj/backtype/storm/daemon/common.clj    |    2 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |    5 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   16 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |   43 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |   15 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 1083 +++++++++++++++++-
 .../jvm/backtype/storm/generated/StormBase.java |  109 +-
 .../backtype/storm/generated/TopologyInfo.java  |   99 +-
 storm-core/src/storm.thrift                     |    3 +
 storm-core/src/ui/public/js/script.js           |   13 +-
 .../templates/topology-page-template.html       |    2 +
 storm-core/src/ui/public/topology.html          |    2 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |   11 +-
 15 files changed, 1369 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index 1a13f7b..1070237 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -240,7 +240,8 @@
     (when cb
       (cb id))))
 
-(defn- maybe-deserialize
+;; public for stubbing in nimbus_test
+(defn maybe-deserialize
   [ser clazz]
   (when ser
     (Utils/deserialize ser clazz)))

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index 63868cd..a429c51 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -151,7 +151,8 @@
     (.set_component_executors (map-val int (:component->executors storm-base)))
     (.set_owner (:owner storm-base))
     (.set_topology_action_options (thriftify-topology-action-options storm-base))
-    (.set_prev_status (convert-to-status-from-symbol (:prev-status storm-base)))))
+    (.set_prev_status (convert-to-status-from-symbol (:prev-status storm-base)))
+    (.set_debug (:debug storm-base))))
 
 (defn clojurify-storm-base [^StormBase storm-base]
   (if storm-base
@@ -163,7 +164,8 @@
       (into {} (.get_component_executors storm-base))
       (.get_owner storm-base)
       (clojurify-topology-action-options (.get_topology_action_options storm-base))
-      (convert-to-symbol-from-status (.get_prev_status storm-base)))))
+      (convert-to-symbol-from-status (.get_prev_status storm-base))
+      (.is_debug storm-base))))
 
 (defn thriftify-stats [stats]
   (if stats

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/clj/backtype/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index 7868eb1..55cecee 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -55,7 +55,7 @@
 
 
 ;; component->executors is a map from spout/bolt id to number of executors for that component
-(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner topology-action-options prev-status])
+(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner topology-action-options prev-status debug])
 
 (defrecord SupervisorInfo [time-secs hostname assignment-id used-ports meta scheduler-meta uptime-secs version])
 

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index 757ee48..b3e356c 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -240,6 +240,7 @@
      :conf (:conf worker)
      :shared-executor-data (HashMap.)
      :storm-active-atom (:storm-active-atom worker)
+     :storm-debug-atom (:storm-debug-atom worker)
      :batch-transfer-queue batch-transfer->worker
      :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
      :suicide-fn (:suicide-fn worker)
@@ -533,7 +534,7 @@
                                                                         overflow-buffer)
                                                            ))
                                          ; Send data to the eventlogger.
-                                         (if has-eventloggers?
+                                         (if (and has-eventloggers? @(:storm-debug-atom executor-data))
                                            (task/send-unanchored
                                              task-data
                                              EVENTLOGGER-STREAM-ID
@@ -746,7 +747,7 @@
                                                                                (MessageId/makeId anchors-to-ids))
                                                                    overflow-buffer)))
                                     ; send the data to the eventlogger
-                                    (if has-eventloggers?
+                                    (if (and has-eventloggers? @(:storm-debug-atom executor-data))
                                       (task/send-unanchored task-data
                                         EVENTLOGGER-STREAM-ID
                                         [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index fd11046..3e39d44 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -754,7 +754,8 @@
                                   num-executors
                                   (storm-conf TOPOLOGY-SUBMITTER-USER)
                                   nil
-                                  nil))))
+                                  nil
+                                  false))))
 
 ;; Master:
 ;; job submit:
@@ -1160,6 +1161,18 @@
           (check-authorization! nimbus storm-name topology-conf "deactivate"))
         (transition-name! nimbus storm-name :inactivate true))
 
+      ;; TODO
+      (debug [this storm-name enable?]
+        (let [storm-cluster-state (:storm-cluster-state nimbus)
+              storm-id (get-storm-id storm-cluster-state storm-name)
+              storm-base-updates {:debug enable?}]
+;;          (check-authorization! nimbus storm-name topology-conf "debug")
+;;          (when-not storm-id
+;;            (throw (NotAliveException. storm-name)))
+          (log-message "Nimbus setting debug to " enable? " for storm-name " storm-name " storm-id " storm-id)
+          (locking (:submit-lock nimbus)
+            (.update-storm! storm-cluster-state storm-id storm-base-updates))))
+
       (uploadNewCredentials [this storm-name credentials]
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
@@ -1339,6 +1352,7 @@
                            executor-summaries
                            (extract-status-str base)
                            errors
+                           (:debug base)
                            )]
             (when-let [owner (:owner base)] (.set_owner topo-info owner))
             (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 2e64fb4..f947637 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -42,7 +42,7 @@
   (log-message "Reading Assignments.")
   (let [assignment (:executor->node+port (.assignment-info storm-cluster-state storm-id nil))]
     (doall
-     (concat     
+     (concat
       [Constants/SYSTEM_EXECUTOR_ID]
       (mapcat (fn [[executor loc]]
                 (if (= loc [assignment-id port])
@@ -63,7 +63,7 @@
                :time-secs (current-time-secs)
                }]
     ;; do the zookeeper heartbeat
-    (.worker-heartbeat! (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) zk-hb)    
+    (.worker-heartbeat! (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) zk-hb)
     ))
 
 (defn do-heartbeat [worker]
@@ -126,7 +126,7 @@
                   remoteMap (HashMap.)]
               (fast-list-iter [[task tuple :as pair] tuple-batch]
                 (if (local-tasks task)
-                  (.add local pair) 
+                  (.add local pair)
 
                   ;;Using java objects directly to avoid performance issues in java code
                   (do
@@ -141,7 +141,7 @@
                 (disruptor/publish transfer-queue remoteMap)
               ))]
     (if try-serialize-local
-      (do 
+      (do
         (log-warn "WILL TRY TO SERIALIZE ALL TUPLES (Turn off " TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE " for production)")
         (fn [^KryoTupleSerializer serializer tuple-batch]
           (assert-can-serialize serializer tuple-batch)
@@ -198,7 +198,7 @@
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
                                                   :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
         executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
-        
+
         receive-queue-map (->> executor-receive-queue-map
                                (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue])))
                                (into {}))
@@ -223,6 +223,7 @@
       ;; and spout and bolt will be activated.
       :worker-active-flag (atom false)
       :storm-active-atom (atom false)
+      :storm-debug-atom (atom false)
       :executors executors
       :task-ids (->> receive-queue-map keys (map int) sort)
       :storm-conf storm-conf
@@ -290,7 +291,7 @@
                                       (filter-key (complement (-> worker :task-ids set))))
               needed-connections (-> needed-assignment vals set)
               needed-tasks (-> needed-assignment keys)
-              
+
               current-connections (set (keys @(:cached-node+port->socket worker)))
               new-connections (set/difference needed-connections current-connections)
               remove-connections (set/difference current-connections needed-connections)]
@@ -316,7 +317,7 @@
                      (:cached-node+port->socket worker)
                      #(HashMap. (apply dissoc (into {} %1) %&))
                      remove-connections)
-              
+
            )))))
 
 (defn refresh-storm-active
@@ -324,11 +325,11 @@
     (refresh-storm-active worker (fn [& ignored] (schedule (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
   ([worker callback]
     (let [base (.storm-base (:storm-cluster-state worker) (:storm-id worker) callback)]
-     (reset!
-      (:storm-active-atom worker)
-       (and (= :active (-> base :status :type)) @(:worker-active-flag worker))
-      ))
-     ))
+      (reset!
+        (:storm-active-atom worker)
+        (and (= :active (-> base :status :type)) @(:worker-active-flag worker)))
+      (reset! (:storm-debug-atom worker) (-> base :debug))
+      (log-message "debug flag is " @(:storm-debug-atom worker)))))
 
 ;; TODO: consider having a max batch size besides what disruptor does automagically to prevent latency issues
 (defn mk-transfer-tuples-handler [worker]
@@ -341,7 +342,7 @@
     (disruptor/clojure-handler
       (fn [packets _ batch-end?]
         (.add drainer packets)
-        
+
         (when batch-end?
           (read-locked endpoint-socket-lock
              (let [node+port->socket @node+port->socket
@@ -423,7 +424,7 @@
         initial-credentials (.credentials storm-cluster-state storm-id nil)
         auto-creds (AuthUtils/GetAutoCredentials storm-conf)
         subject (AuthUtils/populateSubject nil auto-creds initial-credentials)]
-      (Subject/doAs subject (reify PrivilegedExceptionAction 
+      (Subject/doAs subject (reify PrivilegedExceptionAction
         (run [this]
           (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state)
         heartbeat-fn #(do-heartbeat worker)
@@ -452,8 +453,8 @@
         _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e initial-credentials)))
 
         transfer-tuples (mk-transfer-tuples-handler worker)
-        
-        transfer-thread (disruptor/consume-loop* (:transfer-queue worker) transfer-tuples)                                       
+
+        transfer-thread (disruptor/consume-loop* (:transfer-queue worker) transfer-tuples)
         shutdown* (fn []
                     (log-message "Shutting down worker " storm-id " " assignment-id " " port)
                     (doseq [[_ socket] @(:cached-node+port->socket worker)]
@@ -467,7 +468,7 @@
                     (log-message "Shutting down executors")
                     (doseq [executor @executors] (.shutdown executor))
                     (log-message "Shut down executors")
-                                        
+
                     ;;this is fine because the only time this is shared is when it's a local context,
                     ;;in which case it's a noop
                     (.term ^IContext (:mq-context worker))
@@ -483,11 +484,11 @@
                     (cancel-timer (:refresh-active-timer worker))
                     (cancel-timer (:executor-heartbeat-timer worker))
                     (cancel-timer (:user-timer worker))
-                    
+
                     (close-resources worker)
-                    
+
                     ;; TODO: here need to invoke the "shutdown" method of WorkerHook
-                    
+
                     (.remove-worker-heartbeat! (:storm-cluster-state worker) storm-id assignment-id port)
                     (log-message "Disconnecting from storm cluster state context")
                     (.disconnect (:storm-cluster-state worker))
@@ -535,7 +536,7 @@
   :distributed [conf]
   (fn [] (exit-process! 1 "Worker died")))
 
-(defn -main [storm-id assignment-id port-str worker-id]  
+(defn -main [storm-id assignment-id port-str worker-id]
   (let [conf (read-storm-config)]
     (setup-default-uncaught-exception-handler)
     (validate-distributed-mode! conf)

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index 0541c3a..ea7b048 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -724,7 +724,8 @@
        "tasksTotal" (sum-tasks executors)
        "workersTotal" (count workers)
        "executorsTotal" (count executors)
-       "schedulerInfo" (.get_sched_status summ)}))
+       "schedulerInfo" (.get_sched_status summ)
+       "debug" (.is_debug summ)}))
 
 (defn spout-summary-json [topology-id id stats window]
   (let [times (stats-times (:emitted stats))
@@ -1029,6 +1030,18 @@
         (.deactivate nimbus name)
         (log-message "Deactivating topology '" name "'")))
     (json-response (topology-op-response id "deactivate") (m "callback")))
+  (POST "/api/v1/topology/:id/debug/:action" [:as {:keys [cookies servlet-request]} id action & m]
+    (assert-authorized-user servlet-request "debug" (topology-config id))
+    (with-nimbus nimbus
+      (let [tplg (->> (doto
+                        (GetInfoOptions.)
+                        (.set_num_err_choice NumErrorsChoice/NONE))
+                   (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
+            name (.get_name tplg)
+            enable? (= "enable" action)]
+        (.debug nimbus name enable?)
+        (log-message "Debug topology '" name "' [" action "]")))
+    (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
     (assert-authorized-user servlet-request "rebalance" (topology-config id))
     (with-nimbus nimbus

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index cfc4496..b535e69 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-20")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-6")
 public class Nimbus {
 
   public interface Iface {
@@ -70,6 +70,8 @@ public class Nimbus {
 
     public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
+    public void debug(String name, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
     public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException;
@@ -124,6 +126,8 @@ public class Nimbus {
 
     public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void debug(String name, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -372,6 +376,33 @@ public class Nimbus {
       return;
     }
 
+    public void debug(String name, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_debug(name, enable);
+      recv_debug();
+    }
+
+    public void send_debug(String name, boolean enable) throws org.apache.thrift.TException
+    {
+      debug_args args = new debug_args();
+      args.set_name(name);
+      args.set_enable(enable);
+      sendBase("debug", args);
+    }
+
+    public void recv_debug() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      debug_result result = new debug_result();
+      receiveBase(result, "debug");
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      return;
+    }
+
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       send_uploadNewCredentials(name, creds);
@@ -991,6 +1022,41 @@ public class Nimbus {
       }
     }
 
+    public void debug(String name, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      debug_call method_call = new debug_call(name, enable, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class debug_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private boolean enable;
+      public debug_call(String name, boolean enable, 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.name = name;
+        this.enable = enable;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("debug", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        debug_args args = new debug_args();
+        args.set_name(name);
+        args.set_enable(enable);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void 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);
+        (new Client(prot)).recv_debug();
+      }
+    }
+
     public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       uploadNewCredentials_call method_call = new uploadNewCredentials_call(name, creds, resultHandler, this, ___protocolFactory, ___transport);
@@ -1427,6 +1493,7 @@ public class Nimbus {
       processMap.put("activate", new activate());
       processMap.put("deactivate", new deactivate());
       processMap.put("rebalance", new rebalance());
+      processMap.put("debug", new debug());
       processMap.put("uploadNewCredentials", new uploadNewCredentials());
       processMap.put("beginFileUpload", new beginFileUpload());
       processMap.put("uploadChunk", new uploadChunk());
@@ -1631,6 +1698,32 @@ public class Nimbus {
       }
     }
 
+    public static class debug<I extends Iface> extends org.apache.thrift.ProcessFunction<I, debug_args> {
+      public debug() {
+        super("debug");
+      }
+
+      public debug_args getEmptyArgsInstance() {
+        return new debug_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public debug_result getResult(I iface, debug_args args) throws org.apache.thrift.TException {
+        debug_result result = new debug_result();
+        try {
+          iface.debug(args.name, args.enable);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
     public static class uploadNewCredentials<I extends Iface> extends org.apache.thrift.ProcessFunction<I, uploadNewCredentials_args> {
       public uploadNewCredentials() {
         super("uploadNewCredentials");
@@ -1977,6 +2070,7 @@ public class Nimbus {
       processMap.put("activate", new activate());
       processMap.put("deactivate", new deactivate());
       processMap.put("rebalance", new rebalance());
+      processMap.put("debug", new debug());
       processMap.put("uploadNewCredentials", new uploadNewCredentials());
       processMap.put("beginFileUpload", new beginFileUpload());
       processMap.put("uploadChunk", new uploadChunk());
@@ -2435,6 +2529,67 @@ public class Nimbus {
       }
     }
 
+    public static class debug<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, debug_args, Void> {
+      public debug() {
+        super("debug");
+      }
+
+      public debug_args getEmptyArgsInstance() {
+        return new debug_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            debug_result result = new debug_result();
+            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;
+            debug_result result = new debug_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, debug_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.debug(args.name, args.enable,resultHandler);
+      }
+    }
+
     public static class uploadNewCredentials<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, uploadNewCredentials_args, Void> {
       public uploadNewCredentials() {
         super("uploadNewCredentials");
@@ -10257,6 +10412,932 @@ public class Nimbus {
 
   }
 
+  public static class debug_args implements org.apache.thrift.TBase<debug_args, debug_args._Fields>, java.io.Serializable, Cloneable, Comparable<debug_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("debug_args");
+
+    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)1);
+    private static final org.apache.thrift.protocol.TField ENABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("enable", org.apache.thrift.protocol.TType.BOOL, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new debug_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new debug_argsTupleSchemeFactory());
+    }
+
+    private String name; // required
+    private boolean enable; // 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 {
+      NAME((short)1, "name"),
+      ENABLE((short)2, "enable");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      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: // NAME
+            return NAME;
+          case 2: // ENABLE
+            return ENABLE;
+          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 __ENABLE_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.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.ENABLE, new org.apache.thrift.meta_data.FieldMetaData("enable", 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(debug_args.class, metaDataMap);
+    }
+
+    public debug_args() {
+    }
+
+    public debug_args(
+      String name,
+      boolean enable)
+    {
+      this();
+      this.name = name;
+      this.enable = enable;
+      set_enable_isSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public debug_args(debug_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.is_set_name()) {
+        this.name = other.name;
+      }
+      this.enable = other.enable;
+    }
+
+    public debug_args deepCopy() {
+      return new debug_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.name = null;
+      set_enable_isSet(false);
+      this.enable = false;
+    }
+
+    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 boolean is_enable() {
+      return this.enable;
+    }
+
+    public void set_enable(boolean enable) {
+      this.enable = enable;
+      set_enable_isSet(true);
+    }
+
+    public void unset_enable() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ENABLE_ISSET_ID);
+    }
+
+    /** Returns true if field enable is set (has been assigned a value) and false otherwise */
+    public boolean is_set_enable() {
+      return EncodingUtils.testBit(__isset_bitfield, __ENABLE_ISSET_ID);
+    }
+
+    public void set_enable_isSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ENABLE_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case NAME:
+        if (value == null) {
+          unset_name();
+        } else {
+          set_name((String)value);
+        }
+        break;
+
+      case ENABLE:
+        if (value == null) {
+          unset_enable();
+        } else {
+          set_enable((Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case NAME:
+        return get_name();
+
+      case ENABLE:
+        return Boolean.valueOf(is_enable());
+
+      }
+      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 NAME:
+        return is_set_name();
+      case ENABLE:
+        return is_set_enable();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof debug_args)
+        return this.equals((debug_args)that);
+      return false;
+    }
+
+    public boolean equals(debug_args that) {
+      if (that == null)
+        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_enable = true;
+      boolean that_present_enable = true;
+      if (this_present_enable || that_present_enable) {
+        if (!(this_present_enable && that_present_enable))
+          return false;
+        if (this.enable != that.enable)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_name = true && (is_set_name());
+      list.add(present_name);
+      if (present_name)
+        list.add(name);
+
+      boolean present_enable = true;
+      list.add(present_enable);
+      if (present_enable)
+        list.add(enable);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(debug_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      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_enable()).compareTo(other.is_set_enable());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_enable()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enable, other.enable);
+        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("debug_args(");
+      boolean first = true;
+
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("enable:");
+      sb.append(this.enable);
+      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 debug_argsStandardSchemeFactory implements SchemeFactory {
+      public debug_argsStandardScheme getScheme() {
+        return new debug_argsStandardScheme();
+      }
+    }
+
+    private static class debug_argsStandardScheme extends StandardScheme<debug_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, debug_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: // 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 2: // ENABLE
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.enable = iprot.readBool();
+                struct.set_enable_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, debug_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.name != null) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(ENABLE_FIELD_DESC);
+        oprot.writeBool(struct.enable);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class debug_argsTupleSchemeFactory implements SchemeFactory {
+      public debug_argsTupleScheme getScheme() {
+        return new debug_argsTupleScheme();
+      }
+    }
+
+    private static class debug_argsTupleScheme extends TupleScheme<debug_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, debug_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_name()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_enable()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_name()) {
+          oprot.writeString(struct.name);
+        }
+        if (struct.is_set_enable()) {
+          oprot.writeBool(struct.enable);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, debug_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.name = iprot.readString();
+          struct.set_name_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.enable = iprot.readBool();
+          struct.set_enable_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class debug_result implements org.apache.thrift.TBase<debug_result, debug_result._Fields>, java.io.Serializable, Cloneable, Comparable<debug_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("debug_result");
+
+    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<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new debug_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new debug_resultTupleSchemeFactory());
+    }
+
+    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 {
+      E((short)1, "e"),
+      AZE((short)2, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      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: // 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.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(debug_result.class, metaDataMap);
+    }
+
+    public debug_result() {
+    }
+
+    public debug_result(
+      NotAliveException e,
+      AuthorizationException aze)
+    {
+      this();
+      this.e = e;
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public debug_result(debug_result other) {
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public debug_result deepCopy() {
+      return new debug_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.e = null;
+      this.aze = 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 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 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 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 debug_result)
+        return this.equals((debug_result)that);
+      return false;
+    }
+
+    public boolean equals(debug_result that) {
+      if (that == null)
+        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<Object> list = new ArrayList<Object>();
+
+      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(debug_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      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("debug_result(");
+      boolean first = true;
+
+      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
+    }
+
+    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 debug_resultStandardSchemeFactory implements SchemeFactory {
+      public debug_resultStandardScheme getScheme() {
+        return new debug_resultStandardScheme();
+      }
+    }
+
+    private static class debug_resultStandardScheme extends StandardScheme<debug_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, debug_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 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, debug_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        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 debug_resultTupleSchemeFactory implements SchemeFactory {
+      public debug_resultTupleScheme getScheme() {
+        return new debug_resultTupleScheme();
+      }
+    }
+
+    private static class debug_resultTupleScheme extends TupleScheme<debug_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, debug_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_e()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_aze()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        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, debug_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.e = new NotAliveException();
+          struct.e.read(iprot);
+          struct.set_e_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.aze = new AuthorizationException();
+          struct.aze.read(iprot);
+          struct.set_aze_isSet(true);
+        }
+      }
+    }
+
+  }
+
   public static class uploadNewCredentials_args implements org.apache.thrift.TBase<uploadNewCredentials_args, uploadNewCredentials_args._Fields>, java.io.Serializable, Cloneable, Comparable<uploadNewCredentials_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadNewCredentials_args");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/jvm/backtype/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
index b00f072..a51ad45 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-6")
 public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
 
@@ -63,6 +63,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   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)6);
   private static final org.apache.thrift.protocol.TField TOPOLOGY_ACTION_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_action_options", org.apache.thrift.protocol.TType.STRUCT, (short)7);
   private static final org.apache.thrift.protocol.TField PREV_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("prev_status", org.apache.thrift.protocol.TType.I32, (short)8);
+  private static final org.apache.thrift.protocol.TField DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("debug", org.apache.thrift.protocol.TType.BOOL, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -78,6 +79,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   private String owner; // optional
   private TopologyActionOptions topology_action_options; // optional
   private TopologyStatus prev_status; // optional
+  private boolean debug; // 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 {
@@ -96,7 +98,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
      * 
      * @see TopologyStatus
      */
-    PREV_STATUS((short)8, "prev_status");
+    PREV_STATUS((short)8, "prev_status"),
+    DEBUG((short)9, "debug");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -127,6 +130,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           return TOPOLOGY_ACTION_OPTIONS;
         case 8: // PREV_STATUS
           return PREV_STATUS;
+        case 9: // DEBUG
+          return DEBUG;
         default:
           return null;
       }
@@ -169,8 +174,9 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   // isset id assignments
   private static final int __NUM_WORKERS_ISSET_ID = 0;
   private static final int __LAUNCH_TIME_SECS_ISSET_ID = 1;
+  private static final int __DEBUG_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.COMPONENT_EXECUTORS,_Fields.LAUNCH_TIME_SECS,_Fields.OWNER,_Fields.TOPOLOGY_ACTION_OPTIONS,_Fields.PREV_STATUS};
+  private static final _Fields optionals[] = {_Fields.COMPONENT_EXECUTORS,_Fields.LAUNCH_TIME_SECS,_Fields.OWNER,_Fields.TOPOLOGY_ACTION_OPTIONS,_Fields.PREV_STATUS,_Fields.DEBUG};
   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);
@@ -192,6 +198,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyActionOptions.class)));
     tmpMap.put(_Fields.PREV_STATUS, new org.apache.thrift.meta_data.FieldMetaData("prev_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
+    tmpMap.put(_Fields.DEBUG, new org.apache.thrift.meta_data.FieldMetaData("debug", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormBase.class, metaDataMap);
   }
@@ -237,6 +245,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     if (other.is_set_prev_status()) {
       this.prev_status = other.prev_status;
     }
+    this.debug = other.debug;
   }
 
   public StormBase deepCopy() {
@@ -255,6 +264,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     this.owner = null;
     this.topology_action_options = null;
     this.prev_status = null;
+    set_debug_isSet(false);
+    this.debug = false;
   }
 
   public String get_name() {
@@ -466,6 +477,28 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     }
   }
 
+  public boolean is_debug() {
+    return this.debug;
+  }
+
+  public void set_debug(boolean debug) {
+    this.debug = debug;
+    set_debug_isSet(true);
+  }
+
+  public void unset_debug() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  }
+
+  /** Returns true if field debug is set (has been assigned a value) and false otherwise */
+  public boolean is_set_debug() {
+    return EncodingUtils.testBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  }
+
+  public void set_debug_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DEBUG_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case NAME:
@@ -532,6 +565,14 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       break;
 
+    case DEBUG:
+      if (value == null) {
+        unset_debug();
+      } else {
+        set_debug((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -561,6 +602,9 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     case PREV_STATUS:
       return get_prev_status();
 
+    case DEBUG:
+      return Boolean.valueOf(is_debug());
+
     }
     throw new IllegalStateException();
   }
@@ -588,6 +632,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       return is_set_topology_action_options();
     case PREV_STATUS:
       return is_set_prev_status();
+    case DEBUG:
+      return is_set_debug();
     }
     throw new IllegalStateException();
   }
@@ -677,6 +723,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         return false;
     }
 
+    boolean this_present_debug = true && this.is_set_debug();
+    boolean that_present_debug = true && that.is_set_debug();
+    if (this_present_debug || that_present_debug) {
+      if (!(this_present_debug && that_present_debug))
+        return false;
+      if (this.debug != that.debug)
+        return false;
+    }
+
     return true;
   }
 
@@ -724,6 +779,11 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     if (present_prev_status)
       list.add(prev_status.getValue());
 
+    boolean present_debug = true && (is_set_debug());
+    list.add(present_debug);
+    if (present_debug)
+      list.add(debug);
+
     return list.hashCode();
   }
 
@@ -815,6 +875,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_debug()).compareTo(other.is_set_debug());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_debug()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug, other.debug);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -900,6 +970,12 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       first = false;
     }
+    if (is_set_debug()) {
+      if (!first) sb.append(", ");
+      sb.append("debug:");
+      sb.append(this.debug);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1034,6 +1110,14 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 9: // DEBUG
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.debug = iprot.readBool();
+              struct.set_debug_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1101,6 +1185,11 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldEnd();
         }
       }
+      if (struct.is_set_debug()) {
+        oprot.writeFieldBegin(DEBUG_FIELD_DESC);
+        oprot.writeBool(struct.debug);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1137,7 +1226,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_prev_status()) {
         optionals.set(4);
       }
-      oprot.writeBitSet(optionals, 5);
+      if (struct.is_set_debug()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
       if (struct.is_set_component_executors()) {
         {
           oprot.writeI32(struct.component_executors.size());
@@ -1160,6 +1252,9 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_prev_status()) {
         oprot.writeI32(struct.prev_status.getValue());
       }
+      if (struct.is_set_debug()) {
+        oprot.writeBool(struct.debug);
+      }
     }
 
     @Override
@@ -1171,7 +1266,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       struct.set_status_isSet(true);
       struct.num_workers = iprot.readI32();
       struct.set_num_workers_isSet(true);
-      BitSet incoming = iprot.readBitSet(5);
+      BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
           org.apache.thrift.protocol.TMap _map422 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
@@ -1204,6 +1299,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         struct.prev_status = backtype.storm.generated.TopologyStatus.findByValue(iprot.readI32());
         struct.set_prev_status_isSet(true);
       }
+      if (incoming.get(5)) {
+        struct.debug = iprot.readBool();
+        struct.set_debug_isSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index 6e1a503..76b7c83 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-6")
 public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
 
@@ -61,6 +61,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)4);
   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)5);
   private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.MAP, (short)6);
+  private static final org.apache.thrift.protocol.TField DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("debug", org.apache.thrift.protocol.TType.BOOL, (short)7);
   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)513);
   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)514);
 
@@ -76,6 +77,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private List<ExecutorSummary> executors; // required
   private String status; // required
   private Map<String,List<ErrorInfo>> errors; // required
+  private boolean debug; // required
   private String sched_status; // optional
   private String owner; // optional
 
@@ -87,6 +89,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     EXECUTORS((short)4, "executors"),
     STATUS((short)5, "status"),
     ERRORS((short)6, "errors"),
+    DEBUG((short)7, "debug"),
     SCHED_STATUS((short)513, "sched_status"),
     OWNER((short)514, "owner");
 
@@ -115,6 +118,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           return STATUS;
         case 6: // ERRORS
           return ERRORS;
+        case 7: // DEBUG
+          return DEBUG;
         case 513: // SCHED_STATUS
           return SCHED_STATUS;
         case 514: // OWNER
@@ -160,6 +165,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
 
   // isset id assignments
   private static final int __UPTIME_SECS_ISSET_ID = 0;
+  private static final int __DEBUG_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
   private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
@@ -181,6 +187,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             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.DEBUG, new org.apache.thrift.meta_data.FieldMetaData("debug", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     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.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -198,7 +206,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     int uptime_secs,
     List<ExecutorSummary> executors,
     String status,
-    Map<String,List<ErrorInfo>> errors)
+    Map<String,List<ErrorInfo>> errors,
+    boolean debug)
   {
     this();
     this.id = id;
@@ -208,6 +217,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     this.executors = executors;
     this.status = status;
     this.errors = errors;
+    this.debug = debug;
+    set_debug_isSet(true);
   }
 
   /**
@@ -250,6 +261,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       this.errors = __this__errors;
     }
+    this.debug = other.debug;
     if (other.is_set_sched_status()) {
       this.sched_status = other.sched_status;
     }
@@ -271,6 +283,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     this.executors = null;
     this.status = null;
     this.errors = null;
+    set_debug_isSet(false);
+    this.debug = false;
     this.sched_status = null;
     this.owner = null;
   }
@@ -438,6 +452,28 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     }
   }
 
+  public boolean is_debug() {
+    return this.debug;
+  }
+
+  public void set_debug(boolean debug) {
+    this.debug = debug;
+    set_debug_isSet(true);
+  }
+
+  public void unset_debug() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  }
+
+  /** Returns true if field debug is set (has been assigned a value) and false otherwise */
+  public boolean is_set_debug() {
+    return EncodingUtils.testBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  }
+
+  public void set_debug_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DEBUG_ISSET_ID, value);
+  }
+
   public String get_sched_status() {
     return this.sched_status;
   }
@@ -534,6 +570,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       break;
 
+    case DEBUG:
+      if (value == null) {
+        unset_debug();
+      } else {
+        set_debug((Boolean)value);
+      }
+      break;
+
     case SCHED_STATUS:
       if (value == null) {
         unset_sched_status();
@@ -573,6 +617,9 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     case ERRORS:
       return get_errors();
 
+    case DEBUG:
+      return Boolean.valueOf(is_debug());
+
     case SCHED_STATUS:
       return get_sched_status();
 
@@ -602,6 +649,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       return is_set_status();
     case ERRORS:
       return is_set_errors();
+    case DEBUG:
+      return is_set_debug();
     case SCHED_STATUS:
       return is_set_sched_status();
     case OWNER:
@@ -677,6 +726,15 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return false;
     }
 
+    boolean this_present_debug = true;
+    boolean that_present_debug = true;
+    if (this_present_debug || that_present_debug) {
+      if (!(this_present_debug && that_present_debug))
+        return false;
+      if (this.debug != that.debug)
+        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) {
@@ -732,6 +790,11 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     if (present_errors)
       list.add(errors);
 
+    boolean present_debug = true;
+    list.add(present_debug);
+    if (present_debug)
+      list.add(debug);
+
     boolean present_sched_status = true && (is_set_sched_status());
     list.add(present_sched_status);
     if (present_sched_status)
@@ -813,6 +876,16 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_debug()).compareTo(other.is_set_debug());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_debug()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug, other.debug);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(other.is_set_sched_status());
     if (lastComparison != 0) {
       return lastComparison;
@@ -896,6 +969,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       sb.append(this.errors);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("debug:");
+    sb.append(this.debug);
+    first = false;
     if (is_set_sched_status()) {
       if (!first) sb.append(", ");
       sb.append("sched_status:");
@@ -946,6 +1023,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'errors' is unset! Struct:" + toString());
     }
 
+    if (!is_set_debug()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'debug' is unset! Struct:" + toString());
+    }
+
     // check for sub-struct validity
   }
 
@@ -1067,6 +1148,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 7: // DEBUG
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.debug = iprot.readBool();
+              struct.set_debug_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           case 513: // SCHED_STATUS
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.sched_status = iprot.readString();
@@ -1146,6 +1235,9 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         }
         oprot.writeFieldEnd();
       }
+      oprot.writeFieldBegin(DEBUG_FIELD_DESC);
+      oprot.writeBool(struct.debug);
+      oprot.writeFieldEnd();
       if (struct.sched_status != null) {
         if (struct.is_set_sched_status()) {
           oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
@@ -1202,6 +1294,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           }
         }
       }
+      oprot.writeBool(struct.debug);
       BitSet optionals = new BitSet();
       if (struct.is_set_sched_status()) {
         optionals.set(0);
@@ -1264,6 +1357,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         }
       }
       struct.set_errors_isSet(true);
+      struct.debug = iprot.readBool();
+      struct.set_debug_isSet(true);
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.sched_status = iprot.readString();

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index a4b0b2a..e4436be 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -218,6 +218,7 @@ struct TopologyInfo {
   4: required list<ExecutorSummary> executors;
   5: required string status;
   6: required map<string, list<ErrorInfo>> errors;
+  7: required bool debug;
 513: optional string sched_status;
 514: optional string owner;
 }
@@ -289,6 +290,7 @@ struct StormBase {
     6: optional string owner;
     7: optional TopologyActionOptions topology_action_options;
     8: optional TopologyStatus prev_status;//currently only used during rebalance action.
+    9: optional bool debug;
 }
 
 struct ClusterWorkerHeartbeat {
@@ -349,6 +351,7 @@ service Nimbus {
   void activate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void deactivate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
+  void debug(1: string name, 2: bool enable) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
 
   // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/ui/public/js/script.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js
index abd66c1..fda17f2 100644
--- a/storm-core/src/ui/public/js/script.js
+++ b/storm-core/src/ui/public/js/script.js
@@ -81,13 +81,16 @@ function ensureInt(n) {
     return isInt;
 }
 
-function confirmAction(id, name, action, wait, defaultWait) {
+function confirmAction(id, name, action, wait, defaultWait, actionText) {
     var opts = {
         type:'POST',
         url:'/api/v1/topology/' + id + '/' + action
     };
+    if (actionText === undefined) {
+        actionText = action;
+    }
     if (wait) {
-        var waitSecs = prompt('Do you really want to ' + action + ' topology "' + name + '"? ' +
+        var waitSecs = prompt('Do you really want to ' + actionText + ' topology "' + name + '"? ' +
                               'If yes, please, specify wait time in seconds:',
                               defaultWait);
 
@@ -96,7 +99,7 @@ function confirmAction(id, name, action, wait, defaultWait) {
         } else {
             return false;
         }
-    } else if (!confirm('Do you really want to ' + action + ' topology "' + name + '"?')) {
+    } else if (!confirm('Do you really want to ' + actionText + ' topology "' + name + '"?')) {
         return false;
     }
 
@@ -146,7 +149,7 @@ function renderToggleSys(div) {
     }
 }
 
-function topologyActionJson(id, encodedId, name,status,msgTimeout) {
+function topologyActionJson(id, encodedId, name, status, msgTimeout, debug) {
     var jsonData = {};
     jsonData["id"] = id;
     jsonData["encodedId"] = encodedId;
@@ -156,6 +159,8 @@ function topologyActionJson(id, encodedId, name,status,msgTimeout) {
     jsonData["deactivateStatus"] = (status === "ACTIVE") ? "enabled" : "disabled";
     jsonData["rebalanceStatus"] = (status === "ACTIVE" || status === "INACTIVE" ) ? "enabled" : "disabled";
     jsonData["killStatus"] = (status !== "KILLED") ? "enabled" : "disabled";
+    jsonData["startDebugStatus"] = (status === "ACTIVE" && !debug) ? "enabled" : "disabled";
+    jsonData["stopDebugStatus"] = (status === "ACTIVE" && debug) ? "enabled" : "disabled";
     return jsonData;
 }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------
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 ddf89b7..cdf8ef5 100644
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@ -344,5 +344,7 @@
     <input {{deactivateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'deactivate', false, 0)" type="button" value="Deactivate" class="btn btn-default">
     <input {{rebalanceStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'rebalance', true, {{msgTimeout}})" type="button" value="Rebalance" class="btn btn-default">
     <input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30)" type="button" value="Kill" class="btn btn-default">
+    <input {{startDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/enable', false, 0, 'debug')" type="button" value="Debug" class="btn btn-default">
+    <input {{stopDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/disable', false, 0, 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
   </p>
 </script>

http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html
index 0a1a05b..65cc107 100644
--- a/storm-core/src/ui/public/topology.html
+++ b/storm-core/src/ui/public/topology.html
@@ -132,7 +132,7 @@ $(document).ready(function() {
         var topologyActions = $("#topology-actions");
         var topologyVisualization = $("#topology-visualization")
         var formattedConfig = formatConfigData(response["configuration"]);
-        var buttonJsonData = topologyActionJson(response["id"],response["encodedId"],response["name"],response["status"],response["msgTimeout"]);
+        var buttonJsonData = topologyActionJson(response["id"],response["encodedId"],response["name"],response["status"],response["msgTimeout"],response["debug"]);
         $.get("/templates/topology-page-template.html", function(template) {
             topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));
             topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData));


[18/24] storm git commit: merging from upstream

Posted by sr...@apache.org.
merging from upstream


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

Branch: refs/heads/master
Commit: 0151b441b0a68d490074d415d408caa522e47ffb
Parents: e5265d8 154e9ec
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Thu Sep 3 11:39:02 2015 -0700
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Thu Sep 3 11:39:02 2015 -0700

----------------------------------------------------------------------
 .gitignore                                      |   2 +-
 CHANGELOG.md                                    |  30 +
 README.markdown                                 |  13 +
 SECURITY.md                                     |   2 +-
 STORM-UI-REST-API.md                            |  41 +-
 TODO                                            | 178 ----
 bin/storm-config.cmd                            |  18 +-
 bin/storm.py                                    |  22 +-
 conf/defaults.yaml                              |  10 +-
 dev-tools/travis/travis-script.sh               |   2 +-
 docs/documentation/FAQ.md                       |   7 +-
 .../nimbus_ha_leader_election_and_failover.png  | Bin 0 -> 154316 bytes
 .../images/nimbus_ha_topology_submission.png    | Bin 0 -> 134180 bytes
 docs/documentation/nimbus-ha-design.md          | 217 +++++
 examples/storm-starter/pom.xml                  |  29 +
 .../starter/trident/TridentKafkaWordCount.java  | 230 +++++
 .../flux-core/src/test/resources/log4j2.xml     |  34 +
 .../flux-core/src/test/resources/logback.xml    |  30 -
 external/storm-elasticsearch/README.md          |  91 +-
 external/storm-elasticsearch/pom.xml            |  16 +-
 .../elasticsearch/ElasticsearchGetRequest.java  |  36 +
 .../elasticsearch/EsLookupResultOutput.java     |  43 +
 .../elasticsearch/bolt/AbstractEsBolt.java      |  56 +-
 .../storm/elasticsearch/bolt/EsIndexBolt.java   |  25 +-
 .../storm/elasticsearch/bolt/EsLookupBolt.java  |  81 ++
 .../elasticsearch/bolt/EsPercolateBolt.java     |  26 +-
 .../common/DefaultEsTupleMapper.java            |  42 +
 .../storm/elasticsearch/common/EsConfig.java    |  64 +-
 .../elasticsearch/common/EsTupleMapper.java     |  55 ++
 .../common/StormElasticSearchClient.java        |  48 +
 .../common/TransportAddresses.java              |  72 ++
 .../storm/elasticsearch/trident/EsState.java    |  54 +-
 .../elasticsearch/trident/EsStateFactory.java   |  27 +-
 .../storm/elasticsearch/trident/EsUpdater.java  |   6 +-
 .../bolt/AbstractEsBoltIntegrationTest.java     |  91 ++
 .../elasticsearch/bolt/AbstractEsBoltTest.java  |  80 +-
 .../elasticsearch/bolt/EsIndexBoltTest.java     |  43 +-
 .../elasticsearch/bolt/EsIndexTopology.java     |   8 +-
 .../bolt/EsLookupBoltIntegrationTest.java       | 137 +++
 .../elasticsearch/bolt/EsLookupBoltTest.java    | 125 +++
 .../elasticsearch/bolt/EsPercolateBoltTest.java |  35 +-
 .../elasticsearch/common/EsConfigTest.java      |  71 ++
 .../storm/elasticsearch/common/EsTestUtil.java  |   5 +
 .../common/TransportAddressesTest.java          |  81 ++
 .../trident/EsStateFactoryTest.java             |  32 +
 .../trident/TridentEsTopology.java              |   8 +-
 external/storm-hbase/pom.xml                    |   2 +-
 external/storm-hdfs/README.md                   |   9 +
 external/storm-hdfs/pom.xml                     |  15 +-
 .../ha/codedistributor/HDFSCodeDistributor.java | 101 +++
 .../apache/storm/hdfs/trident/HdfsState.java    | 392 ++++++--
 .../trident/rotation/FileRotationPolicy.java    |  14 +
 .../rotation/FileSizeRotationPolicy.java        |  13 +
 .../hdfs/trident/rotation/NoRotationPolicy.java |  10 +
 .../trident/rotation/TimedRotationPolicy.java   |  31 +-
 .../storm/hdfs/trident/HdfsStateTest.java       | 206 +++++
 external/storm-hive/README.md                   |  17 +-
 external/storm-hive/pom.xml                     |  24 +
 .../org/apache/storm/hive/bolt/HiveBolt.java    |  39 +-
 .../apache/storm/hive/common/HiveOptions.java   |  11 +
 .../apache/storm/hive/bolt/TestHiveBolt.java    |  12 +-
 external/storm-jdbc/README.md                   |   6 +-
 .../storm/jdbc/trident/state/JdbcState.java     |   2 +-
 .../UserPersistanceTridentTopology.java         |   2 +-
 external/storm-kafka/CHANGELOG.md               |  13 -
 external/storm-kafka/README.md                  |   3 +
 .../ExponentialBackoffMsgRetryManager.java      |   4 +-
 .../src/jvm/storm/kafka/KafkaSpout.java         |   6 +-
 .../src/jvm/storm/kafka/PartitionManager.java   |  10 +-
 .../src/jvm/storm/kafka/SpoutConfig.java        |   3 +
 .../src/jvm/storm/kafka/bolt/KafkaBolt.java     |  17 +-
 .../test/storm/kafka/bolt/KafkaBoltTest.java    |  32 +
 external/storm-solr/README.md                   | 201 +++++
 external/storm-solr/pom.xml                     |  98 ++
 .../apache/storm/solr/bolt/SolrUpdateBolt.java  | 136 +++
 .../storm/solr/config/CountBasedCommit.java     |  59 ++
 .../storm/solr/config/SolrCommitStrategy.java   |  30 +
 .../apache/storm/solr/config/SolrConfig.java    |  42 +
 .../storm/solr/mapper/SolrFieldsMapper.java     | 182 ++++
 .../storm/solr/mapper/SolrJsonMapper.java       | 160 ++++
 .../apache/storm/solr/mapper/SolrMapper.java    |  32 +
 .../storm/solr/mapper/SolrMapperException.java  |  24 +
 .../org/apache/storm/solr/schema/CopyField.java |  50 ++
 .../org/apache/storm/solr/schema/Field.java     |  50 ++
 .../org/apache/storm/solr/schema/FieldType.java |  63 ++
 .../org/apache/storm/solr/schema/Schema.java    | 116 +++
 .../storm/solr/schema/SolrFieldTypeFinder.java  | 182 ++++
 .../schema/builder/RestJsonSchemaBuilder.java   |  69 ++
 .../solr/schema/builder/SchemaBuilder.java      |  27 +
 .../apache/storm/solr/trident/SolrState.java    |  67 ++
 .../storm/solr/trident/SolrStateFactory.java    |  44 +
 .../apache/storm/solr/trident/SolrUpdater.java  |  33 +
 .../storm/solr/spout/SolrFieldsSpout.java       |  76 ++
 .../apache/storm/solr/spout/SolrJsonSpout.java  | 120 +++
 .../storm/solr/topology/SolrFieldsTopology.java |  56 ++
 .../storm/solr/topology/SolrJsonTopology.java   |  48 +
 .../storm/solr/topology/SolrTopology.java       |  82 ++
 .../solr/trident/SolrFieldsTridentTopology.java |  45 +
 .../solr/trident/SolrJsonTridentTopology.java   |  45 +
 .../org/apache/storm/solr/util/TestUtil.java    |  30 +
 pom.xml                                         |  19 +-
 storm-core/pom.xml                              |  89 +-
 storm-core/src/clj/backtype/storm/cluster.clj   | 100 ++-
 .../backtype/storm/command/shell_submission.clj |   9 +-
 storm-core/src/clj/backtype/storm/config.clj    |  15 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |   2 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 252 ++++--
 .../clj/backtype/storm/daemon/supervisor.clj    |  66 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |  23 +-
 storm-core/src/clj/backtype/storm/timer.clj     |   7 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  93 +-
 storm-core/src/clj/backtype/storm/util.clj      |   2 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj | 108 ++-
 storm-core/src/jvm/backtype/storm/Config.java   |  69 +-
 .../jvm/backtype/storm/ConfigValidation.java    |  13 +-
 .../storm/codedistributor/ICodeDistributor.java |  56 ++
 .../LocalFileSystemCodeDistributor.java         | 106 +++
 .../coordination/BatchSubtopologyBuilder.java   |   2 +-
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |   2 +-
 .../backtype/storm/generated/Assignment.java    | 194 ++--
 .../jvm/backtype/storm/generated/BoltStats.java | 342 +++----
 .../storm/generated/ClusterSummary.java         | 292 +++---
 .../storm/generated/ClusterWorkerHeartbeat.java |  54 +-
 .../backtype/storm/generated/Credentials.java   |  46 +-
 .../backtype/storm/generated/ExecutorStats.java | 162 ++--
 .../storm/generated/LSApprovedWorkers.java      |  46 +-
 .../generated/LSSupervisorAssignments.java      |  50 +-
 .../storm/generated/LSWorkerHeartbeat.java      |  38 +-
 .../storm/generated/LocalAssignment.java        |  38 +-
 .../storm/generated/LocalStateData.java         |  50 +-
 .../backtype/storm/generated/NimbusSummary.java | 796 +++++++++++++++++
 .../jvm/backtype/storm/generated/NodeInfo.java  |  34 +-
 .../storm/generated/RebalanceOptions.java       |  46 +-
 .../backtype/storm/generated/SpoutStats.java    | 226 ++---
 .../jvm/backtype/storm/generated/StormBase.java |  94 +-
 .../storm/generated/SupervisorInfo.java         | 110 +--
 .../backtype/storm/generated/TopologyInfo.java  | 269 ++++--
 .../storm/generated/TopologySummary.java        | 107 ++-
 .../backtype/storm/nimbus/ILeaderElector.java   |  60 ++
 .../jvm/backtype/storm/nimbus/NimbusInfo.java   |  93 ++
 .../security/auth/SaslTransportPlugin.java      |   8 +-
 .../authorizer/DRPCSimpleACLAuthorizer.java     |   2 +-
 .../authorizer/ImpersonationAuthorizer.java     |   2 +-
 .../kerberos/KerberosSaslTransportPlugin.java   |   5 +-
 .../security/auth/kerberos/NoOpTTrasport.java   |  40 +
 .../src/jvm/backtype/storm/task/ShellBolt.java  |   8 +-
 .../testing/OpaqueMemoryTransactionalSpout.java |   2 +-
 .../TransactionalTopologyBuilder.java           |   2 +-
 .../backtype/storm/utils/DisruptorQueue.java    | 127 ++-
 .../jvm/backtype/storm/utils/NimbusClient.java  |  78 +-
 .../jvm/backtype/storm/utils/RotatingMap.java   |  12 +-
 .../jvm/backtype/storm/utils/TimeCacheMap.java  |  60 +-
 .../src/jvm/backtype/storm/utils/Utils.java     |  32 +-
 .../storm/trident/planner/SubtopologyBolt.java  |   2 +-
 .../jvm/storm/trident/spout/ITridentSpout.java  |   4 +-
 .../OpaquePartitionedTridentSpoutExecutor.java  |  20 +-
 .../spout/PartitionedTridentSpoutExecutor.java  |  26 +-
 .../trident/spout/RichSpoutBatchExecutor.java   |   6 +-
 .../trident/spout/RichSpoutBatchTriggerer.java  |  14 +-
 .../trident/spout/TridentSpoutCoordinator.java  |   6 +-
 .../trident/spout/TridentSpoutExecutor.java     |  10 +-
 .../storm/trident/testing/FeederBatchSpout.java |   2 +-
 .../trident/topology/TridentBoltExecutor.java   |  17 +-
 .../topology/TridentTopologyBuilder.java        |   4 +-
 storm-core/src/py/storm/ttypes.py               | 891 +++++++++++--------
 storm-core/src/storm.thrift                     |  12 +-
 storm-core/src/ui/public/index.html             |  21 +
 .../public/templates/index-page-template.html   |  59 +-
 .../templates/topology-page-template.html       |  18 +-
 storm-core/src/ui/public/topology.html          |  35 +-
 .../test/clj/backtype/storm/cluster_test.clj    |  23 +-
 .../test/clj/backtype/storm/config_test.clj     |   4 +-
 .../storm/messaging/netty_unit_test.clj         |   2 +-
 .../test/clj/backtype/storm/nimbus_test.clj     | 210 +++--
 .../backtype/storm/security/auth/auth_test.clj  |   4 +-
 .../storm/security/auth/nimbus_auth_test.clj    |  14 +-
 .../test/clj/backtype/storm/supervisor_test.clj |   1 +
 .../test/clj/backtype/storm/utils_test.clj      |  12 -
 storm-dist/binary/src/main/assembly/binary.xml  |  16 +
 179 files changed, 9028 insertions(+), 2447 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index e20f31b,3ee4cdc..d815439
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@@ -34,8 -37,8 +37,8 @@@
              KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo
              ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice])
    (:import [backtype.storm.daemon Shutdownable])
-   (:use [backtype.storm util config log timer])
+   (:use [backtype.storm util config log timer zookeeper])
 -  (:require [backtype.storm [cluster :as cluster] [stats :as stats]])
 +  (:require [backtype.storm [cluster :as cluster] [stats :as stats] [converter :as converter]])
    (:require [clojure.set :as set])
    (:import [backtype.storm.daemon.common StormBase Assignment])
    (:use [backtype.storm.daemon common])
@@@ -1362,8 -1436,7 +1457,9 @@@
                             )]
              (when-let [owner (:owner base)] (.set_owner topo-info owner))
              (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))
 +            (when-let [component->debug (:component->debug base)]
 +              (.set_component_debug topo-info (map-val converter/thriftify-debugoptions component->debug)))
+             (.set_replication_count topo-info (.getReplicationCount (:code-distributor nimbus) storm-id))
              topo-info
            ))
  

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index 8f66147,4766736..7b44619
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -21,17 -21,18 +21,18 @@@
          ring.middleware.multipart-params)
    (:use [ring.middleware.json :only [wrap-json-params]])
    (:use [hiccup core page-helpers])
-   (:use [backtype.storm config util log tuple])
 -  (:use [backtype.storm config util log zookeeper])
++  (:use [backtype.storm config util log 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]]])
    (:use [clojure.string :only [blank? lower-case trim]])
-   (:import [backtype.storm.utils Utils])
+   (:import [backtype.storm.utils Utils]
+            [backtype.storm.generated NimbusSummary])
    (:import [backtype.storm.generated ExecutorSpecificStats
 -            ExecutorStats ExecutorSummary TopologyInfo SpoutStats BoltStats
 +            ExecutorStats ExecutorSummary ExecutorInfo TopologyInfo SpoutStats BoltStats
              ErrorInfo ClusterSummary SupervisorSummary TopologySummary
              Nimbus$Client StormTopology GlobalStreamId RebalanceOptions
 -            KillOptions GetInfoOptions NumErrorsChoice])
 +            KillOptions GetInfoOptions NumErrorsChoice DebugOptions])
    (:import [backtype.storm.security.auth AuthUtils ReqContext])
    (:import [backtype.storm.generated AuthorizationException])
    (:import [backtype.storm.security.auth AuthUtils])
@@@ -293,53 -286,21 +286,56 @@@
                (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"
 +      host
 +      (*STORM-CONF* LOGVIEWER-HTTPS-PORT)
 +      fname)
 +    (url-format "http://%s:%s/log?file=%s"
 +      host
 +      (*STORM-CONF* LOGVIEWER-PORT)
 +      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?)))
 +
  (defn worker-log-link [host port topology-id secure?]
    (let [fname (logs-filename topology-id port)]
 -    (if (and secure? (*STORM-CONF* LOGVIEWER-HTTPS-PORT))
 -      (url-format "https://%s:%s/log?file=%s"
 -                  host
 -                  (*STORM-CONF* LOGVIEWER-HTTPS-PORT)
 -                  fname)
 -      (url-format "http://%s:%s/log?file=%s"
 -                  host
 -                  (*STORM-CONF* LOGVIEWER-PORT)
 -                  fname))))
 +    (logviewer-link host fname secure?)))
  
+ (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)
@@@ -727,8 -720,7 +757,9 @@@
         "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) 100)}))
+        "replicationCount" (.get_replication_count summ)}))
  
  (defn spout-summary-json [topology-id id stats window]
    (let [times (stats-times (:emitted stats))
@@@ -1045,33 -1029,9 +1081,33 @@@
          (.deactivate nimbus name)
          (log-message "Deactivating topology '" name "'")))
      (json-response (topology-op-response id "deactivate") (m "callback")))
 +  (POST "/api/v1/topology/:id/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id action spct & m]
 +    (assert-authorized-user servlet-request "debug" (topology-config id))
 +    (with-nimbus nimbus
 +      (let [tplg (->> (doto
 +                        (GetInfoOptions.)
 +                        (.set_num_err_choice NumErrorsChoice/NONE))
 +                   (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
 +            name (.get_name tplg)
 +            enable? (= "enable" action)]
 +        (.debug nimbus name "" enable? (Integer/parseInt spct))
 +        (log-message "Debug topology [" name "] action [" action "] sampling pct [" spct "]")))
 +    (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
 +  (POST "/api/v1/topology/:id/component/:component/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id component action spct & m]
 +    (assert-authorized-user servlet-request "debug" (topology-config id))
 +    (with-nimbus nimbus
 +      (let [tplg (->> (doto
 +                        (GetInfoOptions.)
 +                        (.set_num_err_choice NumErrorsChoice/NONE))
 +                   (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
 +            name (.get_name tplg)
 +            enable? (= "enable" action)]
 +        (.debug nimbus name component enable? (Integer/parseInt spct))
 +        (log-message "Debug topology [" name "] component [" component "] action [" action "] sampling pct [" spct "]")))
 +    (json-response (component-op-response id component (str "/debug/" action)) (m "callback")))
    (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
+     (thrift/with-configured-nimbus-connection nimbus
      (assert-authorized-user servlet-request "rebalance" (topology-config id))
-     (with-nimbus nimbus
        (let [tplg (->> (doto
                          (GetInfoOptions.)
                          (.set_num_err_choice NumErrorsChoice/NONE))

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/Assignment.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/Assignment.java
index 933bbe1,2e6e8a3..f576c84
--- a/storm-core/src/jvm/backtype/storm/generated/Assignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable, Comparable<Assignment> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment");
  
@@@ -678,15 -678,15 +678,15 @@@
            case 2: // NODE_HOST
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map380 = iprot.readMapBegin();
-                 struct.node_host = new HashMap<String,String>(2*_map380.size);
-                 String _key381;
-                 String _val382;
-                 for (int _i383 = 0; _i383 < _map380.size; ++_i383)
 -                org.apache.thrift.protocol.TMap _map370 = iprot.readMapBegin();
 -                struct.node_host = new HashMap<String,String>(2*_map370.size);
 -                String _key371;
 -                String _val372;
 -                for (int _i373 = 0; _i373 < _map370.size; ++_i373)
++                org.apache.thrift.protocol.TMap _map388 = iprot.readMapBegin();
++                struct.node_host = new HashMap<String,String>(2*_map388.size);
++                String _key389;
++                String _val390;
++                for (int _i391 = 0; _i391 < _map388.size; ++_i391)
                  {
-                   _key381 = iprot.readString();
-                   _val382 = iprot.readString();
-                   struct.node_host.put(_key381, _val382);
 -                  _key371 = iprot.readString();
 -                  _val372 = iprot.readString();
 -                  struct.node_host.put(_key371, _val372);
++                  _key389 = iprot.readString();
++                  _val390 = iprot.readString();
++                  struct.node_host.put(_key389, _val390);
                  }
                  iprot.readMapEnd();
                }
@@@ -698,26 -698,26 +698,26 @@@
            case 3: // EXECUTOR_NODE_PORT
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map384 = iprot.readMapBegin();
-                 struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map384.size);
-                 List<Long> _key385;
-                 NodeInfo _val386;
-                 for (int _i387 = 0; _i387 < _map384.size; ++_i387)
 -                org.apache.thrift.protocol.TMap _map374 = iprot.readMapBegin();
 -                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map374.size);
 -                List<Long> _key375;
 -                NodeInfo _val376;
 -                for (int _i377 = 0; _i377 < _map374.size; ++_i377)
++                org.apache.thrift.protocol.TMap _map392 = iprot.readMapBegin();
++                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map392.size);
++                List<Long> _key393;
++                NodeInfo _val394;
++                for (int _i395 = 0; _i395 < _map392.size; ++_i395)
                  {
                    {
-                     org.apache.thrift.protocol.TList _list388 = iprot.readListBegin();
-                     _key385 = new ArrayList<Long>(_list388.size);
-                     long _elem389;
-                     for (int _i390 = 0; _i390 < _list388.size; ++_i390)
 -                    org.apache.thrift.protocol.TList _list378 = iprot.readListBegin();
 -                    _key375 = new ArrayList<Long>(_list378.size);
 -                    long _elem379;
 -                    for (int _i380 = 0; _i380 < _list378.size; ++_i380)
++                    org.apache.thrift.protocol.TList _list396 = iprot.readListBegin();
++                    _key393 = new ArrayList<Long>(_list396.size);
++                    long _elem397;
++                    for (int _i398 = 0; _i398 < _list396.size; ++_i398)
                      {
-                       _elem389 = iprot.readI64();
-                       _key385.add(_elem389);
 -                      _elem379 = iprot.readI64();
 -                      _key375.add(_elem379);
++                      _elem397 = iprot.readI64();
++                      _key393.add(_elem397);
                      }
                      iprot.readListEnd();
                    }
-                   _val386 = new NodeInfo();
-                   _val386.read(iprot);
-                   struct.executor_node_port.put(_key385, _val386);
 -                  _val376 = new NodeInfo();
 -                  _val376.read(iprot);
 -                  struct.executor_node_port.put(_key375, _val376);
++                  _val394 = new NodeInfo();
++                  _val394.read(iprot);
++                  struct.executor_node_port.put(_key393, _val394);
                  }
                  iprot.readMapEnd();
                }
@@@ -729,25 -729,25 +729,25 @@@
            case 4: // EXECUTOR_START_TIME_SECS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map391 = iprot.readMapBegin();
-                 struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map391.size);
-                 List<Long> _key392;
-                 long _val393;
-                 for (int _i394 = 0; _i394 < _map391.size; ++_i394)
 -                org.apache.thrift.protocol.TMap _map381 = iprot.readMapBegin();
 -                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map381.size);
 -                List<Long> _key382;
 -                long _val383;
 -                for (int _i384 = 0; _i384 < _map381.size; ++_i384)
++                org.apache.thrift.protocol.TMap _map399 = iprot.readMapBegin();
++                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map399.size);
++                List<Long> _key400;
++                long _val401;
++                for (int _i402 = 0; _i402 < _map399.size; ++_i402)
                  {
                    {
-                     org.apache.thrift.protocol.TList _list395 = iprot.readListBegin();
-                     _key392 = new ArrayList<Long>(_list395.size);
-                     long _elem396;
-                     for (int _i397 = 0; _i397 < _list395.size; ++_i397)
 -                    org.apache.thrift.protocol.TList _list385 = iprot.readListBegin();
 -                    _key382 = new ArrayList<Long>(_list385.size);
 -                    long _elem386;
 -                    for (int _i387 = 0; _i387 < _list385.size; ++_i387)
++                    org.apache.thrift.protocol.TList _list403 = iprot.readListBegin();
++                    _key400 = new ArrayList<Long>(_list403.size);
++                    long _elem404;
++                    for (int _i405 = 0; _i405 < _list403.size; ++_i405)
                      {
-                       _elem396 = iprot.readI64();
-                       _key392.add(_elem396);
 -                      _elem386 = iprot.readI64();
 -                      _key382.add(_elem386);
++                      _elem404 = iprot.readI64();
++                      _key400.add(_elem404);
                      }
                      iprot.readListEnd();
                    }
-                   _val393 = iprot.readI64();
-                   struct.executor_start_time_secs.put(_key392, _val393);
 -                  _val383 = iprot.readI64();
 -                  struct.executor_start_time_secs.put(_key382, _val383);
++                  _val401 = iprot.readI64();
++                  struct.executor_start_time_secs.put(_key400, _val401);
                  }
                  iprot.readMapEnd();
                }
@@@ -779,10 -779,10 +779,10 @@@
            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<String, String> _iter398 : struct.node_host.entrySet())
 -            for (Map.Entry<String, String> _iter388 : struct.node_host.entrySet())
++            for (Map.Entry<String, String> _iter406 : struct.node_host.entrySet())
              {
-               oprot.writeString(_iter398.getKey());
-               oprot.writeString(_iter398.getValue());
 -              oprot.writeString(_iter388.getKey());
 -              oprot.writeString(_iter388.getValue());
++              oprot.writeString(_iter406.getKey());
++              oprot.writeString(_iter406.getValue());
              }
              oprot.writeMapEnd();
            }
@@@ -794,17 -794,17 +794,17 @@@
            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<List<Long>, NodeInfo> _iter399 : struct.executor_node_port.entrySet())
 -            for (Map.Entry<List<Long>, NodeInfo> _iter389 : struct.executor_node_port.entrySet())
++            for (Map.Entry<List<Long>, NodeInfo> _iter407 : struct.executor_node_port.entrySet())
              {
                {
-                 oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter399.getKey().size()));
-                 for (long _iter400 : _iter399.getKey())
 -                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter389.getKey().size()));
 -                for (long _iter390 : _iter389.getKey())
++                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter407.getKey().size()));
++                for (long _iter408 : _iter407.getKey())
                  {
-                   oprot.writeI64(_iter400);
 -                  oprot.writeI64(_iter390);
++                  oprot.writeI64(_iter408);
                  }
                  oprot.writeListEnd();
                }
-               _iter399.getValue().write(oprot);
 -              _iter389.getValue().write(oprot);
++              _iter407.getValue().write(oprot);
              }
              oprot.writeMapEnd();
            }
@@@ -816,17 -816,17 +816,17 @@@
            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<List<Long>, Long> _iter401 : struct.executor_start_time_secs.entrySet())
 -            for (Map.Entry<List<Long>, Long> _iter391 : struct.executor_start_time_secs.entrySet())
++            for (Map.Entry<List<Long>, Long> _iter409 : struct.executor_start_time_secs.entrySet())
              {
                {
-                 oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter401.getKey().size()));
-                 for (long _iter402 : _iter401.getKey())
 -                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter391.getKey().size()));
 -                for (long _iter392 : _iter391.getKey())
++                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter409.getKey().size()));
++                for (long _iter410 : _iter409.getKey())
                  {
-                   oprot.writeI64(_iter402);
 -                  oprot.writeI64(_iter392);
++                  oprot.writeI64(_iter410);
                  }
                  oprot.writeListEnd();
                }
-               oprot.writeI64(_iter401.getValue());
 -              oprot.writeI64(_iter391.getValue());
++              oprot.writeI64(_iter409.getValue());
              }
              oprot.writeMapEnd();
            }
@@@ -865,42 -865,42 +865,42 @@@
        if (struct.is_set_node_host()) {
          {
            oprot.writeI32(struct.node_host.size());
-           for (Map.Entry<String, String> _iter403 : struct.node_host.entrySet())
 -          for (Map.Entry<String, String> _iter393 : struct.node_host.entrySet())
++          for (Map.Entry<String, String> _iter411 : struct.node_host.entrySet())
            {
-             oprot.writeString(_iter403.getKey());
-             oprot.writeString(_iter403.getValue());
 -            oprot.writeString(_iter393.getKey());
 -            oprot.writeString(_iter393.getValue());
++            oprot.writeString(_iter411.getKey());
++            oprot.writeString(_iter411.getValue());
            }
          }
        }
        if (struct.is_set_executor_node_port()) {
          {
            oprot.writeI32(struct.executor_node_port.size());
-           for (Map.Entry<List<Long>, NodeInfo> _iter404 : struct.executor_node_port.entrySet())
 -          for (Map.Entry<List<Long>, NodeInfo> _iter394 : struct.executor_node_port.entrySet())
++          for (Map.Entry<List<Long>, NodeInfo> _iter412 : struct.executor_node_port.entrySet())
            {
              {
-               oprot.writeI32(_iter404.getKey().size());
-               for (long _iter405 : _iter404.getKey())
 -              oprot.writeI32(_iter394.getKey().size());
 -              for (long _iter395 : _iter394.getKey())
++              oprot.writeI32(_iter412.getKey().size());
++              for (long _iter413 : _iter412.getKey())
                {
-                 oprot.writeI64(_iter405);
 -                oprot.writeI64(_iter395);
++                oprot.writeI64(_iter413);
                }
              }
-             _iter404.getValue().write(oprot);
 -            _iter394.getValue().write(oprot);
++            _iter412.getValue().write(oprot);
            }
          }
        }
        if (struct.is_set_executor_start_time_secs()) {
          {
            oprot.writeI32(struct.executor_start_time_secs.size());
-           for (Map.Entry<List<Long>, Long> _iter406 : struct.executor_start_time_secs.entrySet())
 -          for (Map.Entry<List<Long>, Long> _iter396 : struct.executor_start_time_secs.entrySet())
++          for (Map.Entry<List<Long>, Long> _iter414 : struct.executor_start_time_secs.entrySet())
            {
              {
-               oprot.writeI32(_iter406.getKey().size());
-               for (long _iter407 : _iter406.getKey())
 -              oprot.writeI32(_iter396.getKey().size());
 -              for (long _iter397 : _iter396.getKey())
++              oprot.writeI32(_iter414.getKey().size());
++              for (long _iter415 : _iter414.getKey())
                {
-                 oprot.writeI64(_iter407);
 -                oprot.writeI64(_iter397);
++                oprot.writeI64(_iter415);
                }
              }
-             oprot.writeI64(_iter406.getValue());
 -            oprot.writeI64(_iter396.getValue());
++            oprot.writeI64(_iter414.getValue());
            }
          }
        }
@@@ -914,64 -914,64 +914,64 @@@
        BitSet incoming = iprot.readBitSet(3);
        if (incoming.get(0)) {
          {
-           org.apache.thrift.protocol.TMap _map408 = 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<String,String>(2*_map408.size);
-           String _key409;
-           String _val410;
-           for (int _i411 = 0; _i411 < _map408.size; ++_i411)
 -          org.apache.thrift.protocol.TMap _map398 = 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<String,String>(2*_map398.size);
 -          String _key399;
 -          String _val400;
 -          for (int _i401 = 0; _i401 < _map398.size; ++_i401)
++          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<String,String>(2*_map416.size);
++          String _key417;
++          String _val418;
++          for (int _i419 = 0; _i419 < _map416.size; ++_i419)
            {
-             _key409 = iprot.readString();
-             _val410 = iprot.readString();
-             struct.node_host.put(_key409, _val410);
 -            _key399 = iprot.readString();
 -            _val400 = iprot.readString();
 -            struct.node_host.put(_key399, _val400);
++            _key417 = iprot.readString();
++            _val418 = iprot.readString();
++            struct.node_host.put(_key417, _val418);
            }
          }
          struct.set_node_host_isSet(true);
        }
        if (incoming.get(1)) {
          {
-           org.apache.thrift.protocol.TMap _map412 = 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<List<Long>,NodeInfo>(2*_map412.size);
-           List<Long> _key413;
-           NodeInfo _val414;
-           for (int _i415 = 0; _i415 < _map412.size; ++_i415)
 -          org.apache.thrift.protocol.TMap _map402 = 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<List<Long>,NodeInfo>(2*_map402.size);
 -          List<Long> _key403;
 -          NodeInfo _val404;
 -          for (int _i405 = 0; _i405 < _map402.size; ++_i405)
++          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<List<Long>,NodeInfo>(2*_map420.size);
++          List<Long> _key421;
++          NodeInfo _val422;
++          for (int _i423 = 0; _i423 < _map420.size; ++_i423)
            {
              {
-               org.apache.thrift.protocol.TList _list416 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-               _key413 = new ArrayList<Long>(_list416.size);
-               long _elem417;
-               for (int _i418 = 0; _i418 < _list416.size; ++_i418)
 -              org.apache.thrift.protocol.TList _list406 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
 -              _key403 = new ArrayList<Long>(_list406.size);
 -              long _elem407;
 -              for (int _i408 = 0; _i408 < _list406.size; ++_i408)
++              org.apache.thrift.protocol.TList _list424 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
++              _key421 = new ArrayList<Long>(_list424.size);
++              long _elem425;
++              for (int _i426 = 0; _i426 < _list424.size; ++_i426)
                {
-                 _elem417 = iprot.readI64();
-                 _key413.add(_elem417);
 -                _elem407 = iprot.readI64();
 -                _key403.add(_elem407);
++                _elem425 = iprot.readI64();
++                _key421.add(_elem425);
                }
              }
-             _val414 = new NodeInfo();
-             _val414.read(iprot);
-             struct.executor_node_port.put(_key413, _val414);
 -            _val404 = new NodeInfo();
 -            _val404.read(iprot);
 -            struct.executor_node_port.put(_key403, _val404);
++            _val422 = new NodeInfo();
++            _val422.read(iprot);
++            struct.executor_node_port.put(_key421, _val422);
            }
          }
          struct.set_executor_node_port_isSet(true);
        }
        if (incoming.get(2)) {
          {
-           org.apache.thrift.protocol.TMap _map419 = 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<List<Long>,Long>(2*_map419.size);
-           List<Long> _key420;
-           long _val421;
-           for (int _i422 = 0; _i422 < _map419.size; ++_i422)
 -          org.apache.thrift.protocol.TMap _map409 = 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<List<Long>,Long>(2*_map409.size);
 -          List<Long> _key410;
 -          long _val411;
 -          for (int _i412 = 0; _i412 < _map409.size; ++_i412)
++          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<List<Long>,Long>(2*_map427.size);
++          List<Long> _key428;
++          long _val429;
++          for (int _i430 = 0; _i430 < _map427.size; ++_i430)
            {
              {
-               org.apache.thrift.protocol.TList _list423 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-               _key420 = new ArrayList<Long>(_list423.size);
-               long _elem424;
-               for (int _i425 = 0; _i425 < _list423.size; ++_i425)
 -              org.apache.thrift.protocol.TList _list413 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
 -              _key410 = new ArrayList<Long>(_list413.size);
 -              long _elem414;
 -              for (int _i415 = 0; _i415 < _list413.size; ++_i415)
++              org.apache.thrift.protocol.TList _list431 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
++              _key428 = new ArrayList<Long>(_list431.size);
++              long _elem432;
++              for (int _i433 = 0; _i433 < _list431.size; ++_i433)
                {
-                 _elem424 = iprot.readI64();
-                 _key420.add(_elem424);
 -                _elem414 = iprot.readI64();
 -                _key410.add(_elem414);
++                _elem432 = iprot.readI64();
++                _key428.add(_elem432);
                }
              }
-             _val421 = iprot.readI64();
-             struct.executor_start_time_secs.put(_key420, _val421);
 -            _val411 = iprot.readI64();
 -            struct.executor_start_time_secs.put(_key410, _val411);
++            _val429 = iprot.readI64();
++            struct.executor_start_time_secs.put(_key428, _val429);
            }
          }
          struct.set_executor_start_time_secs_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/BoltStats.java
index 1f8a989,1f8a989..26ef5d8
--- a/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
--@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._Fields>, java.io.Serializable, Cloneable, Comparable<BoltStats> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltStats");
  
@@@ -881,41 -881,41 +881,8 @@@
            case 1: // ACKED
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
--                org.apache.thrift.protocol.TMap _map90 = iprot.readMapBegin();
--                struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map90.size);
--                String _key91;
--                Map<GlobalStreamId,Long> _val92;
--                for (int _i93 = 0; _i93 < _map90.size; ++_i93)
--                {
--                  _key91 = iprot.readString();
--                  {
--                    org.apache.thrift.protocol.TMap _map94 = iprot.readMapBegin();
--                    _val92 = new HashMap<GlobalStreamId,Long>(2*_map94.size);
--                    GlobalStreamId _key95;
--                    long _val96;
--                    for (int _i97 = 0; _i97 < _map94.size; ++_i97)
--                    {
--                      _key95 = new GlobalStreamId();
--                      _key95.read(iprot);
--                      _val96 = iprot.readI64();
--                      _val92.put(_key95, _val96);
--                    }
--                    iprot.readMapEnd();
--                  }
--                  struct.acked.put(_key91, _val92);
--                }
--                iprot.readMapEnd();
--              }
--              struct.set_acked_isSet(true);
--            } else { 
--              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
--            }
--            break;
--          case 2: // FAILED
--            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
--              {
                  org.apache.thrift.protocol.TMap _map98 = iprot.readMapBegin();
--                struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map98.size);
++                struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map98.size);
                  String _key99;
                  Map<GlobalStreamId,Long> _val100;
                  for (int _i101 = 0; _i101 < _map98.size; ++_i101)
@@@ -935,106 -935,106 +902,139 @@@
                      }
                      iprot.readMapEnd();
                    }
--                  struct.failed.put(_key99, _val100);
++                  struct.acked.put(_key99, _val100);
                  }
                  iprot.readMapEnd();
                }
--              struct.set_failed_isSet(true);
++              struct.set_acked_isSet(true);
              } else { 
                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
              }
              break;
--          case 3: // PROCESS_MS_AVG
++          case 2: // FAILED
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
                  org.apache.thrift.protocol.TMap _map106 = iprot.readMapBegin();
--                struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map106.size);
++                struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map106.size);
                  String _key107;
--                Map<GlobalStreamId,Double> _val108;
++                Map<GlobalStreamId,Long> _val108;
                  for (int _i109 = 0; _i109 < _map106.size; ++_i109)
                  {
                    _key107 = iprot.readString();
                    {
                      org.apache.thrift.protocol.TMap _map110 = iprot.readMapBegin();
--                    _val108 = new HashMap<GlobalStreamId,Double>(2*_map110.size);
++                    _val108 = new HashMap<GlobalStreamId,Long>(2*_map110.size);
                      GlobalStreamId _key111;
--                    double _val112;
++                    long _val112;
                      for (int _i113 = 0; _i113 < _map110.size; ++_i113)
                      {
                        _key111 = new GlobalStreamId();
                        _key111.read(iprot);
--                      _val112 = iprot.readDouble();
++                      _val112 = iprot.readI64();
                        _val108.put(_key111, _val112);
                      }
                      iprot.readMapEnd();
                    }
--                  struct.process_ms_avg.put(_key107, _val108);
++                  struct.failed.put(_key107, _val108);
                  }
                  iprot.readMapEnd();
                }
--              struct.set_process_ms_avg_isSet(true);
++              struct.set_failed_isSet(true);
              } else { 
                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
              }
              break;
--          case 4: // EXECUTED
++          case 3: // PROCESS_MS_AVG
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
                  org.apache.thrift.protocol.TMap _map114 = iprot.readMapBegin();
--                struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map114.size);
++                struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map114.size);
                  String _key115;
--                Map<GlobalStreamId,Long> _val116;
++                Map<GlobalStreamId,Double> _val116;
                  for (int _i117 = 0; _i117 < _map114.size; ++_i117)
                  {
                    _key115 = iprot.readString();
                    {
                      org.apache.thrift.protocol.TMap _map118 = iprot.readMapBegin();
--                    _val116 = new HashMap<GlobalStreamId,Long>(2*_map118.size);
++                    _val116 = new HashMap<GlobalStreamId,Double>(2*_map118.size);
                      GlobalStreamId _key119;
--                    long _val120;
++                    double _val120;
                      for (int _i121 = 0; _i121 < _map118.size; ++_i121)
                      {
                        _key119 = new GlobalStreamId();
                        _key119.read(iprot);
--                      _val120 = iprot.readI64();
++                      _val120 = iprot.readDouble();
                        _val116.put(_key119, _val120);
                      }
                      iprot.readMapEnd();
                    }
--                  struct.executed.put(_key115, _val116);
++                  struct.process_ms_avg.put(_key115, _val116);
                  }
                  iprot.readMapEnd();
                }
--              struct.set_executed_isSet(true);
++              struct.set_process_ms_avg_isSet(true);
              } else { 
                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
              }
              break;
--          case 5: // EXECUTE_MS_AVG
++          case 4: // EXECUTED
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
                  org.apache.thrift.protocol.TMap _map122 = iprot.readMapBegin();
--                struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map122.size);
++                struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map122.size);
                  String _key123;
--                Map<GlobalStreamId,Double> _val124;
++                Map<GlobalStreamId,Long> _val124;
                  for (int _i125 = 0; _i125 < _map122.size; ++_i125)
                  {
                    _key123 = iprot.readString();
                    {
                      org.apache.thrift.protocol.TMap _map126 = iprot.readMapBegin();
--                    _val124 = new HashMap<GlobalStreamId,Double>(2*_map126.size);
++                    _val124 = new HashMap<GlobalStreamId,Long>(2*_map126.size);
                      GlobalStreamId _key127;
--                    double _val128;
++                    long _val128;
                      for (int _i129 = 0; _i129 < _map126.size; ++_i129)
                      {
                        _key127 = new GlobalStreamId();
                        _key127.read(iprot);
--                      _val128 = iprot.readDouble();
++                      _val128 = iprot.readI64();
                        _val124.put(_key127, _val128);
                      }
                      iprot.readMapEnd();
                    }
--                  struct.execute_ms_avg.put(_key123, _val124);
++                  struct.executed.put(_key123, _val124);
++                }
++                iprot.readMapEnd();
++              }
++              struct.set_executed_isSet(true);
++            } else { 
++              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
++            }
++            break;
++          case 5: // EXECUTE_MS_AVG
++            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
++              {
++                org.apache.thrift.protocol.TMap _map130 = iprot.readMapBegin();
++                struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map130.size);
++                String _key131;
++                Map<GlobalStreamId,Double> _val132;
++                for (int _i133 = 0; _i133 < _map130.size; ++_i133)
++                {
++                  _key131 = iprot.readString();
++                  {
++                    org.apache.thrift.protocol.TMap _map134 = iprot.readMapBegin();
++                    _val132 = new HashMap<GlobalStreamId,Double>(2*_map134.size);
++                    GlobalStreamId _key135;
++                    double _val136;
++                    for (int _i137 = 0; _i137 < _map134.size; ++_i137)
++                    {
++                      _key135 = new GlobalStreamId();
++                      _key135.read(iprot);
++                      _val136 = iprot.readDouble();
++                      _val132.put(_key135, _val136);
++                    }
++                    iprot.readMapEnd();
++                  }
++                  struct.execute_ms_avg.put(_key131, _val132);
                  }
                  iprot.readMapEnd();
                }
@@@ -1060,15 -1060,15 +1060,15 @@@
          oprot.writeFieldBegin(ACKED_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
--          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter130 : struct.acked.entrySet())
++          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter138 : struct.acked.entrySet())
            {
--            oprot.writeString(_iter130.getKey());
++            oprot.writeString(_iter138.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter130.getValue().size()));
--              for (Map.Entry<GlobalStreamId, Long> _iter131 : _iter130.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter138.getValue().size()));
++              for (Map.Entry<GlobalStreamId, Long> _iter139 : _iter138.getValue().entrySet())
                {
--                _iter131.getKey().write(oprot);
--                oprot.writeI64(_iter131.getValue());
++                _iter139.getKey().write(oprot);
++                oprot.writeI64(_iter139.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -1081,15 -1081,15 +1081,15 @@@
          oprot.writeFieldBegin(FAILED_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
--          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter132 : struct.failed.entrySet())
++          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter140 : struct.failed.entrySet())
            {
--            oprot.writeString(_iter132.getKey());
++            oprot.writeString(_iter140.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter132.getValue().size()));
--              for (Map.Entry<GlobalStreamId, Long> _iter133 : _iter132.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter140.getValue().size()));
++              for (Map.Entry<GlobalStreamId, Long> _iter141 : _iter140.getValue().entrySet())
                {
--                _iter133.getKey().write(oprot);
--                oprot.writeI64(_iter133.getValue());
++                _iter141.getKey().write(oprot);
++                oprot.writeI64(_iter141.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -1102,15 -1102,15 +1102,15 @@@
          oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.process_ms_avg.size()));
--          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter134 : struct.process_ms_avg.entrySet())
++          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter142 : struct.process_ms_avg.entrySet())
            {
--            oprot.writeString(_iter134.getKey());
++            oprot.writeString(_iter142.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter134.getValue().size()));
--              for (Map.Entry<GlobalStreamId, Double> _iter135 : _iter134.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter142.getValue().size()));
++              for (Map.Entry<GlobalStreamId, Double> _iter143 : _iter142.getValue().entrySet())
                {
--                _iter135.getKey().write(oprot);
--                oprot.writeDouble(_iter135.getValue());
++                _iter143.getKey().write(oprot);
++                oprot.writeDouble(_iter143.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -1123,15 -1123,15 +1123,15 @@@
          oprot.writeFieldBegin(EXECUTED_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.executed.size()));
--          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter136 : struct.executed.entrySet())
++          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter144 : struct.executed.entrySet())
            {
--            oprot.writeString(_iter136.getKey());
++            oprot.writeString(_iter144.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter136.getValue().size()));
--              for (Map.Entry<GlobalStreamId, Long> _iter137 : _iter136.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter144.getValue().size()));
++              for (Map.Entry<GlobalStreamId, Long> _iter145 : _iter144.getValue().entrySet())
                {
--                _iter137.getKey().write(oprot);
--                oprot.writeI64(_iter137.getValue());
++                _iter145.getKey().write(oprot);
++                oprot.writeI64(_iter145.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -1144,15 -1144,15 +1144,15 @@@
          oprot.writeFieldBegin(EXECUTE_MS_AVG_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.execute_ms_avg.size()));
--          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter138 : struct.execute_ms_avg.entrySet())
++          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter146 : struct.execute_ms_avg.entrySet())
            {
--            oprot.writeString(_iter138.getKey());
++            oprot.writeString(_iter146.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter138.getValue().size()));
--              for (Map.Entry<GlobalStreamId, Double> _iter139 : _iter138.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter146.getValue().size()));
++              for (Map.Entry<GlobalStreamId, Double> _iter147 : _iter146.getValue().entrySet())
                {
--                _iter139.getKey().write(oprot);
--                oprot.writeDouble(_iter139.getValue());
++                _iter147.getKey().write(oprot);
++                oprot.writeDouble(_iter147.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -1180,75 -1180,75 +1180,75 @@@
        TTupleProtocol oprot = (TTupleProtocol) prot;
        {
          oprot.writeI32(struct.acked.size());
--        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter140 : struct.acked.entrySet())
++        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter148 : struct.acked.entrySet())
          {
--          oprot.writeString(_iter140.getKey());
++          oprot.writeString(_iter148.getKey());
            {
--            oprot.writeI32(_iter140.getValue().size());
--            for (Map.Entry<GlobalStreamId, Long> _iter141 : _iter140.getValue().entrySet())
++            oprot.writeI32(_iter148.getValue().size());
++            for (Map.Entry<GlobalStreamId, Long> _iter149 : _iter148.getValue().entrySet())
              {
--              _iter141.getKey().write(oprot);
--              oprot.writeI64(_iter141.getValue());
++              _iter149.getKey().write(oprot);
++              oprot.writeI64(_iter149.getValue());
              }
            }
          }
        }
        {
          oprot.writeI32(struct.failed.size());
--        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter142 : struct.failed.entrySet())
++        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter150 : struct.failed.entrySet())
          {
--          oprot.writeString(_iter142.getKey());
++          oprot.writeString(_iter150.getKey());
            {
--            oprot.writeI32(_iter142.getValue().size());
--            for (Map.Entry<GlobalStreamId, Long> _iter143 : _iter142.getValue().entrySet())
++            oprot.writeI32(_iter150.getValue().size());
++            for (Map.Entry<GlobalStreamId, Long> _iter151 : _iter150.getValue().entrySet())
              {
--              _iter143.getKey().write(oprot);
--              oprot.writeI64(_iter143.getValue());
++              _iter151.getKey().write(oprot);
++              oprot.writeI64(_iter151.getValue());
              }
            }
          }
        }
        {
          oprot.writeI32(struct.process_ms_avg.size());
--        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter144 : struct.process_ms_avg.entrySet())
++        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter152 : struct.process_ms_avg.entrySet())
          {
--          oprot.writeString(_iter144.getKey());
++          oprot.writeString(_iter152.getKey());
            {
--            oprot.writeI32(_iter144.getValue().size());
--            for (Map.Entry<GlobalStreamId, Double> _iter145 : _iter144.getValue().entrySet())
++            oprot.writeI32(_iter152.getValue().size());
++            for (Map.Entry<GlobalStreamId, Double> _iter153 : _iter152.getValue().entrySet())
              {
--              _iter145.getKey().write(oprot);
--              oprot.writeDouble(_iter145.getValue());
++              _iter153.getKey().write(oprot);
++              oprot.writeDouble(_iter153.getValue());
              }
            }
          }
        }
        {
          oprot.writeI32(struct.executed.size());
--        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter146 : struct.executed.entrySet())
++        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter154 : struct.executed.entrySet())
          {
--          oprot.writeString(_iter146.getKey());
++          oprot.writeString(_iter154.getKey());
            {
--            oprot.writeI32(_iter146.getValue().size());
--            for (Map.Entry<GlobalStreamId, Long> _iter147 : _iter146.getValue().entrySet())
++            oprot.writeI32(_iter154.getValue().size());
++            for (Map.Entry<GlobalStreamId, Long> _iter155 : _iter154.getValue().entrySet())
              {
--              _iter147.getKey().write(oprot);
--              oprot.writeI64(_iter147.getValue());
++              _iter155.getKey().write(oprot);
++              oprot.writeI64(_iter155.getValue());
              }
            }
          }
        }
        {
          oprot.writeI32(struct.execute_ms_avg.size());
--        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter148 : struct.execute_ms_avg.entrySet())
++        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter156 : struct.execute_ms_avg.entrySet())
          {
--          oprot.writeString(_iter148.getKey());
++          oprot.writeString(_iter156.getKey());
            {
--            oprot.writeI32(_iter148.getValue().size());
--            for (Map.Entry<GlobalStreamId, Double> _iter149 : _iter148.getValue().entrySet())
++            oprot.writeI32(_iter156.getValue().size());
++            for (Map.Entry<GlobalStreamId, Double> _iter157 : _iter156.getValue().entrySet())
              {
--              _iter149.getKey().write(oprot);
--              oprot.writeDouble(_iter149.getValue());
++              _iter157.getKey().write(oprot);
++              oprot.writeDouble(_iter157.getValue());
              }
            }
          }
@@@ -1259,33 -1259,33 +1259,8 @@@
      public void read(org.apache.thrift.protocol.TProtocol prot, BoltStats struct) throws org.apache.thrift.TException {
        TTupleProtocol iprot = (TTupleProtocol) prot;
        {
--        org.apache.thrift.protocol.TMap _map150 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map150.size);
--        String _key151;
--        Map<GlobalStreamId,Long> _val152;
--        for (int _i153 = 0; _i153 < _map150.size; ++_i153)
--        {
--          _key151 = iprot.readString();
--          {
--            org.apache.thrift.protocol.TMap _map154 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
--            _val152 = new HashMap<GlobalStreamId,Long>(2*_map154.size);
--            GlobalStreamId _key155;
--            long _val156;
--            for (int _i157 = 0; _i157 < _map154.size; ++_i157)
--            {
--              _key155 = new GlobalStreamId();
--              _key155.read(iprot);
--              _val156 = iprot.readI64();
--              _val152.put(_key155, _val156);
--            }
--          }
--          struct.acked.put(_key151, _val152);
--        }
--      }
--      struct.set_acked_isSet(true);
--      {
          org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map158.size);
++        struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map158.size);
          String _key159;
          Map<GlobalStreamId,Long> _val160;
          for (int _i161 = 0; _i161 < _map158.size; ++_i161)
@@@ -1304,82 -1304,82 +1279,107 @@@
                _val160.put(_key163, _val164);
              }
            }
--          struct.failed.put(_key159, _val160);
++          struct.acked.put(_key159, _val160);
          }
        }
--      struct.set_failed_isSet(true);
++      struct.set_acked_isSet(true);
        {
          org.apache.thrift.protocol.TMap _map166 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map166.size);
++        struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map166.size);
          String _key167;
--        Map<GlobalStreamId,Double> _val168;
++        Map<GlobalStreamId,Long> _val168;
          for (int _i169 = 0; _i169 < _map166.size; ++_i169)
          {
            _key167 = iprot.readString();
            {
--            org.apache.thrift.protocol.TMap _map170 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
--            _val168 = new HashMap<GlobalStreamId,Double>(2*_map170.size);
++            org.apache.thrift.protocol.TMap _map170 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
++            _val168 = new HashMap<GlobalStreamId,Long>(2*_map170.size);
              GlobalStreamId _key171;
--            double _val172;
++            long _val172;
              for (int _i173 = 0; _i173 < _map170.size; ++_i173)
              {
                _key171 = new GlobalStreamId();
                _key171.read(iprot);
--              _val172 = iprot.readDouble();
++              _val172 = iprot.readI64();
                _val168.put(_key171, _val172);
              }
            }
--          struct.process_ms_avg.put(_key167, _val168);
++          struct.failed.put(_key167, _val168);
          }
        }
--      struct.set_process_ms_avg_isSet(true);
++      struct.set_failed_isSet(true);
        {
          org.apache.thrift.protocol.TMap _map174 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map174.size);
++        struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map174.size);
          String _key175;
--        Map<GlobalStreamId,Long> _val176;
++        Map<GlobalStreamId,Double> _val176;
          for (int _i177 = 0; _i177 < _map174.size; ++_i177)
          {
            _key175 = iprot.readString();
            {
--            org.apache.thrift.protocol.TMap _map178 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
--            _val176 = new HashMap<GlobalStreamId,Long>(2*_map178.size);
++            org.apache.thrift.protocol.TMap _map178 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
++            _val176 = new HashMap<GlobalStreamId,Double>(2*_map178.size);
              GlobalStreamId _key179;
--            long _val180;
++            double _val180;
              for (int _i181 = 0; _i181 < _map178.size; ++_i181)
              {
                _key179 = new GlobalStreamId();
                _key179.read(iprot);
--              _val180 = iprot.readI64();
++              _val180 = iprot.readDouble();
                _val176.put(_key179, _val180);
              }
            }
--          struct.executed.put(_key175, _val176);
++          struct.process_ms_avg.put(_key175, _val176);
          }
        }
--      struct.set_executed_isSet(true);
++      struct.set_process_ms_avg_isSet(true);
        {
          org.apache.thrift.protocol.TMap _map182 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map182.size);
++        struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map182.size);
          String _key183;
--        Map<GlobalStreamId,Double> _val184;
++        Map<GlobalStreamId,Long> _val184;
          for (int _i185 = 0; _i185 < _map182.size; ++_i185)
          {
            _key183 = iprot.readString();
            {
--            org.apache.thrift.protocol.TMap _map186 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
--            _val184 = new HashMap<GlobalStreamId,Double>(2*_map186.size);
++            org.apache.thrift.protocol.TMap _map186 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
++            _val184 = new HashMap<GlobalStreamId,Long>(2*_map186.size);
              GlobalStreamId _key187;
--            double _val188;
++            long _val188;
              for (int _i189 = 0; _i189 < _map186.size; ++_i189)
              {
                _key187 = new GlobalStreamId();
                _key187.read(iprot);
--              _val188 = iprot.readDouble();
++              _val188 = iprot.readI64();
                _val184.put(_key187, _val188);
              }
            }
--          struct.execute_ms_avg.put(_key183, _val184);
++          struct.executed.put(_key183, _val184);
++        }
++      }
++      struct.set_executed_isSet(true);
++      {
++        org.apache.thrift.protocol.TMap _map190 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
++        struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map190.size);
++        String _key191;
++        Map<GlobalStreamId,Double> _val192;
++        for (int _i193 = 0; _i193 < _map190.size; ++_i193)
++        {
++          _key191 = iprot.readString();
++          {
++            org.apache.thrift.protocol.TMap _map194 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
++            _val192 = new HashMap<GlobalStreamId,Double>(2*_map194.size);
++            GlobalStreamId _key195;
++            double _val196;
++            for (int _i197 = 0; _i197 < _map194.size; ++_i197)
++            {
++              _key195 = new GlobalStreamId();
++              _key195.read(iprot);
++              _val196 = iprot.readDouble();
++              _val192.put(_key195, _val196);
++            }
++          }
++          struct.execute_ms_avg.put(_key191, _val192);
          }
        }
        struct.set_execute_ms_avg_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
index 424ff41,f60540b..ce2f5f4
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
@@@ -51,7 -51,8 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-8")
 -
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWorkerHeartbeat, ClusterWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterWorkerHeartbeat> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterWorkerHeartbeat");
  
@@@ -635,17 -636,17 +635,17 @@@
            case 2: // EXECUTOR_STATS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map446 = iprot.readMapBegin();
-                 struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map446.size);
-                 ExecutorInfo _key447;
-                 ExecutorStats _val448;
-                 for (int _i449 = 0; _i449 < _map446.size; ++_i449)
 -                org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
 -                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map426.size);
 -                ExecutorInfo _key427;
 -                ExecutorStats _val428;
 -                for (int _i429 = 0; _i429 < _map426.size; ++_i429)
++                org.apache.thrift.protocol.TMap _map454 = iprot.readMapBegin();
++                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map454.size);
++                ExecutorInfo _key455;
++                ExecutorStats _val456;
++                for (int _i457 = 0; _i457 < _map454.size; ++_i457)
                  {
-                   _key447 = new ExecutorInfo();
-                   _key447.read(iprot);
-                   _val448 = new ExecutorStats();
-                   _val448.read(iprot);
-                   struct.executor_stats.put(_key447, _val448);
 -                  _key427 = new ExecutorInfo();
 -                  _key427.read(iprot);
 -                  _val428 = new ExecutorStats();
 -                  _val428.read(iprot);
 -                  struct.executor_stats.put(_key427, _val428);
++                  _key455 = new ExecutorInfo();
++                  _key455.read(iprot);
++                  _val456 = new ExecutorStats();
++                  _val456.read(iprot);
++                  struct.executor_stats.put(_key455, _val456);
                  }
                  iprot.readMapEnd();
                }
@@@ -692,10 -693,10 +692,10 @@@
          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<ExecutorInfo, ExecutorStats> _iter450 : struct.executor_stats.entrySet())
 -          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter430 : struct.executor_stats.entrySet())
++          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter458 : struct.executor_stats.entrySet())
            {
-             _iter450.getKey().write(oprot);
-             _iter450.getValue().write(oprot);
 -            _iter430.getKey().write(oprot);
 -            _iter430.getValue().write(oprot);
++            _iter458.getKey().write(oprot);
++            _iter458.getValue().write(oprot);
            }
            oprot.writeMapEnd();
          }
@@@ -727,10 -728,10 +727,10 @@@
        oprot.writeString(struct.storm_id);
        {
          oprot.writeI32(struct.executor_stats.size());
-         for (Map.Entry<ExecutorInfo, ExecutorStats> _iter451 : struct.executor_stats.entrySet())
 -        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter431 : struct.executor_stats.entrySet())
++        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter459 : struct.executor_stats.entrySet())
          {
-           _iter451.getKey().write(oprot);
-           _iter451.getValue().write(oprot);
 -          _iter431.getKey().write(oprot);
 -          _iter431.getValue().write(oprot);
++          _iter459.getKey().write(oprot);
++          _iter459.getValue().write(oprot);
          }
        }
        oprot.writeI32(struct.time_secs);
@@@ -743,17 -744,17 +743,17 @@@
        struct.storm_id = iprot.readString();
        struct.set_storm_id_isSet(true);
        {
-         org.apache.thrift.protocol.TMap _map452 = 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<ExecutorInfo,ExecutorStats>(2*_map452.size);
-         ExecutorInfo _key453;
-         ExecutorStats _val454;
-         for (int _i455 = 0; _i455 < _map452.size; ++_i455)
 -        org.apache.thrift.protocol.TMap _map432 = 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<ExecutorInfo,ExecutorStats>(2*_map432.size);
 -        ExecutorInfo _key433;
 -        ExecutorStats _val434;
 -        for (int _i435 = 0; _i435 < _map432.size; ++_i435)
++        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<ExecutorInfo,ExecutorStats>(2*_map460.size);
++        ExecutorInfo _key461;
++        ExecutorStats _val462;
++        for (int _i463 = 0; _i463 < _map460.size; ++_i463)
          {
-           _key453 = new ExecutorInfo();
-           _key453.read(iprot);
-           _val454 = new ExecutorStats();
-           _val454.read(iprot);
-           struct.executor_stats.put(_key453, _val454);
 -          _key433 = new ExecutorInfo();
 -          _key433.read(iprot);
 -          _val434 = new ExecutorStats();
 -          _val434.read(iprot);
 -          struct.executor_stats.put(_key433, _val434);
++          _key461 = new ExecutorInfo();
++          _key461.read(iprot);
++          _val462 = new ExecutorStats();
++          _val462.read(iprot);
++          struct.executor_stats.put(_key461, _val462);
          }
        }
        struct.set_executor_stats_isSet(true);


[16/24] storm git commit: merging from upstream

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/StormBase.java
index f3c1fb2,b00f072..250cc9c
--- a/storm-core/src/jvm/backtype/storm/generated/StormBase.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
  
@@@ -1090,15 -984,15 +1090,15 @@@
            case 4: // COMPONENT_EXECUTORS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
-                 struct.component_executors = new HashMap<String,Integer>(2*_map426.size);
-                 String _key427;
-                 int _val428;
-                 for (int _i429 = 0; _i429 < _map426.size; ++_i429)
 -                org.apache.thrift.protocol.TMap _map416 = iprot.readMapBegin();
 -                struct.component_executors = new HashMap<String,Integer>(2*_map416.size);
 -                String _key417;
 -                int _val418;
 -                for (int _i419 = 0; _i419 < _map416.size; ++_i419)
++                org.apache.thrift.protocol.TMap _map434 = iprot.readMapBegin();
++                struct.component_executors = new HashMap<String,Integer>(2*_map434.size);
++                String _key435;
++                int _val436;
++                for (int _i437 = 0; _i437 < _map434.size; ++_i437)
                  {
-                   _key427 = iprot.readString();
-                   _val428 = iprot.readI32();
-                   struct.component_executors.put(_key427, _val428);
 -                  _key417 = iprot.readString();
 -                  _val418 = iprot.readI32();
 -                  struct.component_executors.put(_key417, _val418);
++                  _key435 = iprot.readString();
++                  _val436 = iprot.readI32();
++                  struct.component_executors.put(_key435, _val436);
                  }
                  iprot.readMapEnd();
                }
@@@ -1140,27 -1034,6 +1140,27 @@@
                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
              }
              break;
 +          case 9: // COMPONENT_DEBUG
 +            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
 +              {
-                 org.apache.thrift.protocol.TMap _map430 = iprot.readMapBegin();
-                 struct.component_debug = new HashMap<String,DebugOptions>(2*_map430.size);
-                 String _key431;
-                 DebugOptions _val432;
-                 for (int _i433 = 0; _i433 < _map430.size; ++_i433)
++                org.apache.thrift.protocol.TMap _map438 = iprot.readMapBegin();
++                struct.component_debug = new HashMap<String,DebugOptions>(2*_map438.size);
++                String _key439;
++                DebugOptions _val440;
++                for (int _i441 = 0; _i441 < _map438.size; ++_i441)
 +                {
-                   _key431 = iprot.readString();
-                   _val432 = new DebugOptions();
-                   _val432.read(iprot);
-                   struct.component_debug.put(_key431, _val432);
++                  _key439 = iprot.readString();
++                  _val440 = new DebugOptions();
++                  _val440.read(iprot);
++                  struct.component_debug.put(_key439, _val440);
 +                }
 +                iprot.readMapEnd();
 +              }
 +              struct.set_component_debug_isSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
            default:
              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
          }
@@@ -1192,10 -1065,10 +1192,10 @@@
            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<String, Integer> _iter434 : struct.component_executors.entrySet())
 -            for (Map.Entry<String, Integer> _iter420 : struct.component_executors.entrySet())
++            for (Map.Entry<String, Integer> _iter442 : struct.component_executors.entrySet())
              {
-               oprot.writeString(_iter434.getKey());
-               oprot.writeI32(_iter434.getValue());
 -              oprot.writeString(_iter420.getKey());
 -              oprot.writeI32(_iter420.getValue());
++              oprot.writeString(_iter442.getKey());
++              oprot.writeI32(_iter442.getValue());
              }
              oprot.writeMapEnd();
            }
@@@ -1228,21 -1101,6 +1228,21 @@@
            oprot.writeFieldEnd();
          }
        }
 +      if (struct.component_debug != null) {
 +        if (struct.is_set_component_debug()) {
 +          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<String, DebugOptions> _iter435 : struct.component_debug.entrySet())
++            for (Map.Entry<String, DebugOptions> _iter443 : struct.component_debug.entrySet())
 +            {
-               oprot.writeString(_iter435.getKey());
-               _iter435.getValue().write(oprot);
++              oprot.writeString(_iter443.getKey());
++              _iter443.getValue().write(oprot);
 +            }
 +            oprot.writeMapEnd();
 +          }
 +          oprot.writeFieldEnd();
 +        }
 +      }
        oprot.writeFieldStop();
        oprot.writeStructEnd();
      }
@@@ -1286,10 -1141,10 +1286,10 @@@
        if (struct.is_set_component_executors()) {
          {
            oprot.writeI32(struct.component_executors.size());
-           for (Map.Entry<String, Integer> _iter436 : struct.component_executors.entrySet())
 -          for (Map.Entry<String, Integer> _iter421 : struct.component_executors.entrySet())
++          for (Map.Entry<String, Integer> _iter444 : struct.component_executors.entrySet())
            {
-             oprot.writeString(_iter436.getKey());
-             oprot.writeI32(_iter436.getValue());
 -            oprot.writeString(_iter421.getKey());
 -            oprot.writeI32(_iter421.getValue());
++            oprot.writeString(_iter444.getKey());
++            oprot.writeI32(_iter444.getValue());
            }
          }
        }
@@@ -1305,16 -1160,6 +1305,16 @@@
        if (struct.is_set_prev_status()) {
          oprot.writeI32(struct.prev_status.getValue());
        }
 +      if (struct.is_set_component_debug()) {
 +        {
 +          oprot.writeI32(struct.component_debug.size());
-           for (Map.Entry<String, DebugOptions> _iter437 : struct.component_debug.entrySet())
++          for (Map.Entry<String, DebugOptions> _iter445 : struct.component_debug.entrySet())
 +          {
-             oprot.writeString(_iter437.getKey());
-             _iter437.getValue().write(oprot);
++            oprot.writeString(_iter445.getKey());
++            _iter445.getValue().write(oprot);
 +          }
 +        }
 +      }
      }
  
      @Override
@@@ -1326,18 -1171,18 +1326,18 @@@
        struct.set_status_isSet(true);
        struct.num_workers = iprot.readI32();
        struct.set_num_workers_isSet(true);
 -      BitSet incoming = iprot.readBitSet(5);
 +      BitSet incoming = iprot.readBitSet(6);
        if (incoming.get(0)) {
          {
-           org.apache.thrift.protocol.TMap _map438 = 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<String,Integer>(2*_map438.size);
-           String _key439;
-           int _val440;
-           for (int _i441 = 0; _i441 < _map438.size; ++_i441)
 -          org.apache.thrift.protocol.TMap _map422 = 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<String,Integer>(2*_map422.size);
 -          String _key423;
 -          int _val424;
 -          for (int _i425 = 0; _i425 < _map422.size; ++_i425)
++          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<String,Integer>(2*_map446.size);
++          String _key447;
++          int _val448;
++          for (int _i449 = 0; _i449 < _map446.size; ++_i449)
            {
-             _key439 = iprot.readString();
-             _val440 = iprot.readI32();
-             struct.component_executors.put(_key439, _val440);
 -            _key423 = iprot.readString();
 -            _val424 = iprot.readI32();
 -            struct.component_executors.put(_key423, _val424);
++            _key447 = iprot.readString();
++            _val448 = iprot.readI32();
++            struct.component_executors.put(_key447, _val448);
            }
          }
          struct.set_component_executors_isSet(true);
@@@ -1359,22 -1204,6 +1359,22 @@@
          struct.prev_status = backtype.storm.generated.TopologyStatus.findByValue(iprot.readI32());
          struct.set_prev_status_isSet(true);
        }
 +      if (incoming.get(5)) {
 +        {
-           org.apache.thrift.protocol.TMap _map442 = 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<String,DebugOptions>(2*_map442.size);
-           String _key443;
-           DebugOptions _val444;
-           for (int _i445 = 0; _i445 < _map442.size; ++_i445)
++          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<String,DebugOptions>(2*_map450.size);
++          String _key451;
++          DebugOptions _val452;
++          for (int _i453 = 0; _i453 < _map450.size; ++_i453)
 +          {
-             _key443 = iprot.readString();
-             _val444 = new DebugOptions();
-             _val444.read(iprot);
-             struct.component_debug.put(_key443, _val444);
++            _key451 = iprot.readString();
++            _val452 = new DebugOptions();
++            _val452.read(iprot);
++            struct.component_debug.put(_key451, _val452);
 +          }
 +        }
 +        struct.set_component_debug_isSet(true);
 +      }
      }
    }
  

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
index e2b5ad1,9083b0d..bb4698c
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-20")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorInfo> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo");
  
@@@ -990,13 -990,13 +990,13 @@@
            case 4: // USED_PORTS
              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                {
-                 org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
-                 struct.used_ports = new ArrayList<Long>(_list346.size);
-                 long _elem347;
-                 for (int _i348 = 0; _i348 < _list346.size; ++_i348)
 -                org.apache.thrift.protocol.TList _list336 = iprot.readListBegin();
 -                struct.used_ports = new ArrayList<Long>(_list336.size);
 -                long _elem337;
 -                for (int _i338 = 0; _i338 < _list336.size; ++_i338)
++                org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
++                struct.used_ports = new ArrayList<Long>(_list354.size);
++                long _elem355;
++                for (int _i356 = 0; _i356 < _list354.size; ++_i356)
                  {
-                   _elem347 = iprot.readI64();
-                   struct.used_ports.add(_elem347);
 -                  _elem337 = iprot.readI64();
 -                  struct.used_ports.add(_elem337);
++                  _elem355 = iprot.readI64();
++                  struct.used_ports.add(_elem355);
                  }
                  iprot.readListEnd();
                }
@@@ -1008,13 -1008,13 +1008,13 @@@
            case 5: // META
              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                {
-                 org.apache.thrift.protocol.TList _list349 = iprot.readListBegin();
-                 struct.meta = new ArrayList<Long>(_list349.size);
-                 long _elem350;
-                 for (int _i351 = 0; _i351 < _list349.size; ++_i351)
 -                org.apache.thrift.protocol.TList _list339 = iprot.readListBegin();
 -                struct.meta = new ArrayList<Long>(_list339.size);
 -                long _elem340;
 -                for (int _i341 = 0; _i341 < _list339.size; ++_i341)
++                org.apache.thrift.protocol.TList _list357 = iprot.readListBegin();
++                struct.meta = new ArrayList<Long>(_list357.size);
++                long _elem358;
++                for (int _i359 = 0; _i359 < _list357.size; ++_i359)
                  {
-                   _elem350 = iprot.readI64();
-                   struct.meta.add(_elem350);
 -                  _elem340 = iprot.readI64();
 -                  struct.meta.add(_elem340);
++                  _elem358 = iprot.readI64();
++                  struct.meta.add(_elem358);
                  }
                  iprot.readListEnd();
                }
@@@ -1026,15 -1026,15 +1026,15 @@@
            case 6: // SCHEDULER_META
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map352 = iprot.readMapBegin();
-                 struct.scheduler_meta = new HashMap<String,String>(2*_map352.size);
-                 String _key353;
-                 String _val354;
-                 for (int _i355 = 0; _i355 < _map352.size; ++_i355)
 -                org.apache.thrift.protocol.TMap _map342 = iprot.readMapBegin();
 -                struct.scheduler_meta = new HashMap<String,String>(2*_map342.size);
 -                String _key343;
 -                String _val344;
 -                for (int _i345 = 0; _i345 < _map342.size; ++_i345)
++                org.apache.thrift.protocol.TMap _map360 = iprot.readMapBegin();
++                struct.scheduler_meta = new HashMap<String,String>(2*_map360.size);
++                String _key361;
++                String _val362;
++                for (int _i363 = 0; _i363 < _map360.size; ++_i363)
                  {
-                   _key353 = iprot.readString();
-                   _val354 = iprot.readString();
-                   struct.scheduler_meta.put(_key353, _val354);
 -                  _key343 = iprot.readString();
 -                  _val344 = iprot.readString();
 -                  struct.scheduler_meta.put(_key343, _val344);
++                  _key361 = iprot.readString();
++                  _val362 = iprot.readString();
++                  struct.scheduler_meta.put(_key361, _val362);
                  }
                  iprot.readMapEnd();
                }
@@@ -1092,9 -1092,9 +1092,9 @@@
            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 _iter356 : struct.used_ports)
 -            for (long _iter346 : struct.used_ports)
++            for (long _iter364 : struct.used_ports)
              {
-               oprot.writeI64(_iter356);
 -              oprot.writeI64(_iter346);
++              oprot.writeI64(_iter364);
              }
              oprot.writeListEnd();
            }
@@@ -1106,9 -1106,9 +1106,9 @@@
            oprot.writeFieldBegin(META_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size()));
-             for (long _iter357 : struct.meta)
 -            for (long _iter347 : struct.meta)
++            for (long _iter365 : struct.meta)
              {
-               oprot.writeI64(_iter357);
 -              oprot.writeI64(_iter347);
++              oprot.writeI64(_iter365);
              }
              oprot.writeListEnd();
            }
@@@ -1120,10 -1120,10 +1120,10 @@@
            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<String, String> _iter358 : struct.scheduler_meta.entrySet())
 -            for (Map.Entry<String, String> _iter348 : struct.scheduler_meta.entrySet())
++            for (Map.Entry<String, String> _iter366 : struct.scheduler_meta.entrySet())
              {
-               oprot.writeString(_iter358.getKey());
-               oprot.writeString(_iter358.getValue());
 -              oprot.writeString(_iter348.getKey());
 -              oprot.writeString(_iter348.getValue());
++              oprot.writeString(_iter366.getKey());
++              oprot.writeString(_iter366.getValue());
              }
              oprot.writeMapEnd();
            }
@@@ -1187,28 -1187,28 +1187,28 @@@
        if (struct.is_set_used_ports()) {
          {
            oprot.writeI32(struct.used_ports.size());
-           for (long _iter359 : struct.used_ports)
 -          for (long _iter349 : struct.used_ports)
++          for (long _iter367 : struct.used_ports)
            {
-             oprot.writeI64(_iter359);
 -            oprot.writeI64(_iter349);
++            oprot.writeI64(_iter367);
            }
          }
        }
        if (struct.is_set_meta()) {
          {
            oprot.writeI32(struct.meta.size());
-           for (long _iter360 : struct.meta)
 -          for (long _iter350 : struct.meta)
++          for (long _iter368 : struct.meta)
            {
-             oprot.writeI64(_iter360);
 -            oprot.writeI64(_iter350);
++            oprot.writeI64(_iter368);
            }
          }
        }
        if (struct.is_set_scheduler_meta()) {
          {
            oprot.writeI32(struct.scheduler_meta.size());
-           for (Map.Entry<String, String> _iter361 : struct.scheduler_meta.entrySet())
 -          for (Map.Entry<String, String> _iter351 : struct.scheduler_meta.entrySet())
++          for (Map.Entry<String, String> _iter369 : struct.scheduler_meta.entrySet())
            {
-             oprot.writeString(_iter361.getKey());
-             oprot.writeString(_iter361.getValue());
 -            oprot.writeString(_iter351.getKey());
 -            oprot.writeString(_iter351.getValue());
++            oprot.writeString(_iter369.getKey());
++            oprot.writeString(_iter369.getValue());
            }
          }
        }
@@@ -1234,41 -1234,41 +1234,41 @@@
        }
        if (incoming.get(1)) {
          {
-           org.apache.thrift.protocol.TList _list362 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-           struct.used_ports = new ArrayList<Long>(_list362.size);
-           long _elem363;
-           for (int _i364 = 0; _i364 < _list362.size; ++_i364)
 -          org.apache.thrift.protocol.TList _list352 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
 -          struct.used_ports = new ArrayList<Long>(_list352.size);
 -          long _elem353;
 -          for (int _i354 = 0; _i354 < _list352.size; ++_i354)
++          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<Long>(_list370.size);
++          long _elem371;
++          for (int _i372 = 0; _i372 < _list370.size; ++_i372)
            {
-             _elem363 = iprot.readI64();
-             struct.used_ports.add(_elem363);
 -            _elem353 = iprot.readI64();
 -            struct.used_ports.add(_elem353);
++            _elem371 = iprot.readI64();
++            struct.used_ports.add(_elem371);
            }
          }
          struct.set_used_ports_isSet(true);
        }
        if (incoming.get(2)) {
          {
-           org.apache.thrift.protocol.TList _list365 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-           struct.meta = new ArrayList<Long>(_list365.size);
-           long _elem366;
-           for (int _i367 = 0; _i367 < _list365.size; ++_i367)
 -          org.apache.thrift.protocol.TList _list355 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
 -          struct.meta = new ArrayList<Long>(_list355.size);
 -          long _elem356;
 -          for (int _i357 = 0; _i357 < _list355.size; ++_i357)
++          org.apache.thrift.protocol.TList _list373 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
++          struct.meta = new ArrayList<Long>(_list373.size);
++          long _elem374;
++          for (int _i375 = 0; _i375 < _list373.size; ++_i375)
            {
-             _elem366 = iprot.readI64();
-             struct.meta.add(_elem366);
 -            _elem356 = iprot.readI64();
 -            struct.meta.add(_elem356);
++            _elem374 = iprot.readI64();
++            struct.meta.add(_elem374);
            }
          }
          struct.set_meta_isSet(true);
        }
        if (incoming.get(3)) {
          {
-           org.apache.thrift.protocol.TMap _map368 = 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<String,String>(2*_map368.size);
-           String _key369;
-           String _val370;
-           for (int _i371 = 0; _i371 < _map368.size; ++_i371)
 -          org.apache.thrift.protocol.TMap _map358 = 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<String,String>(2*_map358.size);
 -          String _key359;
 -          String _val360;
 -          for (int _i361 = 0; _i361 < _map358.size; ++_i361)
++          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<String,String>(2*_map376.size);
++          String _key377;
++          String _val378;
++          for (int _i379 = 0; _i379 < _map376.size; ++_i379)
            {
-             _key369 = iprot.readString();
-             _val370 = iprot.readString();
-             struct.scheduler_meta.put(_key369, _val370);
 -            _key359 = iprot.readString();
 -            _val360 = iprot.readString();
 -            struct.scheduler_meta.put(_key359, _val360);
++            _key377 = iprot.readString();
++            _val378 = iprot.readString();
++            struct.scheduler_meta.put(_key377, _val378);
            }
          }
          struct.set_scheduler_meta_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index f0107af,37c5c63..6c2cc69
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-2")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
  
@@@ -61,9 -61,9 +61,10 @@@
    private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)4);
    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)5);
    private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.MAP, (short)6);
 +  private static final org.apache.thrift.protocol.TField COMPONENT_DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("component_debug", org.apache.thrift.protocol.TType.MAP, (short)7);
    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)513);
    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)514);
+   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)515);
  
    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
    static {
@@@ -77,9 -77,9 +78,10 @@@
    private List<ExecutorSummary> executors; // required
    private String status; // required
    private Map<String,List<ErrorInfo>> errors; // required
 +  private Map<String,DebugOptions> component_debug; // optional
    private String sched_status; // optional
    private String owner; // 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 {
@@@ -89,9 -89,9 +91,10 @@@
      EXECUTORS((short)4, "executors"),
      STATUS((short)5, "status"),
      ERRORS((short)6, "errors"),
 +    COMPONENT_DEBUG((short)7, "component_debug"),
      SCHED_STATUS((short)513, "sched_status"),
-     OWNER((short)514, "owner");
+     OWNER((short)514, "owner"),
+     REPLICATION_COUNT((short)515, "replication_count");
  
      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
  
@@@ -165,8 -165,9 +170,9 @@@
  
    // isset id assignments
    private static final int __UPTIME_SECS_ISSET_ID = 0;
+   private static final int __REPLICATION_COUNT_ISSET_ID = 1;
    private byte __isset_bitfield = 0;
-   private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER};
 -  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT};
++  private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER,_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);
@@@ -295,9 -280,10 +304,11 @@@
      this.executors = null;
      this.status = null;
      this.errors = null;
 +    this.component_debug = null;
      this.sched_status = null;
      this.owner = null;
+     set_replication_count_isSet(false);
+     this.replication_count = 0;
    }
  
    public String get_id() {
@@@ -1173,27 -1143,6 +1249,27 @@@
                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
              }
              break;
 +          case 7: // COMPONENT_DEBUG
 +            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
 +              {
-                 org.apache.thrift.protocol.TMap _map300 = iprot.readMapBegin();
-                 struct.component_debug = new HashMap<String,DebugOptions>(2*_map300.size);
-                 String _key301;
-                 DebugOptions _val302;
-                 for (int _i303 = 0; _i303 < _map300.size; ++_i303)
++                org.apache.thrift.protocol.TMap _map308 = iprot.readMapBegin();
++                struct.component_debug = new HashMap<String,DebugOptions>(2*_map308.size);
++                String _key309;
++                DebugOptions _val310;
++                for (int _i311 = 0; _i311 < _map308.size; ++_i311)
 +                {
-                   _key301 = iprot.readString();
-                   _val302 = new DebugOptions();
-                   _val302.read(iprot);
-                   struct.component_debug.put(_key301, _val302);
++                  _key309 = iprot.readString();
++                  _val310 = new DebugOptions();
++                  _val310.read(iprot);
++                  struct.component_debug.put(_key309, _val310);
 +                }
 +                iprot.readMapEnd();
 +              }
 +              struct.set_component_debug_isSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
            case 513: // SCHED_STATUS
              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
                struct.sched_status = iprot.readString();
@@@ -1240,9 -1197,9 +1324,9 @@@
          oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
          {
            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-           for (ExecutorSummary _iter304 : struct.executors)
 -          for (ExecutorSummary _iter308 : struct.executors)
++          for (ExecutorSummary _iter312 : struct.executors)
            {
-             _iter304.write(oprot);
 -            _iter308.write(oprot);
++            _iter312.write(oprot);
            }
            oprot.writeListEnd();
          }
@@@ -1257,14 -1214,14 +1341,14 @@@
          oprot.writeFieldBegin(ERRORS_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.errors.size()));
-           for (Map.Entry<String, List<ErrorInfo>> _iter305 : struct.errors.entrySet())
 -          for (Map.Entry<String, List<ErrorInfo>> _iter309 : struct.errors.entrySet())
++          for (Map.Entry<String, List<ErrorInfo>> _iter313 : struct.errors.entrySet())
            {
-             oprot.writeString(_iter305.getKey());
 -            oprot.writeString(_iter309.getKey());
++            oprot.writeString(_iter313.getKey());
              {
-               oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter305.getValue().size()));
-               for (ErrorInfo _iter306 : _iter305.getValue())
 -              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter309.getValue().size()));
 -              for (ErrorInfo _iter310 : _iter309.getValue())
++              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter313.getValue().size()));
++              for (ErrorInfo _iter314 : _iter313.getValue())
                {
-                 _iter306.write(oprot);
 -                _iter310.write(oprot);
++                _iter314.write(oprot);
                }
                oprot.writeListEnd();
              }
@@@ -1273,21 -1230,6 +1357,21 @@@
          }
          oprot.writeFieldEnd();
        }
 +      if (struct.component_debug != null) {
 +        if (struct.is_set_component_debug()) {
 +          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<String, DebugOptions> _iter307 : struct.component_debug.entrySet())
++            for (Map.Entry<String, DebugOptions> _iter315 : struct.component_debug.entrySet())
 +            {
-               oprot.writeString(_iter307.getKey());
-               _iter307.getValue().write(oprot);
++              oprot.writeString(_iter315.getKey());
++              _iter315.getValue().write(oprot);
 +            }
 +            oprot.writeMapEnd();
 +          }
 +          oprot.writeFieldEnd();
 +        }
 +      }
        if (struct.sched_status != null) {
          if (struct.is_set_sched_status()) {
            oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
@@@ -1324,47 -1271,37 +1413,50 @@@
        oprot.writeI32(struct.uptime_secs);
        {
          oprot.writeI32(struct.executors.size());
-         for (ExecutorSummary _iter308 : struct.executors)
 -        for (ExecutorSummary _iter311 : struct.executors)
++        for (ExecutorSummary _iter316 : struct.executors)
          {
-           _iter308.write(oprot);
 -          _iter311.write(oprot);
++          _iter316.write(oprot);
          }
        }
        oprot.writeString(struct.status);
        {
          oprot.writeI32(struct.errors.size());
-         for (Map.Entry<String, List<ErrorInfo>> _iter309 : struct.errors.entrySet())
 -        for (Map.Entry<String, List<ErrorInfo>> _iter312 : struct.errors.entrySet())
++        for (Map.Entry<String, List<ErrorInfo>> _iter317 : struct.errors.entrySet())
          {
-           oprot.writeString(_iter309.getKey());
 -          oprot.writeString(_iter312.getKey());
++          oprot.writeString(_iter317.getKey());
            {
-             oprot.writeI32(_iter309.getValue().size());
-             for (ErrorInfo _iter310 : _iter309.getValue())
 -            oprot.writeI32(_iter312.getValue().size());
 -            for (ErrorInfo _iter313 : _iter312.getValue())
++            oprot.writeI32(_iter317.getValue().size());
++            for (ErrorInfo _iter318 : _iter317.getValue())
              {
-               _iter310.write(oprot);
 -              _iter313.write(oprot);
++              _iter318.write(oprot);
              }
            }
          }
        }
        BitSet optionals = new BitSet();
 -      if (struct.is_set_sched_status()) {
 +      if (struct.is_set_component_debug()) {
          optionals.set(0);
        }
 -      if (struct.is_set_owner()) {
 +      if (struct.is_set_sched_status()) {
          optionals.set(1);
        }
 -      if (struct.is_set_replication_count()) {
 +      if (struct.is_set_owner()) {
          optionals.set(2);
        }
--      oprot.writeBitSet(optionals, 3);
++      if (struct.is_set_replication_count()) {
++        optionals.set(3);
++      }
++      oprot.writeBitSet(optionals, 4);
 +      if (struct.is_set_component_debug()) {
 +        {
 +          oprot.writeI32(struct.component_debug.size());
-           for (Map.Entry<String, DebugOptions> _iter311 : struct.component_debug.entrySet())
++          for (Map.Entry<String, DebugOptions> _iter319 : struct.component_debug.entrySet())
 +          {
-             oprot.writeString(_iter311.getKey());
-             _iter311.getValue().write(oprot);
++            oprot.writeString(_iter319.getKey());
++            _iter319.getValue().write(oprot);
 +          }
 +        }
 +      }
        if (struct.is_set_sched_status()) {
          oprot.writeString(struct.sched_status);
        }
@@@ -1383,60 -1323,44 +1478,60 @@@
        struct.uptime_secs = iprot.readI32();
        struct.set_uptime_secs_isSet(true);
        {
-         org.apache.thrift.protocol.TList _list312 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-         struct.executors = new ArrayList<ExecutorSummary>(_list312.size);
-         ExecutorSummary _elem313;
-         for (int _i314 = 0; _i314 < _list312.size; ++_i314)
 -        org.apache.thrift.protocol.TList _list314 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -        struct.executors = new ArrayList<ExecutorSummary>(_list314.size);
 -        ExecutorSummary _elem315;
 -        for (int _i316 = 0; _i316 < _list314.size; ++_i316)
++        org.apache.thrift.protocol.TList _list320 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++        struct.executors = new ArrayList<ExecutorSummary>(_list320.size);
++        ExecutorSummary _elem321;
++        for (int _i322 = 0; _i322 < _list320.size; ++_i322)
          {
-           _elem313 = new ExecutorSummary();
-           _elem313.read(iprot);
-           struct.executors.add(_elem313);
 -          _elem315 = new ExecutorSummary();
 -          _elem315.read(iprot);
 -          struct.executors.add(_elem315);
++          _elem321 = new ExecutorSummary();
++          _elem321.read(iprot);
++          struct.executors.add(_elem321);
          }
        }
        struct.set_executors_isSet(true);
        struct.status = iprot.readString();
        struct.set_status_isSet(true);
        {
-         org.apache.thrift.protocol.TMap _map315 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-         struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map315.size);
-         String _key316;
-         List<ErrorInfo> _val317;
-         for (int _i318 = 0; _i318 < _map315.size; ++_i318)
 -        org.apache.thrift.protocol.TMap _map317 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
 -        struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map317.size);
 -        String _key318;
 -        List<ErrorInfo> _val319;
 -        for (int _i320 = 0; _i320 < _map317.size; ++_i320)
++        org.apache.thrift.protocol.TMap _map323 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
++        struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map323.size);
++        String _key324;
++        List<ErrorInfo> _val325;
++        for (int _i326 = 0; _i326 < _map323.size; ++_i326)
          {
-           _key316 = iprot.readString();
 -          _key318 = iprot.readString();
++          _key324 = iprot.readString();
            {
-             org.apache.thrift.protocol.TList _list319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             _val317 = new ArrayList<ErrorInfo>(_list319.size);
-             ErrorInfo _elem320;
-             for (int _i321 = 0; _i321 < _list319.size; ++_i321)
 -            org.apache.thrift.protocol.TList _list321 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            _val319 = new ArrayList<ErrorInfo>(_list321.size);
 -            ErrorInfo _elem322;
 -            for (int _i323 = 0; _i323 < _list321.size; ++_i323)
++            org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            _val325 = new ArrayList<ErrorInfo>(_list327.size);
++            ErrorInfo _elem328;
++            for (int _i329 = 0; _i329 < _list327.size; ++_i329)
              {
-               _elem320 = new ErrorInfo();
-               _elem320.read(iprot);
-               _val317.add(_elem320);
 -              _elem322 = new ErrorInfo();
 -              _elem322.read(iprot);
 -              _val319.add(_elem322);
++              _elem328 = new ErrorInfo();
++              _elem328.read(iprot);
++              _val325.add(_elem328);
              }
            }
-           struct.errors.put(_key316, _val317);
 -          struct.errors.put(_key318, _val319);
++          struct.errors.put(_key324, _val325);
          }
        }
        struct.set_errors_isSet(true);
--      BitSet incoming = iprot.readBitSet(3);
++      BitSet incoming = iprot.readBitSet(4);
        if (incoming.get(0)) {
 +        {
-           org.apache.thrift.protocol.TMap _map322 = 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<String,DebugOptions>(2*_map322.size);
-           String _key323;
-           DebugOptions _val324;
-           for (int _i325 = 0; _i325 < _map322.size; ++_i325)
++          org.apache.thrift.protocol.TMap _map330 = 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<String,DebugOptions>(2*_map330.size);
++          String _key331;
++          DebugOptions _val332;
++          for (int _i333 = 0; _i333 < _map330.size; ++_i333)
 +          {
-             _key323 = iprot.readString();
-             _val324 = new DebugOptions();
-             _val324.read(iprot);
-             struct.component_debug.put(_key323, _val324);
++            _key331 = iprot.readString();
++            _val332 = new DebugOptions();
++            _val332.read(iprot);
++            struct.component_debug.put(_key331, _val332);
 +          }
 +        }
 +        struct.set_component_debug_isSet(true);
 +      }
 +      if (incoming.get(1)) {
          struct.sched_status = iprot.readString();
          struct.set_sched_status_isSet(true);
        }
@@@ -1444,6 -1368,10 +1539,10 @@@
          struct.owner = iprot.readString();
          struct.set_owner_isSet(true);
        }
 -      if (incoming.get(2)) {
++      if (incoming.get(3)) {
+         struct.replication_count = iprot.readI32();
+         struct.set_replication_count_isSet(true);
+       }
      }
    }
  

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --cc storm-core/src/py/storm/ttypes.py
index 9b77461,85dd2c7..4f48449
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@@ -3566,9 -3718,9 +3716,10 @@@ class TopologyInfo
     - executors
     - status
     - errors
 +   - component_debug
     - sched_status
     - owner
+    - replication_count
    """
  
    thrift_spec = (
@@@ -4087,18 -4239,19 +4238,20 @@@
      None, # 512
      (513, TType.STRING, 'sched_status', None, None, ), # 513
      (514, TType.STRING, 'owner', None, None, ), # 514
+     (515, TType.I32, 'replication_count', None, None, ), # 515
    )
  
-   def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, component_debug=None, sched_status=None, owner=None,):
 -  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, sched_status=None, owner=None, replication_count=None,):
++  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, component_debug=None, sched_status=None, owner=None, replication_count=None,):
      self.id = id
      self.name = name
      self.uptime_secs = uptime_secs
      self.executors = executors
      self.status = status
      self.errors = errors
 +    self.component_debug = component_debug
      self.sched_status = sched_status
      self.owner = owner
+     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:
@@@ -4157,18 -4310,6 +4310,18 @@@
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
 +      elif fid == 7:
 +        if ftype == TType.MAP:
 +          self.component_debug = {}
-           (_ktype280, _vtype281, _size279 ) = iprot.readMapBegin()
-           for _i283 in xrange(_size279):
-             _key284 = iprot.readString().decode('utf-8')
-             _val285 = DebugOptions()
-             _val285.read(iprot)
-             self.component_debug[_key284] = _val285
++          (_ktype287, _vtype288, _size286 ) = iprot.readMapBegin()
++          for _i290 in xrange(_size286):
++            _key291 = iprot.readString().decode('utf-8')
++            _val292 = DebugOptions()
++            _val292.read(iprot)
++            self.component_debug[_key291] = _val292
 +          iprot.readMapEnd()
 +        else:
 +          iprot.skip(ftype)
        elif fid == 513:
          if ftype == TType.STRING:
            self.sched_status = iprot.readString().decode('utf-8')
@@@ -4204,8 -4350,8 +4362,8 @@@
      if self.executors is not None:
        oprot.writeFieldBegin('executors', TType.LIST, 4)
        oprot.writeListBegin(TType.STRUCT, len(self.executors))
--      for iter286 in self.executors:
--        iter286.write(oprot)
++      for iter293 in self.executors:
++        iter293.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.status is not None:
@@@ -4215,22 -4361,14 +4373,22 @@@
      if self.errors is not None:
        oprot.writeFieldBegin('errors', TType.MAP, 6)
        oprot.writeMapBegin(TType.STRING, TType.LIST, len(self.errors))
--      for kiter287,viter288 in self.errors.items():
--        oprot.writeString(kiter287.encode('utf-8'))
--        oprot.writeListBegin(TType.STRUCT, len(viter288))
--        for iter289 in viter288:
--          iter289.write(oprot)
++      for kiter294,viter295 in self.errors.items():
++        oprot.writeString(kiter294.encode('utf-8'))
++        oprot.writeListBegin(TType.STRUCT, len(viter295))
++        for iter296 in viter295:
++          iter296.write(oprot)
          oprot.writeListEnd()
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
 +    if self.component_debug is not None:
 +      oprot.writeFieldBegin('component_debug', TType.MAP, 7)
 +      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
-       for kiter290,viter291 in self.component_debug.items():
-         oprot.writeString(kiter290.encode('utf-8'))
-         viter291.write(oprot)
++      for kiter297,viter298 in self.component_debug.items():
++        oprot.writeString(kiter297.encode('utf-8'))
++        viter298.write(oprot)
 +      oprot.writeMapEnd()
 +      oprot.writeFieldEnd()
      if self.sched_status is not None:
        oprot.writeFieldBegin('sched_status', TType.STRING, 513)
        oprot.writeString(self.sched_status.encode('utf-8'))
@@@ -4266,9 -4408,9 +4428,10 @@@
      value = (value * 31) ^ hash(self.executors)
      value = (value * 31) ^ hash(self.status)
      value = (value * 31) ^ hash(self.errors)
 +    value = (value * 31) ^ hash(self.component_debug)
      value = (value * 31) ^ hash(self.sched_status)
      value = (value * 31) ^ hash(self.owner)
+     value = (value * 31) ^ hash(self.replication_count)
      return value
  
    def __repr__(self):
@@@ -4467,11 -4531,11 +4630,11 @@@ class RebalanceOptions
        elif fid == 3:
          if ftype == TType.MAP:
            self.num_executors = {}
-           (_ktype293, _vtype294, _size292 ) = iprot.readMapBegin()
-           for _i296 in xrange(_size292):
-             _key297 = iprot.readString().decode('utf-8')
-             _val298 = iprot.readI32();
-             self.num_executors[_key297] = _val298
 -          (_ktype291, _vtype292, _size290 ) = iprot.readMapBegin()
 -          for _i294 in xrange(_size290):
 -            _key295 = iprot.readString().decode('utf-8')
 -            _val296 = iprot.readI32();
 -            self.num_executors[_key295] = _val296
++          (_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()
          else:
            iprot.skip(ftype)
@@@ -4496,9 -4560,9 +4659,9 @@@
      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 kiter299,viter300 in self.num_executors.items():
-         oprot.writeString(kiter299.encode('utf-8'))
-         oprot.writeI32(viter300)
 -      for kiter297,viter298 in self.num_executors.items():
 -        oprot.writeString(kiter297.encode('utf-8'))
 -        oprot.writeI32(viter298)
++      for kiter306,viter307 in self.num_executors.items():
++        oprot.writeString(kiter306.encode('utf-8'))
++        oprot.writeI32(viter307)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -4552,11 -4616,11 +4715,11 @@@ class Credentials
        if fid == 1:
          if ftype == TType.MAP:
            self.creds = {}
-           (_ktype302, _vtype303, _size301 ) = iprot.readMapBegin()
-           for _i305 in xrange(_size301):
-             _key306 = iprot.readString().decode('utf-8')
-             _val307 = iprot.readString().decode('utf-8')
-             self.creds[_key306] = _val307
 -          (_ktype300, _vtype301, _size299 ) = iprot.readMapBegin()
 -          for _i303 in xrange(_size299):
 -            _key304 = iprot.readString().decode('utf-8')
 -            _val305 = iprot.readString().decode('utf-8')
 -            self.creds[_key304] = _val305
++          (_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()
          else:
            iprot.skip(ftype)
@@@ -4573,9 -4637,9 +4736,9 @@@
      if self.creds is not None:
        oprot.writeFieldBegin('creds', TType.MAP, 1)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds))
-       for kiter308,viter309 in self.creds.items():
-         oprot.writeString(kiter308.encode('utf-8'))
-         oprot.writeString(viter309.encode('utf-8'))
 -      for kiter306,viter307 in self.creds.items():
 -        oprot.writeString(kiter306.encode('utf-8'))
 -        oprot.writeString(viter307.encode('utf-8'))
++      for kiter315,viter316 in self.creds.items():
++        oprot.writeString(kiter315.encode('utf-8'))
++        oprot.writeString(viter316.encode('utf-8'))
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -4746,31 -4810,31 +4909,31 @@@ class SupervisorInfo
        elif fid == 4:
          if ftype == TType.LIST:
            self.used_ports = []
-           (_etype313, _size310) = iprot.readListBegin()
-           for _i314 in xrange(_size310):
-             _elem315 = iprot.readI64();
-             self.used_ports.append(_elem315)
 -          (_etype304, _size301) = iprot.readListBegin()
 -          for _i305 in xrange(_size301):
 -            _elem306 = iprot.readI64();
 -            self.used_ports.append(_elem306)
++          (_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 = []
-           (_etype319, _size316) = iprot.readListBegin()
-           for _i320 in xrange(_size316):
-             _elem321 = iprot.readI64();
-             self.meta.append(_elem321)
 -          (_etype310, _size307) = iprot.readListBegin()
 -          for _i311 in xrange(_size307):
 -            _elem312 = iprot.readI64();
 -            self.meta.append(_elem312)
++          (_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 = {}
-           (_ktype323, _vtype324, _size322 ) = iprot.readMapBegin()
-           for _i326 in xrange(_size322):
-             _key327 = iprot.readString().decode('utf-8')
-             _val328 = iprot.readString().decode('utf-8')
-             self.scheduler_meta[_key327] = _val328
 -          (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin()
 -          for _i317 in xrange(_size313):
 -            _key318 = iprot.readString().decode('utf-8')
 -            _val319 = iprot.readString().decode('utf-8')
 -            self.scheduler_meta[_key318] = _val319
++          (_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)
@@@ -4809,23 -4873,23 +4972,23 @@@
      if self.used_ports is not None:
        oprot.writeFieldBegin('used_ports', TType.LIST, 4)
        oprot.writeListBegin(TType.I64, len(self.used_ports))
-       for iter329 in self.used_ports:
-         oprot.writeI64(iter329)
 -      for iter320 in self.used_ports:
 -        oprot.writeI64(iter320)
++      for iter336 in self.used_ports:
++        oprot.writeI64(iter336)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.meta is not None:
        oprot.writeFieldBegin('meta', TType.LIST, 5)
        oprot.writeListBegin(TType.I64, len(self.meta))
-       for iter330 in self.meta:
-         oprot.writeI64(iter330)
 -      for iter321 in self.meta:
 -        oprot.writeI64(iter321)
++      for iter337 in self.meta:
++        oprot.writeI64(iter337)
        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 kiter331,viter332 in self.scheduler_meta.items():
-         oprot.writeString(kiter331.encode('utf-8'))
-         oprot.writeString(viter332.encode('utf-8'))
 -      for kiter322,viter323 in self.scheduler_meta.items():
 -        oprot.writeString(kiter322.encode('utf-8'))
 -        oprot.writeString(viter323.encode('utf-8'))
++      for kiter338,viter339 in self.scheduler_meta.items():
++        oprot.writeString(kiter338.encode('utf-8'))
++        oprot.writeString(viter339.encode('utf-8'))
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.uptime_secs is not None:
@@@ -4904,10 -4968,10 +5067,10 @@@ class NodeInfo
        elif fid == 2:
          if ftype == TType.SET:
            self.port = set()
-           (_etype336, _size333) = iprot.readSetBegin()
-           for _i337 in xrange(_size333):
-             _elem338 = iprot.readI64();
-             self.port.add(_elem338)
 -          (_etype327, _size324) = iprot.readSetBegin()
 -          for _i328 in xrange(_size324):
 -            _elem329 = iprot.readI64();
 -            self.port.add(_elem329)
++          (_etype343, _size340) = iprot.readSetBegin()
++          for _i344 in xrange(_size340):
++            _elem345 = iprot.readI64();
++            self.port.add(_elem345)
            iprot.readSetEnd()
          else:
            iprot.skip(ftype)
@@@ -4928,8 -4992,8 +5091,8 @@@
      if self.port is not None:
        oprot.writeFieldBegin('port', TType.SET, 2)
        oprot.writeSetBegin(TType.I64, len(self.port))
-       for iter339 in self.port:
-         oprot.writeI64(iter339)
 -      for iter330 in self.port:
 -        oprot.writeI64(iter330)
++      for iter346 in self.port:
++        oprot.writeI64(iter346)
        oprot.writeSetEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -5012,44 -5076,44 +5175,44 @@@ class Assignment
        elif fid == 2:
          if ftype == TType.MAP:
            self.node_host = {}
-           (_ktype341, _vtype342, _size340 ) = iprot.readMapBegin()
-           for _i344 in xrange(_size340):
-             _key345 = iprot.readString().decode('utf-8')
-             _val346 = iprot.readString().decode('utf-8')
-             self.node_host[_key345] = _val346
 -          (_ktype332, _vtype333, _size331 ) = iprot.readMapBegin()
 -          for _i335 in xrange(_size331):
 -            _key336 = iprot.readString().decode('utf-8')
 -            _val337 = iprot.readString().decode('utf-8')
 -            self.node_host[_key336] = _val337
++          (_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
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 3:
          if ftype == TType.MAP:
            self.executor_node_port = {}
-           (_ktype348, _vtype349, _size347 ) = iprot.readMapBegin()
-           for _i351 in xrange(_size347):
-             _key352 = []
-             (_etype357, _size354) = iprot.readListBegin()
-             for _i358 in xrange(_size354):
-               _elem359 = iprot.readI64();
-               _key352.append(_elem359)
 -          (_ktype339, _vtype340, _size338 ) = iprot.readMapBegin()
 -          for _i342 in xrange(_size338):
 -            _key343 = []
 -            (_etype348, _size345) = iprot.readListBegin()
 -            for _i349 in xrange(_size345):
 -              _elem350 = iprot.readI64();
 -              _key343.append(_elem350)
++          (_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)
              iprot.readListEnd()
-             _val353 = NodeInfo()
-             _val353.read(iprot)
-             self.executor_node_port[_key352] = _val353
 -            _val344 = NodeInfo()
 -            _val344.read(iprot)
 -            self.executor_node_port[_key343] = _val344
++            _val360 = NodeInfo()
++            _val360.read(iprot)
++            self.executor_node_port[_key359] = _val360
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 4:
          if ftype == TType.MAP:
            self.executor_start_time_secs = {}
-           (_ktype361, _vtype362, _size360 ) = iprot.readMapBegin()
-           for _i364 in xrange(_size360):
-             _key365 = []
-             (_etype370, _size367) = iprot.readListBegin()
-             for _i371 in xrange(_size367):
-               _elem372 = iprot.readI64();
-               _key365.append(_elem372)
 -          (_ktype352, _vtype353, _size351 ) = iprot.readMapBegin()
 -          for _i355 in xrange(_size351):
 -            _key356 = []
 -            (_etype361, _size358) = iprot.readListBegin()
 -            for _i362 in xrange(_size358):
 -              _elem363 = iprot.readI64();
 -              _key356.append(_elem363)
++          (_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)
              iprot.readListEnd()
-             _val366 = iprot.readI64();
-             self.executor_start_time_secs[_key365] = _val366
 -            _val357 = iprot.readI64();
 -            self.executor_start_time_secs[_key356] = _val357
++            _val373 = iprot.readI64();
++            self.executor_start_time_secs[_key372] = _val373
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -5070,31 -5134,31 +5233,31 @@@
      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 kiter373,viter374 in self.node_host.items():
-         oprot.writeString(kiter373.encode('utf-8'))
-         oprot.writeString(viter374.encode('utf-8'))
 -      for kiter364,viter365 in self.node_host.items():
 -        oprot.writeString(kiter364.encode('utf-8'))
 -        oprot.writeString(viter365.encode('utf-8'))
++      for kiter380,viter381 in self.node_host.items():
++        oprot.writeString(kiter380.encode('utf-8'))
++        oprot.writeString(viter381.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 kiter375,viter376 in self.executor_node_port.items():
-         oprot.writeListBegin(TType.I64, len(kiter375))
-         for iter377 in kiter375:
-           oprot.writeI64(iter377)
 -      for kiter366,viter367 in self.executor_node_port.items():
 -        oprot.writeListBegin(TType.I64, len(kiter366))
 -        for iter368 in kiter366:
 -          oprot.writeI64(iter368)
++      for kiter382,viter383 in self.executor_node_port.items():
++        oprot.writeListBegin(TType.I64, len(kiter382))
++        for iter384 in kiter382:
++          oprot.writeI64(iter384)
          oprot.writeListEnd()
-         viter376.write(oprot)
 -        viter367.write(oprot)
++        viter383.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 kiter378,viter379 in self.executor_start_time_secs.items():
-         oprot.writeListBegin(TType.I64, len(kiter378))
-         for iter380 in kiter378:
-           oprot.writeI64(iter380)
 -      for kiter369,viter370 in self.executor_start_time_secs.items():
 -        oprot.writeListBegin(TType.I64, len(kiter369))
 -        for iter371 in kiter369:
 -          oprot.writeI64(iter371)
++      for kiter385,viter386 in self.executor_start_time_secs.items():
++        oprot.writeListBegin(TType.I64, len(kiter385))
++        for iter387 in kiter385:
++          oprot.writeI64(iter387)
          oprot.writeListEnd()
-         oprot.writeI64(viter379)
 -        oprot.writeI64(viter370)
++        oprot.writeI64(viter386)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -5270,11 -5331,11 +5433,11 @@@ class StormBase
        elif fid == 4:
          if ftype == TType.MAP:
            self.component_executors = {}
-           (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin()
-           for _i385 in xrange(_size381):
-             _key386 = iprot.readString().decode('utf-8')
-             _val387 = iprot.readI32();
-             self.component_executors[_key386] = _val387
 -          (_ktype373, _vtype374, _size372 ) = iprot.readMapBegin()
 -          for _i376 in xrange(_size372):
 -            _key377 = iprot.readString().decode('utf-8')
 -            _val378 = iprot.readI32();
 -            self.component_executors[_key377] = _val378
++          (_ktype389, _vtype390, _size388 ) = iprot.readMapBegin()
++          for _i392 in xrange(_size388):
++            _key393 = iprot.readString().decode('utf-8')
++            _val394 = iprot.readI32();
++            self.component_executors[_key393] = _val394
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -5299,18 -5360,6 +5462,18 @@@
            self.prev_status = iprot.readI32();
          else:
            iprot.skip(ftype)
 +      elif fid == 9:
 +        if ftype == TType.MAP:
 +          self.component_debug = {}
-           (_ktype389, _vtype390, _size388 ) = iprot.readMapBegin()
-           for _i392 in xrange(_size388):
-             _key393 = iprot.readString().decode('utf-8')
-             _val394 = DebugOptions()
-             _val394.read(iprot)
-             self.component_debug[_key393] = _val394
++          (_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
 +          iprot.readMapEnd()
 +        else:
 +          iprot.skip(ftype)
        else:
          iprot.skip(ftype)
        iprot.readFieldEnd()
@@@ -5336,9 -5385,9 +5499,9 @@@
      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 kiter395,viter396 in self.component_executors.items():
-         oprot.writeString(kiter395.encode('utf-8'))
-         oprot.writeI32(viter396)
 -      for kiter379,viter380 in self.component_executors.items():
 -        oprot.writeString(kiter379.encode('utf-8'))
 -        oprot.writeI32(viter380)
++      for kiter402,viter403 in self.component_executors.items():
++        oprot.writeString(kiter402.encode('utf-8'))
++        oprot.writeI32(viter403)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.launch_time_secs is not None:
@@@ -5357,14 -5406,6 +5520,14 @@@
        oprot.writeFieldBegin('prev_status', TType.I32, 8)
        oprot.writeI32(self.prev_status)
        oprot.writeFieldEnd()
 +    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 kiter397,viter398 in self.component_debug.items():
-         oprot.writeString(kiter397.encode('utf-8'))
-         viter398.write(oprot)
++      for kiter404,viter405 in self.component_debug.items():
++        oprot.writeString(kiter404.encode('utf-8'))
++        viter405.write(oprot)
 +      oprot.writeMapEnd()
 +      oprot.writeFieldEnd()
      oprot.writeFieldStop()
      oprot.writeStructEnd()
  
@@@ -5442,13 -5482,13 +5605,13 @@@ class ClusterWorkerHeartbeat
        elif fid == 2:
          if ftype == TType.MAP:
            self.executor_stats = {}
-           (_ktype400, _vtype401, _size399 ) = iprot.readMapBegin()
-           for _i403 in xrange(_size399):
-             _key404 = ExecutorInfo()
-             _key404.read(iprot)
-             _val405 = ExecutorStats()
-             _val405.read(iprot)
-             self.executor_stats[_key404] = _val405
 -          (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin()
 -          for _i385 in xrange(_size381):
 -            _key386 = ExecutorInfo()
 -            _key386.read(iprot)
 -            _val387 = ExecutorStats()
 -            _val387.read(iprot)
 -            self.executor_stats[_key386] = _val387
++          (_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
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -5479,9 -5519,9 +5642,9 @@@
      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 kiter406,viter407 in self.executor_stats.items():
-         kiter406.write(oprot)
-         viter407.write(oprot)
 -      for kiter388,viter389 in self.executor_stats.items():
 -        kiter388.write(oprot)
 -        viter389.write(oprot)
++      for kiter413,viter414 in self.executor_stats.items():
++        kiter413.write(oprot)
++        viter414.write(oprot)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.time_secs is not None:
@@@ -5634,12 -5674,12 +5797,12 @@@ class LocalStateData
        if fid == 1:
          if ftype == TType.MAP:
            self.serialized_parts = {}
-           (_ktype409, _vtype410, _size408 ) = iprot.readMapBegin()
-           for _i412 in xrange(_size408):
-             _key413 = iprot.readString().decode('utf-8')
-             _val414 = ThriftSerializedObject()
-             _val414.read(iprot)
-             self.serialized_parts[_key413] = _val414
 -          (_ktype391, _vtype392, _size390 ) = iprot.readMapBegin()
 -          for _i394 in xrange(_size390):
 -            _key395 = iprot.readString().decode('utf-8')
 -            _val396 = ThriftSerializedObject()
 -            _val396.read(iprot)
 -            self.serialized_parts[_key395] = _val396
++          (_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
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -5656,9 -5696,9 +5819,9 @@@
      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 kiter415,viter416 in self.serialized_parts.items():
-         oprot.writeString(kiter415.encode('utf-8'))
-         viter416.write(oprot)
 -      for kiter397,viter398 in self.serialized_parts.items():
 -        oprot.writeString(kiter397.encode('utf-8'))
 -        viter398.write(oprot)
++      for kiter422,viter423 in self.serialized_parts.items():
++        oprot.writeString(kiter422.encode('utf-8'))
++        viter423.write(oprot)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -5720,11 -5760,11 +5883,11 @@@ class LocalAssignment
        elif fid == 2:
          if ftype == TType.LIST:
            self.executors = []
-           (_etype420, _size417) = iprot.readListBegin()
-           for _i421 in xrange(_size417):
-             _elem422 = ExecutorInfo()
-             _elem422.read(iprot)
-             self.executors.append(_elem422)
 -          (_etype402, _size399) = iprot.readListBegin()
 -          for _i403 in xrange(_size399):
 -            _elem404 = ExecutorInfo()
 -            _elem404.read(iprot)
 -            self.executors.append(_elem404)
++          (_etype427, _size424) = iprot.readListBegin()
++          for _i428 in xrange(_size424):
++            _elem429 = ExecutorInfo()
++            _elem429.read(iprot)
++            self.executors.append(_elem429)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -5745,8 -5785,8 +5908,8 @@@
      if self.executors is not None:
        oprot.writeFieldBegin('executors', TType.LIST, 2)
        oprot.writeListBegin(TType.STRUCT, len(self.executors))
-       for iter423 in self.executors:
-         iter423.write(oprot)
 -      for iter405 in self.executors:
 -        iter405.write(oprot)
++      for iter430 in self.executors:
++        iter430.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -5870,11 -5910,11 +6033,11 @@@ class LSApprovedWorkers
        if fid == 1:
          if ftype == TType.MAP:
            self.approved_workers = {}
-           (_ktype425, _vtype426, _size424 ) = iprot.readMapBegin()
-           for _i428 in xrange(_size424):
-             _key429 = iprot.readString().decode('utf-8')
-             _val430 = iprot.readI32();
-             self.approved_workers[_key429] = _val430
 -          (_ktype407, _vtype408, _size406 ) = iprot.readMapBegin()
 -          for _i410 in xrange(_size406):
 -            _key411 = iprot.readString().decode('utf-8')
 -            _val412 = iprot.readI32();
 -            self.approved_workers[_key411] = _val412
++          (_ktype432, _vtype433, _size431 ) = iprot.readMapBegin()
++          for _i435 in xrange(_size431):
++            _key436 = iprot.readString().decode('utf-8')
++            _val437 = iprot.readI32();
++            self.approved_workers[_key436] = _val437
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -5891,9 -5931,9 +6054,9 @@@
      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 kiter431,viter432 in self.approved_workers.items():
-         oprot.writeString(kiter431.encode('utf-8'))
-         oprot.writeI32(viter432)
 -      for kiter413,viter414 in self.approved_workers.items():
 -        oprot.writeString(kiter413.encode('utf-8'))
 -        oprot.writeI32(viter414)
++      for kiter438,viter439 in self.approved_workers.items():
++        oprot.writeString(kiter438.encode('utf-8'))
++        oprot.writeI32(viter439)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -5947,12 -5987,12 +6110,12 @@@ class LSSupervisorAssignments
        if fid == 1:
          if ftype == TType.MAP:
            self.assignments = {}
-           (_ktype434, _vtype435, _size433 ) = iprot.readMapBegin()
-           for _i437 in xrange(_size433):
-             _key438 = iprot.readI32();
-             _val439 = LocalAssignment()
-             _val439.read(iprot)
-             self.assignments[_key438] = _val439
 -          (_ktype416, _vtype417, _size415 ) = iprot.readMapBegin()
 -          for _i419 in xrange(_size415):
 -            _key420 = iprot.readI32();
 -            _val421 = LocalAssignment()
 -            _val421.read(iprot)
 -            self.assignments[_key420] = _val421
++          (_ktype441, _vtype442, _size440 ) = iprot.readMapBegin()
++          for _i444 in xrange(_size440):
++            _key445 = iprot.readI32();
++            _val446 = LocalAssignment()
++            _val446.read(iprot)
++            self.assignments[_key445] = _val446
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -5969,9 -6009,9 +6132,9 @@@
      if self.assignments is not None:
        oprot.writeFieldBegin('assignments', TType.MAP, 1)
        oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments))
-       for kiter440,viter441 in self.assignments.items():
-         oprot.writeI32(kiter440)
-         viter441.write(oprot)
 -      for kiter422,viter423 in self.assignments.items():
 -        oprot.writeI32(kiter422)
 -        viter423.write(oprot)
++      for kiter447,viter448 in self.assignments.items():
++        oprot.writeI32(kiter447)
++        viter448.write(oprot)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -6044,11 -6084,11 +6207,11 @@@ class LSWorkerHeartbeat
        elif fid == 3:
          if ftype == TType.LIST:
            self.executors = []
-           (_etype445, _size442) = iprot.readListBegin()
-           for _i446 in xrange(_size442):
-             _elem447 = ExecutorInfo()
-             _elem447.read(iprot)
-             self.executors.append(_elem447)
 -          (_etype427, _size424) = iprot.readListBegin()
 -          for _i428 in xrange(_size424):
 -            _elem429 = ExecutorInfo()
 -            _elem429.read(iprot)
 -            self.executors.append(_elem429)
++          (_etype452, _size449) = iprot.readListBegin()
++          for _i453 in xrange(_size449):
++            _elem454 = ExecutorInfo()
++            _elem454.read(iprot)
++            self.executors.append(_elem454)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -6078,8 -6118,8 +6241,8 @@@
      if self.executors is not None:
        oprot.writeFieldBegin('executors', TType.LIST, 3)
        oprot.writeListBegin(TType.STRUCT, len(self.executors))
-       for iter448 in self.executors:
-         iter448.write(oprot)
 -      for iter430 in self.executors:
 -        iter430.write(oprot)
++      for iter455 in self.executors:
++        iter455.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.port is not None:

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --cc storm-core/src/storm.thrift
index 2556fbe,a585924..20dbbbe
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@@ -218,16 -227,11 +227,17 @@@ struct TopologyInfo 
    4: required list<ExecutorSummary> executors;
    5: required string status;
    6: required map<string, list<ErrorInfo>> errors;
 +  7: optional map<string, DebugOptions> component_debug;
  513: optional string sched_status;
  514: optional string owner;
+ 515: optional i32 replication_count;
  }
  
 +struct DebugOptions {
 +  1: optional bool enable
 +  2: optional double samplingpct
 +}
 +
  struct KillOptions {
    1: optional i32 wait_secs;
  }

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/templates/topology-page-template.html
index 628edfc,1811fb8..3292ee0
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@@ -340,11 -356,9 +356,11 @@@
  <script id="topology-actions-template" type="text/html">
    <h2>Topology actions</h2>
    <p id="topology-actions">
 -    <input {{activateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'activate', false, 0)" type="button" value="Activate" class="btn btn-default">
 -    <input {{deactivateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'deactivate', false, 0)" type="button" value="Deactivate" class="btn btn-default">
 -    <input {{rebalanceStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'rebalance', true, {{msgTimeout}})" type="button" value="Rebalance" class="btn btn-default">
 -    <input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30)" type="button" value="Kill" class="btn btn-default">
 +    <input {{activateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'activate', false)" type="button" value="Activate" class="btn btn-default">
 +    <input {{deactivateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'deactivate', false)" type="button" value="Deactivate" class="btn btn-default">
 +    <input {{rebalanceStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'rebalance', true, {{msgTimeout}}, 'wait time in seconds')" type="button" value="Rebalance" class="btn btn-default">
 +    <input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30, 'wait time in seconds')" type="button" value="Kill" class="btn btn-default">
 +    <input {{startDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/enable', true, {{currentSamplingPct}}, 'sampling percentage', 'debug')" type="button" value="Debug" class="btn btn-default">
 +    <input {{stopDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/disable', false, 0, 'sampling percentage', 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
    </p>
- </script>
+ </script>

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/cluster_test.clj
index 251c0c6,ba1cf02..b413d43
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@@ -170,8 -172,12 +172,12 @@@
      (let [state (mk-storm-state zk-port)
            assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {})
            assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {})
+           nimbusInfo1 (NimbusInfo. "nimbus1" 6667 false)
+           nimbusInfo2 (NimbusInfo. "nimbus2" 6667 false)
+           nimbusSummary1 (NimbusSummary. "nimbus1" 6667 (current-time-secs) false "v1")
+           nimbusSummary2 (NimbusSummary. "nimbus2" 6667 (current-time-secs) false "v2")
 -          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil)
 -          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil)]
 +          base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil {})
 +          base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil {})]
        (is (= [] (.assignments state nil)))
        (.set-assignment! state "storm1" assignment1)
        (is (= assignment1 (.assignment-info state "storm1" nil)))

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/nimbus_test.clj
index 3d0978c,cbd88c4..5956128
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@@ -21,8 -20,9 +21,9 @@@
    (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount
              TestAggregatesCounter TestPlannerSpout TestPlannerBolt])
    (:import [backtype.storm.scheduler INimbus])
+   (:import [backtype.storm.nimbus ILeaderElector NimbusInfo])
    (:import [backtype.storm.generated Credentials NotAliveException SubmitOptions
 -            TopologyInitialStatus AlreadyAliveException KillOptions RebalanceOptions
 +            TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
              InvalidTopologyException AuthorizationException])
    (:import [java.util HashMap])
    (:import [java.io File])


[17/24] storm git commit: merging from upstream

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/Credentials.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/Credentials.java
index 896d227,c1ef293..8484b18
--- a/storm-core/src/jvm/backtype/storm/generated/Credentials.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class Credentials implements org.apache.thrift.TBase<Credentials, Credentials._Fields>, java.io.Serializable, Cloneable, Comparable<Credentials> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials");
  
@@@ -365,15 -365,15 +365,15 @@@
            case 1: // CREDS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map336 = iprot.readMapBegin();
-                 struct.creds = new HashMap<String,String>(2*_map336.size);
-                 String _key337;
-                 String _val338;
-                 for (int _i339 = 0; _i339 < _map336.size; ++_i339)
 -                org.apache.thrift.protocol.TMap _map326 = iprot.readMapBegin();
 -                struct.creds = new HashMap<String,String>(2*_map326.size);
 -                String _key327;
 -                String _val328;
 -                for (int _i329 = 0; _i329 < _map326.size; ++_i329)
++                org.apache.thrift.protocol.TMap _map344 = iprot.readMapBegin();
++                struct.creds = new HashMap<String,String>(2*_map344.size);
++                String _key345;
++                String _val346;
++                for (int _i347 = 0; _i347 < _map344.size; ++_i347)
                  {
-                   _key337 = iprot.readString();
-                   _val338 = iprot.readString();
-                   struct.creds.put(_key337, _val338);
 -                  _key327 = iprot.readString();
 -                  _val328 = iprot.readString();
 -                  struct.creds.put(_key327, _val328);
++                  _key345 = iprot.readString();
++                  _val346 = iprot.readString();
++                  struct.creds.put(_key345, _val346);
                  }
                  iprot.readMapEnd();
                }
@@@ -399,10 -399,10 +399,10 @@@
          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<String, String> _iter340 : struct.creds.entrySet())
 -          for (Map.Entry<String, String> _iter330 : struct.creds.entrySet())
++          for (Map.Entry<String, String> _iter348 : struct.creds.entrySet())
            {
-             oprot.writeString(_iter340.getKey());
-             oprot.writeString(_iter340.getValue());
 -            oprot.writeString(_iter330.getKey());
 -            oprot.writeString(_iter330.getValue());
++            oprot.writeString(_iter348.getKey());
++            oprot.writeString(_iter348.getValue());
            }
            oprot.writeMapEnd();
          }
@@@ -427,10 -427,10 +427,10 @@@
        TTupleProtocol oprot = (TTupleProtocol) prot;
        {
          oprot.writeI32(struct.creds.size());
-         for (Map.Entry<String, String> _iter341 : struct.creds.entrySet())
 -        for (Map.Entry<String, String> _iter331 : struct.creds.entrySet())
++        for (Map.Entry<String, String> _iter349 : struct.creds.entrySet())
          {
-           oprot.writeString(_iter341.getKey());
-           oprot.writeString(_iter341.getValue());
 -          oprot.writeString(_iter331.getKey());
 -          oprot.writeString(_iter331.getValue());
++          oprot.writeString(_iter349.getKey());
++          oprot.writeString(_iter349.getValue());
          }
        }
      }
@@@ -439,15 -439,15 +439,15 @@@
      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 _map342 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-         struct.creds = new HashMap<String,String>(2*_map342.size);
-         String _key343;
-         String _val344;
-         for (int _i345 = 0; _i345 < _map342.size; ++_i345)
 -        org.apache.thrift.protocol.TMap _map332 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -        struct.creds = new HashMap<String,String>(2*_map332.size);
 -        String _key333;
 -        String _val334;
 -        for (int _i335 = 0; _i335 < _map332.size; ++_i335)
++        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<String,String>(2*_map350.size);
++        String _key351;
++        String _val352;
++        for (int _i353 = 0; _i353 < _map350.size; ++_i353)
          {
-           _key343 = iprot.readString();
-           _val344 = iprot.readString();
-           struct.creds.put(_key343, _val344);
 -          _key333 = iprot.readString();
 -          _val334 = iprot.readString();
 -          struct.creds.put(_key333, _val334);
++          _key351 = iprot.readString();
++          _val352 = iprot.readString();
++          struct.creds.put(_key351, _val352);
          }
        }
        struct.set_creds_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
index 001fcff,001fcff..58a7936
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
--@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, ExecutorStats._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorStats> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorStats");
  
@@@ -660,27 -660,27 +660,27 @@@
            case 1: // EMITTED
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
--                org.apache.thrift.protocol.TMap _map250 = iprot.readMapBegin();
--                struct.emitted = new HashMap<String,Map<String,Long>>(2*_map250.size);
--                String _key251;
--                Map<String,Long> _val252;
--                for (int _i253 = 0; _i253 < _map250.size; ++_i253)
++                org.apache.thrift.protocol.TMap _map258 = iprot.readMapBegin();
++                struct.emitted = new HashMap<String,Map<String,Long>>(2*_map258.size);
++                String _key259;
++                Map<String,Long> _val260;
++                for (int _i261 = 0; _i261 < _map258.size; ++_i261)
                  {
--                  _key251 = iprot.readString();
++                  _key259 = iprot.readString();
                    {
--                    org.apache.thrift.protocol.TMap _map254 = iprot.readMapBegin();
--                    _val252 = new HashMap<String,Long>(2*_map254.size);
--                    String _key255;
--                    long _val256;
--                    for (int _i257 = 0; _i257 < _map254.size; ++_i257)
++                    org.apache.thrift.protocol.TMap _map262 = iprot.readMapBegin();
++                    _val260 = new HashMap<String,Long>(2*_map262.size);
++                    String _key263;
++                    long _val264;
++                    for (int _i265 = 0; _i265 < _map262.size; ++_i265)
                      {
--                      _key255 = iprot.readString();
--                      _val256 = iprot.readI64();
--                      _val252.put(_key255, _val256);
++                      _key263 = iprot.readString();
++                      _val264 = iprot.readI64();
++                      _val260.put(_key263, _val264);
                      }
                      iprot.readMapEnd();
                    }
--                  struct.emitted.put(_key251, _val252);
++                  struct.emitted.put(_key259, _val260);
                  }
                  iprot.readMapEnd();
                }
@@@ -692,27 -692,27 +692,27 @@@
            case 2: // TRANSFERRED
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
--                org.apache.thrift.protocol.TMap _map258 = iprot.readMapBegin();
--                struct.transferred = new HashMap<String,Map<String,Long>>(2*_map258.size);
--                String _key259;
--                Map<String,Long> _val260;
--                for (int _i261 = 0; _i261 < _map258.size; ++_i261)
++                org.apache.thrift.protocol.TMap _map266 = iprot.readMapBegin();
++                struct.transferred = new HashMap<String,Map<String,Long>>(2*_map266.size);
++                String _key267;
++                Map<String,Long> _val268;
++                for (int _i269 = 0; _i269 < _map266.size; ++_i269)
                  {
--                  _key259 = iprot.readString();
++                  _key267 = iprot.readString();
                    {
--                    org.apache.thrift.protocol.TMap _map262 = iprot.readMapBegin();
--                    _val260 = new HashMap<String,Long>(2*_map262.size);
--                    String _key263;
--                    long _val264;
--                    for (int _i265 = 0; _i265 < _map262.size; ++_i265)
++                    org.apache.thrift.protocol.TMap _map270 = iprot.readMapBegin();
++                    _val268 = new HashMap<String,Long>(2*_map270.size);
++                    String _key271;
++                    long _val272;
++                    for (int _i273 = 0; _i273 < _map270.size; ++_i273)
                      {
--                      _key263 = iprot.readString();
--                      _val264 = iprot.readI64();
--                      _val260.put(_key263, _val264);
++                      _key271 = iprot.readString();
++                      _val272 = iprot.readI64();
++                      _val268.put(_key271, _val272);
                      }
                      iprot.readMapEnd();
                    }
--                  struct.transferred.put(_key259, _val260);
++                  struct.transferred.put(_key267, _val268);
                  }
                  iprot.readMapEnd();
                }
@@@ -755,15 -755,15 +755,15 @@@
          oprot.writeFieldBegin(EMITTED_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.emitted.size()));
--          for (Map.Entry<String, Map<String,Long>> _iter266 : struct.emitted.entrySet())
++          for (Map.Entry<String, Map<String,Long>> _iter274 : struct.emitted.entrySet())
            {
--            oprot.writeString(_iter266.getKey());
++            oprot.writeString(_iter274.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter266.getValue().size()));
--              for (Map.Entry<String, Long> _iter267 : _iter266.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter274.getValue().size()));
++              for (Map.Entry<String, Long> _iter275 : _iter274.getValue().entrySet())
                {
--                oprot.writeString(_iter267.getKey());
--                oprot.writeI64(_iter267.getValue());
++                oprot.writeString(_iter275.getKey());
++                oprot.writeI64(_iter275.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -776,15 -776,15 +776,15 @@@
          oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.transferred.size()));
--          for (Map.Entry<String, Map<String,Long>> _iter268 : struct.transferred.entrySet())
++          for (Map.Entry<String, Map<String,Long>> _iter276 : struct.transferred.entrySet())
            {
--            oprot.writeString(_iter268.getKey());
++            oprot.writeString(_iter276.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter268.getValue().size()));
--              for (Map.Entry<String, Long> _iter269 : _iter268.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter276.getValue().size()));
++              for (Map.Entry<String, Long> _iter277 : _iter276.getValue().entrySet())
                {
--                oprot.writeString(_iter269.getKey());
--                oprot.writeI64(_iter269.getValue());
++                oprot.writeString(_iter277.getKey());
++                oprot.writeI64(_iter277.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -820,30 -820,30 +820,30 @@@
        TTupleProtocol oprot = (TTupleProtocol) prot;
        {
          oprot.writeI32(struct.emitted.size());
--        for (Map.Entry<String, Map<String,Long>> _iter270 : struct.emitted.entrySet())
++        for (Map.Entry<String, Map<String,Long>> _iter278 : struct.emitted.entrySet())
          {
--          oprot.writeString(_iter270.getKey());
++          oprot.writeString(_iter278.getKey());
            {
--            oprot.writeI32(_iter270.getValue().size());
--            for (Map.Entry<String, Long> _iter271 : _iter270.getValue().entrySet())
++            oprot.writeI32(_iter278.getValue().size());
++            for (Map.Entry<String, Long> _iter279 : _iter278.getValue().entrySet())
              {
--              oprot.writeString(_iter271.getKey());
--              oprot.writeI64(_iter271.getValue());
++              oprot.writeString(_iter279.getKey());
++              oprot.writeI64(_iter279.getValue());
              }
            }
          }
        }
        {
          oprot.writeI32(struct.transferred.size());
--        for (Map.Entry<String, Map<String,Long>> _iter272 : struct.transferred.entrySet())
++        for (Map.Entry<String, Map<String,Long>> _iter280 : struct.transferred.entrySet())
          {
--          oprot.writeString(_iter272.getKey());
++          oprot.writeString(_iter280.getKey());
            {
--            oprot.writeI32(_iter272.getValue().size());
--            for (Map.Entry<String, Long> _iter273 : _iter272.getValue().entrySet())
++            oprot.writeI32(_iter280.getValue().size());
++            for (Map.Entry<String, Long> _iter281 : _iter280.getValue().entrySet())
              {
--              oprot.writeString(_iter273.getKey());
--              oprot.writeI64(_iter273.getValue());
++              oprot.writeString(_iter281.getKey());
++              oprot.writeI64(_iter281.getValue());
              }
            }
          }
@@@ -856,32 -856,32 +856,8 @@@
      public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorStats struct) throws org.apache.thrift.TException {
        TTupleProtocol iprot = (TTupleProtocol) prot;
        {
--        org.apache.thrift.protocol.TMap _map274 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.emitted = new HashMap<String,Map<String,Long>>(2*_map274.size);
--        String _key275;
--        Map<String,Long> _val276;
--        for (int _i277 = 0; _i277 < _map274.size; ++_i277)
--        {
--          _key275 = iprot.readString();
--          {
--            org.apache.thrift.protocol.TMap _map278 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
--            _val276 = new HashMap<String,Long>(2*_map278.size);
--            String _key279;
--            long _val280;
--            for (int _i281 = 0; _i281 < _map278.size; ++_i281)
--            {
--              _key279 = iprot.readString();
--              _val280 = iprot.readI64();
--              _val276.put(_key279, _val280);
--            }
--          }
--          struct.emitted.put(_key275, _val276);
--        }
--      }
--      struct.set_emitted_isSet(true);
--      {
          org.apache.thrift.protocol.TMap _map282 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.transferred = new HashMap<String,Map<String,Long>>(2*_map282.size);
++        struct.emitted = new HashMap<String,Map<String,Long>>(2*_map282.size);
          String _key283;
          Map<String,Long> _val284;
          for (int _i285 = 0; _i285 < _map282.size; ++_i285)
@@@ -899,7 -899,7 +875,31 @@@
                _val284.put(_key287, _val288);
              }
            }
--          struct.transferred.put(_key283, _val284);
++          struct.emitted.put(_key283, _val284);
++        }
++      }
++      struct.set_emitted_isSet(true);
++      {
++        org.apache.thrift.protocol.TMap _map290 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
++        struct.transferred = new HashMap<String,Map<String,Long>>(2*_map290.size);
++        String _key291;
++        Map<String,Long> _val292;
++        for (int _i293 = 0; _i293 < _map290.size; ++_i293)
++        {
++          _key291 = iprot.readString();
++          {
++            org.apache.thrift.protocol.TMap _map294 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
++            _val292 = new HashMap<String,Long>(2*_map294.size);
++            String _key295;
++            long _val296;
++            for (int _i297 = 0; _i297 < _map294.size; ++_i297)
++            {
++              _key295 = iprot.readString();
++              _val296 = iprot.readI64();
++              _val292.put(_key295, _val296);
++            }
++          }
++          struct.transferred.put(_key291, _val292);
          }
        }
        struct.set_transferred_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
index be950cb,1d97dca..fd9740b
--- a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWorkers, LSApprovedWorkers._Fields>, java.io.Serializable, Cloneable, Comparable<LSApprovedWorkers> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSApprovedWorkers");
  
@@@ -365,15 -365,15 +365,15 @@@
            case 1: // APPROVED_WORKERS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map474 = iprot.readMapBegin();
-                 struct.approved_workers = new HashMap<String,Integer>(2*_map474.size);
-                 String _key475;
-                 int _val476;
-                 for (int _i477 = 0; _i477 < _map474.size; ++_i477)
 -                org.apache.thrift.protocol.TMap _map454 = iprot.readMapBegin();
 -                struct.approved_workers = new HashMap<String,Integer>(2*_map454.size);
 -                String _key455;
 -                int _val456;
 -                for (int _i457 = 0; _i457 < _map454.size; ++_i457)
++                org.apache.thrift.protocol.TMap _map482 = iprot.readMapBegin();
++                struct.approved_workers = new HashMap<String,Integer>(2*_map482.size);
++                String _key483;
++                int _val484;
++                for (int _i485 = 0; _i485 < _map482.size; ++_i485)
                  {
-                   _key475 = iprot.readString();
-                   _val476 = iprot.readI32();
-                   struct.approved_workers.put(_key475, _val476);
 -                  _key455 = iprot.readString();
 -                  _val456 = iprot.readI32();
 -                  struct.approved_workers.put(_key455, _val456);
++                  _key483 = iprot.readString();
++                  _val484 = iprot.readI32();
++                  struct.approved_workers.put(_key483, _val484);
                  }
                  iprot.readMapEnd();
                }
@@@ -399,10 -399,10 +399,10 @@@
          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<String, Integer> _iter478 : struct.approved_workers.entrySet())
 -          for (Map.Entry<String, Integer> _iter458 : struct.approved_workers.entrySet())
++          for (Map.Entry<String, Integer> _iter486 : struct.approved_workers.entrySet())
            {
-             oprot.writeString(_iter478.getKey());
-             oprot.writeI32(_iter478.getValue());
 -            oprot.writeString(_iter458.getKey());
 -            oprot.writeI32(_iter458.getValue());
++            oprot.writeString(_iter486.getKey());
++            oprot.writeI32(_iter486.getValue());
            }
            oprot.writeMapEnd();
          }
@@@ -427,10 -427,10 +427,10 @@@
        TTupleProtocol oprot = (TTupleProtocol) prot;
        {
          oprot.writeI32(struct.approved_workers.size());
-         for (Map.Entry<String, Integer> _iter479 : struct.approved_workers.entrySet())
 -        for (Map.Entry<String, Integer> _iter459 : struct.approved_workers.entrySet())
++        for (Map.Entry<String, Integer> _iter487 : struct.approved_workers.entrySet())
          {
-           oprot.writeString(_iter479.getKey());
-           oprot.writeI32(_iter479.getValue());
 -          oprot.writeString(_iter459.getKey());
 -          oprot.writeI32(_iter459.getValue());
++          oprot.writeString(_iter487.getKey());
++          oprot.writeI32(_iter487.getValue());
          }
        }
      }
@@@ -439,15 -439,15 +439,15 @@@
      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 _map480 = 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<String,Integer>(2*_map480.size);
-         String _key481;
-         int _val482;
-         for (int _i483 = 0; _i483 < _map480.size; ++_i483)
 -        org.apache.thrift.protocol.TMap _map460 = 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<String,Integer>(2*_map460.size);
 -        String _key461;
 -        int _val462;
 -        for (int _i463 = 0; _i463 < _map460.size; ++_i463)
++        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<String,Integer>(2*_map488.size);
++        String _key489;
++        int _val490;
++        for (int _i491 = 0; _i491 < _map488.size; ++_i491)
          {
-           _key481 = iprot.readString();
-           _val482 = iprot.readI32();
-           struct.approved_workers.put(_key481, _val482);
 -          _key461 = iprot.readString();
 -          _val462 = iprot.readI32();
 -          struct.approved_workers.put(_key461, _val462);
++          _key489 = iprot.readString();
++          _val490 = iprot.readI32();
++          struct.approved_workers.put(_key489, _val490);
          }
        }
        struct.set_approved_workers_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
index 4ad9aaf,4667287..1d85cec
--- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSupervisorAssignments, LSSupervisorAssignments._Fields>, java.io.Serializable, Cloneable, Comparable<LSSupervisorAssignments> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorAssignments");
  
@@@ -376,16 -376,16 +376,16 @@@
            case 1: // ASSIGNMENTS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map484 = iprot.readMapBegin();
-                 struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map484.size);
-                 int _key485;
-                 LocalAssignment _val486;
-                 for (int _i487 = 0; _i487 < _map484.size; ++_i487)
 -                org.apache.thrift.protocol.TMap _map464 = iprot.readMapBegin();
 -                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map464.size);
 -                int _key465;
 -                LocalAssignment _val466;
 -                for (int _i467 = 0; _i467 < _map464.size; ++_i467)
++                org.apache.thrift.protocol.TMap _map492 = iprot.readMapBegin();
++                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map492.size);
++                int _key493;
++                LocalAssignment _val494;
++                for (int _i495 = 0; _i495 < _map492.size; ++_i495)
                  {
-                   _key485 = iprot.readI32();
-                   _val486 = new LocalAssignment();
-                   _val486.read(iprot);
-                   struct.assignments.put(_key485, _val486);
 -                  _key465 = iprot.readI32();
 -                  _val466 = new LocalAssignment();
 -                  _val466.read(iprot);
 -                  struct.assignments.put(_key465, _val466);
++                  _key493 = iprot.readI32();
++                  _val494 = new LocalAssignment();
++                  _val494.read(iprot);
++                  struct.assignments.put(_key493, _val494);
                  }
                  iprot.readMapEnd();
                }
@@@ -411,10 -411,10 +411,10 @@@
          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<Integer, LocalAssignment> _iter488 : struct.assignments.entrySet())
 -          for (Map.Entry<Integer, LocalAssignment> _iter468 : struct.assignments.entrySet())
++          for (Map.Entry<Integer, LocalAssignment> _iter496 : struct.assignments.entrySet())
            {
-             oprot.writeI32(_iter488.getKey());
-             _iter488.getValue().write(oprot);
 -            oprot.writeI32(_iter468.getKey());
 -            _iter468.getValue().write(oprot);
++            oprot.writeI32(_iter496.getKey());
++            _iter496.getValue().write(oprot);
            }
            oprot.writeMapEnd();
          }
@@@ -439,10 -439,10 +439,10 @@@
        TTupleProtocol oprot = (TTupleProtocol) prot;
        {
          oprot.writeI32(struct.assignments.size());
-         for (Map.Entry<Integer, LocalAssignment> _iter489 : struct.assignments.entrySet())
 -        for (Map.Entry<Integer, LocalAssignment> _iter469 : struct.assignments.entrySet())
++        for (Map.Entry<Integer, LocalAssignment> _iter497 : struct.assignments.entrySet())
          {
-           oprot.writeI32(_iter489.getKey());
-           _iter489.getValue().write(oprot);
 -          oprot.writeI32(_iter469.getKey());
 -          _iter469.getValue().write(oprot);
++          oprot.writeI32(_iter497.getKey());
++          _iter497.getValue().write(oprot);
          }
        }
      }
@@@ -451,16 -451,16 +451,16 @@@
      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 _map490 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-         struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map490.size);
-         int _key491;
-         LocalAssignment _val492;
-         for (int _i493 = 0; _i493 < _map490.size; ++_i493)
 -        org.apache.thrift.protocol.TMap _map470 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map470.size);
 -        int _key471;
 -        LocalAssignment _val472;
 -        for (int _i473 = 0; _i473 < _map470.size; ++_i473)
++        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<Integer,LocalAssignment>(2*_map498.size);
++        int _key499;
++        LocalAssignment _val500;
++        for (int _i501 = 0; _i501 < _map498.size; ++_i501)
          {
-           _key491 = iprot.readI32();
-           _val492 = new LocalAssignment();
-           _val492.read(iprot);
-           struct.assignments.put(_key491, _val492);
 -          _key471 = iprot.readI32();
 -          _val472 = new LocalAssignment();
 -          _val472.read(iprot);
 -          struct.assignments.put(_key471, _val472);
++          _key499 = iprot.readI32();
++          _val500 = new LocalAssignment();
++          _val500.read(iprot);
++          struct.assignments.put(_key499, _val500);
          }
        }
        struct.set_assignments_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
index 5384063,80f917b..98726cc
--- a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartbeat, LSWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<LSWorkerHeartbeat> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSWorkerHeartbeat");
  
@@@ -638,14 -638,14 +638,14 @@@
            case 3: // EXECUTORS
              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                {
-                 org.apache.thrift.protocol.TList _list494 = iprot.readListBegin();
-                 struct.executors = new ArrayList<ExecutorInfo>(_list494.size);
-                 ExecutorInfo _elem495;
-                 for (int _i496 = 0; _i496 < _list494.size; ++_i496)
 -                org.apache.thrift.protocol.TList _list474 = iprot.readListBegin();
 -                struct.executors = new ArrayList<ExecutorInfo>(_list474.size);
 -                ExecutorInfo _elem475;
 -                for (int _i476 = 0; _i476 < _list474.size; ++_i476)
++                org.apache.thrift.protocol.TList _list502 = iprot.readListBegin();
++                struct.executors = new ArrayList<ExecutorInfo>(_list502.size);
++                ExecutorInfo _elem503;
++                for (int _i504 = 0; _i504 < _list502.size; ++_i504)
                  {
-                   _elem495 = new ExecutorInfo();
-                   _elem495.read(iprot);
-                   struct.executors.add(_elem495);
 -                  _elem475 = new ExecutorInfo();
 -                  _elem475.read(iprot);
 -                  struct.executors.add(_elem475);
++                  _elem503 = new ExecutorInfo();
++                  _elem503.read(iprot);
++                  struct.executors.add(_elem503);
                  }
                  iprot.readListEnd();
                }
@@@ -687,9 -687,9 +687,9 @@@
          oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
          {
            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-           for (ExecutorInfo _iter497 : struct.executors)
 -          for (ExecutorInfo _iter477 : struct.executors)
++          for (ExecutorInfo _iter505 : struct.executors)
            {
-             _iter497.write(oprot);
 -            _iter477.write(oprot);
++            _iter505.write(oprot);
            }
            oprot.writeListEnd();
          }
@@@ -719,9 -719,9 +719,9 @@@
        oprot.writeString(struct.topology_id);
        {
          oprot.writeI32(struct.executors.size());
-         for (ExecutorInfo _iter498 : struct.executors)
 -        for (ExecutorInfo _iter478 : struct.executors)
++        for (ExecutorInfo _iter506 : struct.executors)
          {
-           _iter498.write(oprot);
 -          _iter478.write(oprot);
++          _iter506.write(oprot);
          }
        }
        oprot.writeI32(struct.port);
@@@ -735,14 -735,14 +735,14 @@@
        struct.topology_id = iprot.readString();
        struct.set_topology_id_isSet(true);
        {
-         org.apache.thrift.protocol.TList _list499 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-         struct.executors = new ArrayList<ExecutorInfo>(_list499.size);
-         ExecutorInfo _elem500;
-         for (int _i501 = 0; _i501 < _list499.size; ++_i501)
 -        org.apache.thrift.protocol.TList _list479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -        struct.executors = new ArrayList<ExecutorInfo>(_list479.size);
 -        ExecutorInfo _elem480;
 -        for (int _i481 = 0; _i481 < _list479.size; ++_i481)
++        org.apache.thrift.protocol.TList _list507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++        struct.executors = new ArrayList<ExecutorInfo>(_list507.size);
++        ExecutorInfo _elem508;
++        for (int _i509 = 0; _i509 < _list507.size; ++_i509)
          {
-           _elem500 = new ExecutorInfo();
-           _elem500.read(iprot);
-           struct.executors.add(_elem500);
 -          _elem480 = new ExecutorInfo();
 -          _elem480.read(iprot);
 -          struct.executors.add(_elem480);
++          _elem508 = new ExecutorInfo();
++          _elem508.read(iprot);
++          struct.executors.add(_elem508);
          }
        }
        struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
index 695c2cb,3c34891..d985cba
--- a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment, LocalAssignment._Fields>, java.io.Serializable, Cloneable, Comparable<LocalAssignment> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalAssignment");
  
@@@ -464,14 -464,14 +464,14 @@@
            case 2: // EXECUTORS
              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                {
-                 org.apache.thrift.protocol.TList _list466 = iprot.readListBegin();
-                 struct.executors = new ArrayList<ExecutorInfo>(_list466.size);
-                 ExecutorInfo _elem467;
-                 for (int _i468 = 0; _i468 < _list466.size; ++_i468)
 -                org.apache.thrift.protocol.TList _list446 = iprot.readListBegin();
 -                struct.executors = new ArrayList<ExecutorInfo>(_list446.size);
 -                ExecutorInfo _elem447;
 -                for (int _i448 = 0; _i448 < _list446.size; ++_i448)
++                org.apache.thrift.protocol.TList _list474 = iprot.readListBegin();
++                struct.executors = new ArrayList<ExecutorInfo>(_list474.size);
++                ExecutorInfo _elem475;
++                for (int _i476 = 0; _i476 < _list474.size; ++_i476)
                  {
-                   _elem467 = new ExecutorInfo();
-                   _elem467.read(iprot);
-                   struct.executors.add(_elem467);
 -                  _elem447 = new ExecutorInfo();
 -                  _elem447.read(iprot);
 -                  struct.executors.add(_elem447);
++                  _elem475 = new ExecutorInfo();
++                  _elem475.read(iprot);
++                  struct.executors.add(_elem475);
                  }
                  iprot.readListEnd();
                }
@@@ -502,9 -502,9 +502,9 @@@
          oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
          {
            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-           for (ExecutorInfo _iter469 : struct.executors)
 -          for (ExecutorInfo _iter449 : struct.executors)
++          for (ExecutorInfo _iter477 : struct.executors)
            {
-             _iter469.write(oprot);
 -            _iter449.write(oprot);
++            _iter477.write(oprot);
            }
            oprot.writeListEnd();
          }
@@@ -530,9 -530,9 +530,9 @@@
        oprot.writeString(struct.topology_id);
        {
          oprot.writeI32(struct.executors.size());
-         for (ExecutorInfo _iter470 : struct.executors)
 -        for (ExecutorInfo _iter450 : struct.executors)
++        for (ExecutorInfo _iter478 : struct.executors)
          {
-           _iter470.write(oprot);
 -          _iter450.write(oprot);
++          _iter478.write(oprot);
          }
        }
      }
@@@ -543,14 -543,14 +543,14 @@@
        struct.topology_id = iprot.readString();
        struct.set_topology_id_isSet(true);
        {
-         org.apache.thrift.protocol.TList _list471 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-         struct.executors = new ArrayList<ExecutorInfo>(_list471.size);
-         ExecutorInfo _elem472;
-         for (int _i473 = 0; _i473 < _list471.size; ++_i473)
 -        org.apache.thrift.protocol.TList _list451 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -        struct.executors = new ArrayList<ExecutorInfo>(_list451.size);
 -        ExecutorInfo _elem452;
 -        for (int _i453 = 0; _i453 < _list451.size; ++_i453)
++        org.apache.thrift.protocol.TList _list479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++        struct.executors = new ArrayList<ExecutorInfo>(_list479.size);
++        ExecutorInfo _elem480;
++        for (int _i481 = 0; _i481 < _list479.size; ++_i481)
          {
-           _elem472 = new ExecutorInfo();
-           _elem472.read(iprot);
-           struct.executors.add(_elem472);
 -          _elem452 = new ExecutorInfo();
 -          _elem452.read(iprot);
 -          struct.executors.add(_elem452);
++          _elem480 = new ExecutorInfo();
++          _elem480.read(iprot);
++          struct.executors.add(_elem480);
          }
        }
        struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
index 69eb2a9,448711d..0ce2d7a
--- a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, LocalStateData._Fields>, java.io.Serializable, Cloneable, Comparable<LocalStateData> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalStateData");
  
@@@ -376,16 -376,16 +376,16 @@@
            case 1: // SERIALIZED_PARTS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map456 = iprot.readMapBegin();
-                 struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map456.size);
-                 String _key457;
-                 ThriftSerializedObject _val458;
-                 for (int _i459 = 0; _i459 < _map456.size; ++_i459)
 -                org.apache.thrift.protocol.TMap _map436 = iprot.readMapBegin();
 -                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map436.size);
 -                String _key437;
 -                ThriftSerializedObject _val438;
 -                for (int _i439 = 0; _i439 < _map436.size; ++_i439)
++                org.apache.thrift.protocol.TMap _map464 = iprot.readMapBegin();
++                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map464.size);
++                String _key465;
++                ThriftSerializedObject _val466;
++                for (int _i467 = 0; _i467 < _map464.size; ++_i467)
                  {
-                   _key457 = iprot.readString();
-                   _val458 = new ThriftSerializedObject();
-                   _val458.read(iprot);
-                   struct.serialized_parts.put(_key457, _val458);
 -                  _key437 = iprot.readString();
 -                  _val438 = new ThriftSerializedObject();
 -                  _val438.read(iprot);
 -                  struct.serialized_parts.put(_key437, _val438);
++                  _key465 = iprot.readString();
++                  _val466 = new ThriftSerializedObject();
++                  _val466.read(iprot);
++                  struct.serialized_parts.put(_key465, _val466);
                  }
                  iprot.readMapEnd();
                }
@@@ -411,10 -411,10 +411,10 @@@
          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<String, ThriftSerializedObject> _iter460 : struct.serialized_parts.entrySet())
 -          for (Map.Entry<String, ThriftSerializedObject> _iter440 : struct.serialized_parts.entrySet())
++          for (Map.Entry<String, ThriftSerializedObject> _iter468 : struct.serialized_parts.entrySet())
            {
-             oprot.writeString(_iter460.getKey());
-             _iter460.getValue().write(oprot);
 -            oprot.writeString(_iter440.getKey());
 -            _iter440.getValue().write(oprot);
++            oprot.writeString(_iter468.getKey());
++            _iter468.getValue().write(oprot);
            }
            oprot.writeMapEnd();
          }
@@@ -439,10 -439,10 +439,10 @@@
        TTupleProtocol oprot = (TTupleProtocol) prot;
        {
          oprot.writeI32(struct.serialized_parts.size());
-         for (Map.Entry<String, ThriftSerializedObject> _iter461 : struct.serialized_parts.entrySet())
 -        for (Map.Entry<String, ThriftSerializedObject> _iter441 : struct.serialized_parts.entrySet())
++        for (Map.Entry<String, ThriftSerializedObject> _iter469 : struct.serialized_parts.entrySet())
          {
-           oprot.writeString(_iter461.getKey());
-           _iter461.getValue().write(oprot);
 -          oprot.writeString(_iter441.getKey());
 -          _iter441.getValue().write(oprot);
++          oprot.writeString(_iter469.getKey());
++          _iter469.getValue().write(oprot);
          }
        }
      }
@@@ -451,16 -451,16 +451,16 @@@
      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 _map462 = 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<String,ThriftSerializedObject>(2*_map462.size);
-         String _key463;
-         ThriftSerializedObject _val464;
-         for (int _i465 = 0; _i465 < _map462.size; ++_i465)
 -        org.apache.thrift.protocol.TMap _map442 = 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<String,ThriftSerializedObject>(2*_map442.size);
 -        String _key443;
 -        ThriftSerializedObject _val444;
 -        for (int _i445 = 0; _i445 < _map442.size; ++_i445)
++        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<String,ThriftSerializedObject>(2*_map470.size);
++        String _key471;
++        ThriftSerializedObject _val472;
++        for (int _i473 = 0; _i473 < _map470.size; ++_i473)
          {
-           _key463 = iprot.readString();
-           _val464 = new ThriftSerializedObject();
-           _val464.read(iprot);
-           struct.serialized_parts.put(_key463, _val464);
 -          _key443 = iprot.readString();
 -          _val444 = new ThriftSerializedObject();
 -          _val444.read(iprot);
 -          struct.serialized_parts.put(_key443, _val444);
++          _key471 = iprot.readString();
++          _val472 = new ThriftSerializedObject();
++          _val472.read(iprot);
++          struct.serialized_parts.put(_key471, _val472);
          }
        }
        struct.set_serialized_parts_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
index f844023,a71d1b4..f272cd8
--- a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-5")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<NodeInfo> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo");
  
@@@ -461,13 -461,13 +461,13 @@@
            case 2: // PORT
              if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
                {
-                 org.apache.thrift.protocol.TSet _set372 = iprot.readSetBegin();
-                 struct.port = new HashSet<Long>(2*_set372.size);
-                 long _elem373;
-                 for (int _i374 = 0; _i374 < _set372.size; ++_i374)
 -                org.apache.thrift.protocol.TSet _set362 = iprot.readSetBegin();
 -                struct.port = new HashSet<Long>(2*_set362.size);
 -                long _elem363;
 -                for (int _i364 = 0; _i364 < _set362.size; ++_i364)
++                org.apache.thrift.protocol.TSet _set380 = iprot.readSetBegin();
++                struct.port = new HashSet<Long>(2*_set380.size);
++                long _elem381;
++                for (int _i382 = 0; _i382 < _set380.size; ++_i382)
                  {
-                   _elem373 = iprot.readI64();
-                   struct.port.add(_elem373);
 -                  _elem363 = iprot.readI64();
 -                  struct.port.add(_elem363);
++                  _elem381 = iprot.readI64();
++                  struct.port.add(_elem381);
                  }
                  iprot.readSetEnd();
                }
@@@ -498,9 -498,9 +498,9 @@@
          oprot.writeFieldBegin(PORT_FIELD_DESC);
          {
            oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size()));
-           for (long _iter375 : struct.port)
 -          for (long _iter365 : struct.port)
++          for (long _iter383 : struct.port)
            {
-             oprot.writeI64(_iter375);
 -            oprot.writeI64(_iter365);
++            oprot.writeI64(_iter383);
            }
            oprot.writeSetEnd();
          }
@@@ -526,9 -526,9 +526,9 @@@
        oprot.writeString(struct.node);
        {
          oprot.writeI32(struct.port.size());
-         for (long _iter376 : struct.port)
 -        for (long _iter366 : struct.port)
++        for (long _iter384 : struct.port)
          {
-           oprot.writeI64(_iter376);
 -          oprot.writeI64(_iter366);
++          oprot.writeI64(_iter384);
          }
        }
      }
@@@ -539,13 -539,13 +539,13 @@@
        struct.node = iprot.readString();
        struct.set_node_isSet(true);
        {
-         org.apache.thrift.protocol.TSet _set377 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-         struct.port = new HashSet<Long>(2*_set377.size);
-         long _elem378;
-         for (int _i379 = 0; _i379 < _set377.size; ++_i379)
 -        org.apache.thrift.protocol.TSet _set367 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
 -        struct.port = new HashSet<Long>(2*_set367.size);
 -        long _elem368;
 -        for (int _i369 = 0; _i369 < _set367.size; ++_i369)
++        org.apache.thrift.protocol.TSet _set385 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
++        struct.port = new HashSet<Long>(2*_set385.size);
++        long _elem386;
++        for (int _i387 = 0; _i387 < _set385.size; ++_i387)
          {
-           _elem378 = iprot.readI64();
-           struct.port.add(_elem378);
 -          _elem368 = iprot.readI64();
 -          struct.port.add(_elem368);
++          _elem386 = iprot.readI64();
++          struct.port.add(_elem386);
          }
        }
        struct.set_port_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
index 98d035a,f973ffc..2cc7762
--- a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
- @Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 -@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOptions, RebalanceOptions._Fields>, java.io.Serializable, Cloneable, Comparable<RebalanceOptions> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RebalanceOptions");
  
@@@ -529,15 -529,15 +529,15 @@@
            case 3: // NUM_EXECUTORS
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
-                 org.apache.thrift.protocol.TMap _map326 = iprot.readMapBegin();
-                 struct.num_executors = new HashMap<String,Integer>(2*_map326.size);
-                 String _key327;
-                 int _val328;
-                 for (int _i329 = 0; _i329 < _map326.size; ++_i329)
 -                org.apache.thrift.protocol.TMap _map316 = iprot.readMapBegin();
 -                struct.num_executors = new HashMap<String,Integer>(2*_map316.size);
 -                String _key317;
 -                int _val318;
 -                for (int _i319 = 0; _i319 < _map316.size; ++_i319)
++                org.apache.thrift.protocol.TMap _map334 = iprot.readMapBegin();
++                struct.num_executors = new HashMap<String,Integer>(2*_map334.size);
++                String _key335;
++                int _val336;
++                for (int _i337 = 0; _i337 < _map334.size; ++_i337)
                  {
-                   _key327 = iprot.readString();
-                   _val328 = iprot.readI32();
-                   struct.num_executors.put(_key327, _val328);
 -                  _key317 = iprot.readString();
 -                  _val318 = iprot.readI32();
 -                  struct.num_executors.put(_key317, _val318);
++                  _key335 = iprot.readString();
++                  _val336 = iprot.readI32();
++                  struct.num_executors.put(_key335, _val336);
                  }
                  iprot.readMapEnd();
                }
@@@ -574,10 -574,10 +574,10 @@@
            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<String, Integer> _iter330 : struct.num_executors.entrySet())
 -            for (Map.Entry<String, Integer> _iter320 : struct.num_executors.entrySet())
++            for (Map.Entry<String, Integer> _iter338 : struct.num_executors.entrySet())
              {
-               oprot.writeString(_iter330.getKey());
-               oprot.writeI32(_iter330.getValue());
 -              oprot.writeString(_iter320.getKey());
 -              oprot.writeI32(_iter320.getValue());
++              oprot.writeString(_iter338.getKey());
++              oprot.writeI32(_iter338.getValue());
              }
              oprot.writeMapEnd();
            }
@@@ -621,10 -621,10 +621,10 @@@
        if (struct.is_set_num_executors()) {
          {
            oprot.writeI32(struct.num_executors.size());
-           for (Map.Entry<String, Integer> _iter331 : struct.num_executors.entrySet())
 -          for (Map.Entry<String, Integer> _iter321 : struct.num_executors.entrySet())
++          for (Map.Entry<String, Integer> _iter339 : struct.num_executors.entrySet())
            {
-             oprot.writeString(_iter331.getKey());
-             oprot.writeI32(_iter331.getValue());
 -            oprot.writeString(_iter321.getKey());
 -            oprot.writeI32(_iter321.getValue());
++            oprot.writeString(_iter339.getKey());
++            oprot.writeI32(_iter339.getValue());
            }
          }
        }
@@@ -644,15 -644,15 +644,15 @@@
        }
        if (incoming.get(2)) {
          {
-           org.apache.thrift.protocol.TMap _map332 = 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<String,Integer>(2*_map332.size);
-           String _key333;
-           int _val334;
-           for (int _i335 = 0; _i335 < _map332.size; ++_i335)
 -          org.apache.thrift.protocol.TMap _map322 = 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<String,Integer>(2*_map322.size);
 -          String _key323;
 -          int _val324;
 -          for (int _i325 = 0; _i325 < _map322.size; ++_i325)
++          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<String,Integer>(2*_map340.size);
++          String _key341;
++          int _val342;
++          for (int _i343 = 0; _i343 < _map340.size; ++_i343)
            {
-             _key333 = iprot.readString();
-             _val334 = iprot.readI32();
-             struct.num_executors.put(_key333, _val334);
 -            _key323 = iprot.readString();
 -            _val324 = iprot.readI32();
 -            struct.num_executors.put(_key323, _val324);
++            _key341 = iprot.readString();
++            _val342 = iprot.readI32();
++            struct.num_executors.put(_key341, _val342);
            }
          }
          struct.set_num_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/0151b441/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
index 7826fe4,7826fe4..c18a7d0
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
@@@ -51,7 -51,7 +51,7 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
--@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
++@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
  public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStats._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutStats> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutStats");
  
@@@ -602,40 -602,40 +602,8 @@@
            case 1: // ACKED
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
--                org.apache.thrift.protocol.TMap _map190 = iprot.readMapBegin();
--                struct.acked = new HashMap<String,Map<String,Long>>(2*_map190.size);
--                String _key191;
--                Map<String,Long> _val192;
--                for (int _i193 = 0; _i193 < _map190.size; ++_i193)
--                {
--                  _key191 = iprot.readString();
--                  {
--                    org.apache.thrift.protocol.TMap _map194 = iprot.readMapBegin();
--                    _val192 = new HashMap<String,Long>(2*_map194.size);
--                    String _key195;
--                    long _val196;
--                    for (int _i197 = 0; _i197 < _map194.size; ++_i197)
--                    {
--                      _key195 = iprot.readString();
--                      _val196 = iprot.readI64();
--                      _val192.put(_key195, _val196);
--                    }
--                    iprot.readMapEnd();
--                  }
--                  struct.acked.put(_key191, _val192);
--                }
--                iprot.readMapEnd();
--              }
--              struct.set_acked_isSet(true);
--            } else { 
--              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
--            }
--            break;
--          case 2: // FAILED
--            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
--              {
                  org.apache.thrift.protocol.TMap _map198 = iprot.readMapBegin();
--                struct.failed = new HashMap<String,Map<String,Long>>(2*_map198.size);
++                struct.acked = new HashMap<String,Map<String,Long>>(2*_map198.size);
                  String _key199;
                  Map<String,Long> _val200;
                  for (int _i201 = 0; _i201 < _map198.size; ++_i201)
@@@ -654,39 -654,39 +622,71 @@@
                      }
                      iprot.readMapEnd();
                    }
--                  struct.failed.put(_key199, _val200);
++                  struct.acked.put(_key199, _val200);
                  }
                  iprot.readMapEnd();
                }
--              struct.set_failed_isSet(true);
++              struct.set_acked_isSet(true);
              } else { 
                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
              }
              break;
--          case 3: // COMPLETE_MS_AVG
++          case 2: // FAILED
              if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                {
                  org.apache.thrift.protocol.TMap _map206 = iprot.readMapBegin();
--                struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map206.size);
++                struct.failed = new HashMap<String,Map<String,Long>>(2*_map206.size);
                  String _key207;
--                Map<String,Double> _val208;
++                Map<String,Long> _val208;
                  for (int _i209 = 0; _i209 < _map206.size; ++_i209)
                  {
                    _key207 = iprot.readString();
                    {
                      org.apache.thrift.protocol.TMap _map210 = iprot.readMapBegin();
--                    _val208 = new HashMap<String,Double>(2*_map210.size);
++                    _val208 = new HashMap<String,Long>(2*_map210.size);
                      String _key211;
--                    double _val212;
++                    long _val212;
                      for (int _i213 = 0; _i213 < _map210.size; ++_i213)
                      {
                        _key211 = iprot.readString();
--                      _val212 = iprot.readDouble();
++                      _val212 = iprot.readI64();
                        _val208.put(_key211, _val212);
                      }
                      iprot.readMapEnd();
                    }
--                  struct.complete_ms_avg.put(_key207, _val208);
++                  struct.failed.put(_key207, _val208);
++                }
++                iprot.readMapEnd();
++              }
++              struct.set_failed_isSet(true);
++            } else { 
++              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
++            }
++            break;
++          case 3: // COMPLETE_MS_AVG
++            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
++              {
++                org.apache.thrift.protocol.TMap _map214 = iprot.readMapBegin();
++                struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map214.size);
++                String _key215;
++                Map<String,Double> _val216;
++                for (int _i217 = 0; _i217 < _map214.size; ++_i217)
++                {
++                  _key215 = iprot.readString();
++                  {
++                    org.apache.thrift.protocol.TMap _map218 = iprot.readMapBegin();
++                    _val216 = new HashMap<String,Double>(2*_map218.size);
++                    String _key219;
++                    double _val220;
++                    for (int _i221 = 0; _i221 < _map218.size; ++_i221)
++                    {
++                      _key219 = iprot.readString();
++                      _val220 = iprot.readDouble();
++                      _val216.put(_key219, _val220);
++                    }
++                    iprot.readMapEnd();
++                  }
++                  struct.complete_ms_avg.put(_key215, _val216);
                  }
                  iprot.readMapEnd();
                }
@@@ -712,15 -712,15 +712,15 @@@
          oprot.writeFieldBegin(ACKED_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
--          for (Map.Entry<String, Map<String,Long>> _iter214 : struct.acked.entrySet())
++          for (Map.Entry<String, Map<String,Long>> _iter222 : struct.acked.entrySet())
            {
--            oprot.writeString(_iter214.getKey());
++            oprot.writeString(_iter222.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter214.getValue().size()));
--              for (Map.Entry<String, Long> _iter215 : _iter214.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter222.getValue().size()));
++              for (Map.Entry<String, Long> _iter223 : _iter222.getValue().entrySet())
                {
--                oprot.writeString(_iter215.getKey());
--                oprot.writeI64(_iter215.getValue());
++                oprot.writeString(_iter223.getKey());
++                oprot.writeI64(_iter223.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -733,15 -733,15 +733,15 @@@
          oprot.writeFieldBegin(FAILED_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
--          for (Map.Entry<String, Map<String,Long>> _iter216 : struct.failed.entrySet())
++          for (Map.Entry<String, Map<String,Long>> _iter224 : struct.failed.entrySet())
            {
--            oprot.writeString(_iter216.getKey());
++            oprot.writeString(_iter224.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter216.getValue().size()));
--              for (Map.Entry<String, Long> _iter217 : _iter216.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter224.getValue().size()));
++              for (Map.Entry<String, Long> _iter225 : _iter224.getValue().entrySet())
                {
--                oprot.writeString(_iter217.getKey());
--                oprot.writeI64(_iter217.getValue());
++                oprot.writeString(_iter225.getKey());
++                oprot.writeI64(_iter225.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -754,15 -754,15 +754,15 @@@
          oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC);
          {
            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.complete_ms_avg.size()));
--          for (Map.Entry<String, Map<String,Double>> _iter218 : struct.complete_ms_avg.entrySet())
++          for (Map.Entry<String, Map<String,Double>> _iter226 : struct.complete_ms_avg.entrySet())
            {
--            oprot.writeString(_iter218.getKey());
++            oprot.writeString(_iter226.getKey());
              {
--              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, _iter218.getValue().size()));
--              for (Map.Entry<String, Double> _iter219 : _iter218.getValue().entrySet())
++              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, _iter226.getValue().size()));
++              for (Map.Entry<String, Double> _iter227 : _iter226.getValue().entrySet())
                {
--                oprot.writeString(_iter219.getKey());
--                oprot.writeDouble(_iter219.getValue());
++                oprot.writeString(_iter227.getKey());
++                oprot.writeDouble(_iter227.getValue());
                }
                oprot.writeMapEnd();
              }
@@@ -790,45 -790,45 +790,45 @@@
        TTupleProtocol oprot = (TTupleProtocol) prot;
        {
          oprot.writeI32(struct.acked.size());
--        for (Map.Entry<String, Map<String,Long>> _iter220 : struct.acked.entrySet())
++        for (Map.Entry<String, Map<String,Long>> _iter228 : struct.acked.entrySet())
          {
--          oprot.writeString(_iter220.getKey());
++          oprot.writeString(_iter228.getKey());
            {
--            oprot.writeI32(_iter220.getValue().size());
--            for (Map.Entry<String, Long> _iter221 : _iter220.getValue().entrySet())
++            oprot.writeI32(_iter228.getValue().size());
++            for (Map.Entry<String, Long> _iter229 : _iter228.getValue().entrySet())
              {
--              oprot.writeString(_iter221.getKey());
--              oprot.writeI64(_iter221.getValue());
++              oprot.writeString(_iter229.getKey());
++              oprot.writeI64(_iter229.getValue());
              }
            }
          }
        }
        {
          oprot.writeI32(struct.failed.size());
--        for (Map.Entry<String, Map<String,Long>> _iter222 : struct.failed.entrySet())
++        for (Map.Entry<String, Map<String,Long>> _iter230 : struct.failed.entrySet())
          {
--          oprot.writeString(_iter222.getKey());
++          oprot.writeString(_iter230.getKey());
            {
--            oprot.writeI32(_iter222.getValue().size());
--            for (Map.Entry<String, Long> _iter223 : _iter222.getValue().entrySet())
++            oprot.writeI32(_iter230.getValue().size());
++            for (Map.Entry<String, Long> _iter231 : _iter230.getValue().entrySet())
              {
--              oprot.writeString(_iter223.getKey());
--              oprot.writeI64(_iter223.getValue());
++              oprot.writeString(_iter231.getKey());
++              oprot.writeI64(_iter231.getValue());
              }
            }
          }
        }
        {
          oprot.writeI32(struct.complete_ms_avg.size());
--        for (Map.Entry<String, Map<String,Double>> _iter224 : struct.complete_ms_avg.entrySet())
++        for (Map.Entry<String, Map<String,Double>> _iter232 : struct.complete_ms_avg.entrySet())
          {
--          oprot.writeString(_iter224.getKey());
++          oprot.writeString(_iter232.getKey());
            {
--            oprot.writeI32(_iter224.getValue().size());
--            for (Map.Entry<String, Double> _iter225 : _iter224.getValue().entrySet())
++            oprot.writeI32(_iter232.getValue().size());
++            for (Map.Entry<String, Double> _iter233 : _iter232.getValue().entrySet())
              {
--              oprot.writeString(_iter225.getKey());
--              oprot.writeDouble(_iter225.getValue());
++              oprot.writeString(_iter233.getKey());
++              oprot.writeDouble(_iter233.getValue());
              }
            }
          }
@@@ -839,32 -839,32 +839,8 @@@
      public void read(org.apache.thrift.protocol.TProtocol prot, SpoutStats struct) throws org.apache.thrift.TException {
        TTupleProtocol iprot = (TTupleProtocol) prot;
        {
--        org.apache.thrift.protocol.TMap _map226 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.acked = new HashMap<String,Map<String,Long>>(2*_map226.size);
--        String _key227;
--        Map<String,Long> _val228;
--        for (int _i229 = 0; _i229 < _map226.size; ++_i229)
--        {
--          _key227 = iprot.readString();
--          {
--            org.apache.thrift.protocol.TMap _map230 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
--            _val228 = new HashMap<String,Long>(2*_map230.size);
--            String _key231;
--            long _val232;
--            for (int _i233 = 0; _i233 < _map230.size; ++_i233)
--            {
--              _key231 = iprot.readString();
--              _val232 = iprot.readI64();
--              _val228.put(_key231, _val232);
--            }
--          }
--          struct.acked.put(_key227, _val228);
--        }
--      }
--      struct.set_acked_isSet(true);
--      {
          org.apache.thrift.protocol.TMap _map234 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.failed = new HashMap<String,Map<String,Long>>(2*_map234.size);
++        struct.acked = new HashMap<String,Map<String,Long>>(2*_map234.size);
          String _key235;
          Map<String,Long> _val236;
          for (int _i237 = 0; _i237 < _map234.size; ++_i237)
@@@ -882,31 -882,31 +858,55 @@@
                _val236.put(_key239, _val240);
              }
            }
--          struct.failed.put(_key235, _val236);
++          struct.acked.put(_key235, _val236);
          }
        }
--      struct.set_failed_isSet(true);
++      struct.set_acked_isSet(true);
        {
          org.apache.thrift.protocol.TMap _map242 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
--        struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map242.size);
++        struct.failed = new HashMap<String,Map<String,Long>>(2*_map242.size);
          String _key243;
--        Map<String,Double> _val244;
++        Map<String,Long> _val244;
          for (int _i245 = 0; _i245 < _map242.size; ++_i245)
          {
            _key243 = iprot.readString();
            {
--            org.apache.thrift.protocol.TMap _map246 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
--            _val244 = new HashMap<String,Double>(2*_map246.size);
++            org.apache.thrift.protocol.TMap _map246 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
++            _val244 = new HashMap<String,Long>(2*_map246.size);
              String _key247;
--            double _val248;
++            long _val248;
              for (int _i249 = 0; _i249 < _map246.size; ++_i249)
              {
                _key247 = iprot.readString();
--              _val248 = iprot.readDouble();
++              _val248 = iprot.readI64();
                _val244.put(_key247, _val248);
              }
            }
--          struct.complete_ms_avg.put(_key243, _val244);
++          struct.failed.put(_key243, _val244);
++        }
++      }
++      struct.set_failed_isSet(true);
++      {
++        org.apache.thrift.protocol.TMap _map250 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
++        struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map250.size);
++        String _key251;
++        Map<String,Double> _val252;
++        for (int _i253 = 0; _i253 < _map250.size; ++_i253)
++        {
++          _key251 = iprot.readString();
++          {
++            org.apache.thrift.protocol.TMap _map254 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
++            _val252 = new HashMap<String,Double>(2*_map254.size);
++            String _key255;
++            double _val256;
++            for (int _i257 = 0; _i257 < _map254.size; ++_i257)
++            {
++              _key255 = iprot.readString();
++              _val256 = iprot.readDouble();
++              _val252.put(_key255, _val256);
++            }
++          }
++          struct.complete_ms_avg.put(_key251, _val252);
          }
        }
        struct.set_complete_ms_avg_isSet(true);


[14/24] storm git commit: Add support for sampling percentage

Posted by sr...@apache.org.
Add support for sampling percentage

1. Added DebugOptions to store component level flag + sampling pct
2. Added extra sampling pct param in nimbus debug api
3. UI changes for passing sampling pct.


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

Branch: refs/heads/master
Commit: f9a4b0100de2e2417296c094caf7c077e5755a0d
Parents: 788db9d
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Wed Aug 26 13:50:19 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Wed Aug 26 13:50:19 2015 +0530

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   |   2 +-
 storm-core/src/clj/backtype/storm/converter.clj |  21 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |  34 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  17 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  25 +-
 .../backtype/storm/generated/DebugOptions.java  | 506 +++++++++++++++++++
 .../jvm/backtype/storm/generated/Nimbus.java    | 133 ++++-
 .../jvm/backtype/storm/generated/StormBase.java |  55 +-
 .../backtype/storm/generated/TopologyInfo.java  |  53 +-
 storm-core/src/py/storm/Nimbus-remote           |   8 +-
 storm-core/src/py/storm/Nimbus.py               |  32 +-
 storm-core/src/py/storm/ttypes.py               |  96 +++-
 storm-core/src/storm.thrift                     |  13 +-
 storm-core/src/ui/public/js/script.js           |  49 +-
 .../templates/component-page-template.html      |   4 +-
 .../templates/topology-page-template.html       |  12 +-
 storm-core/src/ui/public/topology.html          |   3 +-
 17 files changed, 926 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index d96cb98..ff1bb9c 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -413,7 +413,7 @@
               executors (:component->executors base)
               component->debug (:component->debug base)
               new-elems (update new-elems :component->executors (partial merge executors))
-              new-elems (update new-elems :component->debug (partial merge component->debug))]
+              new-elems (update new-elems :component->debug (partial merge-with merge component->debug))]
           (set-data cluster-state (storm-path storm-id)
                     (-> base
                         (merge new-elems)

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index dbda8a9..b3d0cc6 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -15,7 +15,8 @@
 ;; limitations under the License.
 (ns backtype.storm.converter
   (:import [backtype.storm.generated SupervisorInfo NodeInfo Assignment
-            StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions TopologyActionOptions])
+            StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
+            TopologyActionOptions DebugOptions])
   (:use [backtype.storm util stats log])
   (:require [backtype.storm.daemon [common :as common]]))
 
@@ -142,6 +143,19 @@
              (clojurify-rebalance-options
                (.get_rebalance_options topology-action-options))))))
 
+(defn clojurify-debugoptions [^DebugOptions options]
+  (if options
+    {
+      :enable (.is_enable options)
+      :samplingpct (.get_samplingpct options)
+      }
+    ))
+
+(defn thriftify-debugoptions [options]
+  (doto (DebugOptions.)
+    (.set_enable (if options (:enable options) false))
+    (.set_samplingpct (if options (:samplingpct options) 0))))
+
 (defn thriftify-storm-base [storm-base]
   (doto (StormBase.)
     (.set_name (:storm-name storm-base))
@@ -152,7 +166,7 @@
     (.set_owner (:owner storm-base))
     (.set_topology_action_options (thriftify-topology-action-options storm-base))
     (.set_prev_status (convert-to-status-from-symbol (:prev-status storm-base)))
-    (.set_component_debug (map-val boolean (:component->debug storm-base)))))
+    (.set_component_debug (map-val thriftify-debugoptions (:component->debug storm-base)))))
 
 (defn clojurify-storm-base [^StormBase storm-base]
   (if storm-base
@@ -165,7 +179,7 @@
       (.get_owner storm-base)
       (clojurify-topology-action-options (.get_topology_action_options storm-base))
       (convert-to-symbol-from-status (.get_prev_status storm-base))
-      (into {} (.get_component_debug storm-base)))))
+      (map-val clojurify-debugoptions (.get_component_debug storm-base)))))
 
 (defn thriftify-stats [stats]
   (if stats
@@ -222,4 +236,3 @@
     (into {} (.get_creds credentials))
     nil
     ))
-

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index 593ea0e..6a6d7f0 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -446,6 +446,21 @@
     ret
     ))
 
+;; Send sampled data to the eventlogger if the global or component level
+;; debug flag is set (via nimbus api).
+(defn send-to-eventlogger [executor-data task-data values overflow-buffer component-id random]
+    (let [c->d @(:storm-component->debug-atom executor-data)
+          options (get c->d component-id (get c->d (:storm-id executor-data)))
+          spct    (if (and (not-nil? options) (:enable options)) (:samplingpct options) 0)]
+      ;; the thread's initialized random number generator is used to generate
+      ;; uniformily distributed random numbers.
+      (if (and (> spct 0) (< (* 100 (.nextDouble random)) spct))
+        (task/send-unanchored
+          task-data
+          EVENTLOGGER-STREAM-ID
+          [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
+          overflow-buffer))))
+
 (defmethod mk-threads :spout [executor-data task-datas initial-credentials]
   (let [{:keys [storm-conf component-id worker-context transfer-fn report-error sampler open-or-prepare-was-called?]} executor-data
         ^ISpoutWaitStrategy spout-wait-strategy (init-spout-wait-strategy storm-conf)
@@ -533,16 +548,8 @@
                                                                         out-tuple
                                                                         overflow-buffer)
                                                            ))
-                                         ;; Send data to the eventlogger if event logging is enabled in conf
-                                         ;; and the global or component level debug flag is set (via nimbus api).
                                          (if has-eventloggers?
-                                           (let [c->d @(:storm-component->debug-atom executor-data)]
-                                             (if (get c->d component-id (get c->d (:storm-id executor-data) false))
-                                               (task/send-unanchored
-                                                 task-data
-                                                 EVENTLOGGER-STREAM-ID
-                                                 [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
-                                                 overflow-buffer))))
+                                           (send-to-eventlogger executor-data task-data values overflow-buffer component-id rand))
                                          (if (and rooted?
                                                   (not (.isEmpty out-ids)))
                                            (do
@@ -749,15 +756,8 @@
                                                                                stream
                                                                                (MessageId/makeId anchors-to-ids))
                                                                    overflow-buffer)))
-                                    ;; Send data to the eventlogger if event logging is enabled in conf
-                                    ;; and the global or component level debug flag is set (via nimbus api).
                                     (if has-eventloggers?
-                                      (let [c->d @(:storm-component->debug-atom executor-data)]
-                                        (if (get c->d component-id (get c->d (:storm-id executor-data) false))
-                                          (task/send-unanchored task-data
-                                            EVENTLOGGER-STREAM-ID
-                                            [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
-                                            overflow-buffer))))
+                                      (send-to-eventlogger executor-data task-data values overflow-buffer component-id rand))
                                     (or out-tasks [])))]]
           (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
           (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials)) 

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 67ad239..e20f31b 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -35,7 +35,7 @@
             ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice])
   (:import [backtype.storm.daemon Shutdownable])
   (:use [backtype.storm util config log timer])
-  (:require [backtype.storm [cluster :as cluster] [stats :as stats]])
+  (:require [backtype.storm [cluster :as cluster] [stats :as stats] [converter :as converter]])
   (:require [clojure.set :as set])
   (:import [backtype.storm.daemon.common StormBase Assignment])
   (:use [backtype.storm.daemon common])
@@ -1161,15 +1161,21 @@
           (check-authorization! nimbus storm-name topology-conf "deactivate"))
         (transition-name! nimbus storm-name :inactivate true))
 
-      (debug [this storm-name component-id enable?]
+      (debug [this storm-name component-id enable? samplingPct]
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
               topology-conf (try-read-storm-conf conf storm-id)
-              storm-base-updates (assoc {} :component->debug (if (empty? component-id) {storm-id enable?} {component-id enable?}))]
+              ;; make sure samplingPct is within bounds.
+              spct (Math/max (Math/min samplingPct 100.0) 0.0)
+              ;; while disabling we retain the sampling pct.
+              debug-options (if enable? {:enable enable? :samplingpct spct} {:enable enable?})
+              storm-base-updates (assoc {} :component->debug (if (empty? component-id)
+                                                               {storm-id debug-options}
+                                                               {component-id debug-options}))]
           (check-authorization! nimbus storm-name topology-conf "debug")
           (when-not storm-id
             (throw (NotAliveException. storm-name)))
-          (log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "'"
+          (log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "' sampling pct '" spct "'"
             (if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
           (locking (:submit-lock nimbus)
             (.update-storm! storm-cluster-state storm-id storm-base-updates))))
@@ -1356,7 +1362,8 @@
                            )]
             (when-let [owner (:owner base)] (.set_owner topo-info owner))
             (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))
-            (when-let [component->debug (:component->debug base)] (.set_component_debug topo-info (map-val boolean component->debug)))
+            (when-let [component->debug (:component->debug base)]
+              (.set_component_debug topo-info (map-val converter/thriftify-debugoptions component->debug)))
             topo-info
           ))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index d8fe30f..8f66147 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -31,7 +31,7 @@
             ExecutorStats ExecutorSummary ExecutorInfo TopologyInfo SpoutStats BoltStats
             ErrorInfo ClusterSummary SupervisorSummary TopologySummary
             Nimbus$Client StormTopology GlobalStreamId RebalanceOptions
-            KillOptions GetInfoOptions NumErrorsChoice])
+            KillOptions GetInfoOptions NumErrorsChoice DebugOptions])
   (:import [backtype.storm.security.auth AuthUtils ReqContext])
   (:import [backtype.storm.generated AuthorizationException])
   (:import [backtype.storm.security.auth AuthUtils])
@@ -716,7 +716,7 @@
         workers (set (for [^ExecutorSummary e executors]
                        [(.get_host e) (.get_port e)]))
         topology-id (.get_id summ)
-        component->debug (.get_component_debug summ)]
+        debug-options (get (.get_component_debug summ) topology-id)]
       {"id" topology-id
        "encodedId" (url-encode (.get_id summ))
        "owner" (.get_owner summ)
@@ -727,7 +727,8 @@
        "workersTotal" (count workers)
        "executorsTotal" (count executors)
        "schedulerInfo" (.get_sched_status summ)
-       "debug" (get component->debug topology-id false)}))
+       "debug" (if (not-nil? debug-options) (.is_enable debug-options) false)
+       "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 100)}))
 
 (defn spout-summary-json [topology-id id stats window]
   (let [times (stats-times (:emitted stats))
@@ -937,7 +938,8 @@
           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)]
+          component->debug (.get_component_debug summ)
+          debug-options (get component->debug component (get component->debug topology-id))]
       (merge
         {"user" user
          "id" component
@@ -951,7 +953,8 @@
          "window" window
          "componentType" (name type)
          "windowHint" (window-hint window)
-         "debug" (get component->debug component (get component->debug topology-id false))
+         "debug" (if (not-nil? debug-options) (.is_enable debug-options) false)
+         "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 100)
          "eventLogLink" (event-log-link topology-id summ topology component secure?)}
        spec errors))))
 
@@ -1042,7 +1045,7 @@
         (.deactivate nimbus name)
         (log-message "Deactivating topology '" name "'")))
     (json-response (topology-op-response id "deactivate") (m "callback")))
-  (POST "/api/v1/topology/:id/debug/:action" [:as {:keys [cookies servlet-request]} id action & m]
+  (POST "/api/v1/topology/:id/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id action spct & m]
     (assert-authorized-user servlet-request "debug" (topology-config id))
     (with-nimbus nimbus
       (let [tplg (->> (doto
@@ -1051,10 +1054,10 @@
                    (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
             name (.get_name tplg)
             enable? (= "enable" action)]
-        (.debug nimbus name "" enable?)
-        (log-message "Debug topology [" name "] action [" action "]")))
+        (.debug nimbus name "" enable? (Integer/parseInt spct))
+        (log-message "Debug topology [" name "] action [" action "] sampling pct [" spct "]")))
     (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
-  (POST "/api/v1/topology/:id/component/:component/debug/:action" [:as {:keys [cookies servlet-request]} id component action & m]
+  (POST "/api/v1/topology/:id/component/:component/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id component action spct & m]
     (assert-authorized-user servlet-request "debug" (topology-config id))
     (with-nimbus nimbus
       (let [tplg (->> (doto
@@ -1063,8 +1066,8 @@
                    (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
             name (.get_name tplg)
             enable? (= "enable" action)]
-        (.debug nimbus name component enable?)
-        (log-message "Debug topology [" name "] component [" component "] action [" action "]")))
+        (.debug nimbus name component enable? (Integer/parseInt spct))
+        (log-message "Debug topology [" name "] component [" component "] action [" action "] sampling pct [" spct "]")))
     (json-response (component-op-response id component (str "/debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
     (assert-authorized-user servlet-request "rebalance" (topology-config id))

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
new file mode 100644
index 0000000..60c0e24
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
@@ -0,0 +1,506 @@
+/**
+ * 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-8-24")
+public class DebugOptions implements org.apache.thrift.TBase<DebugOptions, DebugOptions._Fields>, java.io.Serializable, Cloneable, Comparable<DebugOptions> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DebugOptions");
+
+  private static final org.apache.thrift.protocol.TField ENABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("enable", org.apache.thrift.protocol.TType.BOOL, (short)1);
+  private static final org.apache.thrift.protocol.TField SAMPLINGPCT_FIELD_DESC = new org.apache.thrift.protocol.TField("samplingpct", org.apache.thrift.protocol.TType.DOUBLE, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new DebugOptionsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new DebugOptionsTupleSchemeFactory());
+  }
+
+  private boolean enable; // optional
+  private double samplingpct; // 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 {
+    ENABLE((short)1, "enable"),
+    SAMPLINGPCT((short)2, "samplingpct");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    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: // ENABLE
+          return ENABLE;
+        case 2: // SAMPLINGPCT
+          return SAMPLINGPCT;
+        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 __ENABLE_ISSET_ID = 0;
+  private static final int __SAMPLINGPCT_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.ENABLE,_Fields.SAMPLINGPCT};
+  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.ENABLE, new org.apache.thrift.meta_data.FieldMetaData("enable", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.SAMPLINGPCT, new org.apache.thrift.meta_data.FieldMetaData("samplingpct", 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(DebugOptions.class, metaDataMap);
+  }
+
+  public DebugOptions() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public DebugOptions(DebugOptions other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.enable = other.enable;
+    this.samplingpct = other.samplingpct;
+  }
+
+  public DebugOptions deepCopy() {
+    return new DebugOptions(this);
+  }
+
+  @Override
+  public void clear() {
+    set_enable_isSet(false);
+    this.enable = false;
+    set_samplingpct_isSet(false);
+    this.samplingpct = 0.0;
+  }
+
+  public boolean is_enable() {
+    return this.enable;
+  }
+
+  public void set_enable(boolean enable) {
+    this.enable = enable;
+    set_enable_isSet(true);
+  }
+
+  public void unset_enable() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ENABLE_ISSET_ID);
+  }
+
+  /** Returns true if field enable is set (has been assigned a value) and false otherwise */
+  public boolean is_set_enable() {
+    return EncodingUtils.testBit(__isset_bitfield, __ENABLE_ISSET_ID);
+  }
+
+  public void set_enable_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ENABLE_ISSET_ID, value);
+  }
+
+  public double get_samplingpct() {
+    return this.samplingpct;
+  }
+
+  public void set_samplingpct(double samplingpct) {
+    this.samplingpct = samplingpct;
+    set_samplingpct_isSet(true);
+  }
+
+  public void unset_samplingpct() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SAMPLINGPCT_ISSET_ID);
+  }
+
+  /** Returns true if field samplingpct is set (has been assigned a value) and false otherwise */
+  public boolean is_set_samplingpct() {
+    return EncodingUtils.testBit(__isset_bitfield, __SAMPLINGPCT_ISSET_ID);
+  }
+
+  public void set_samplingpct_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SAMPLINGPCT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ENABLE:
+      if (value == null) {
+        unset_enable();
+      } else {
+        set_enable((Boolean)value);
+      }
+      break;
+
+    case SAMPLINGPCT:
+      if (value == null) {
+        unset_samplingpct();
+      } else {
+        set_samplingpct((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ENABLE:
+      return Boolean.valueOf(is_enable());
+
+    case SAMPLINGPCT:
+      return Double.valueOf(get_samplingpct());
+
+    }
+    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 ENABLE:
+      return is_set_enable();
+    case SAMPLINGPCT:
+      return is_set_samplingpct();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof DebugOptions)
+      return this.equals((DebugOptions)that);
+    return false;
+  }
+
+  public boolean equals(DebugOptions that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_enable = true && this.is_set_enable();
+    boolean that_present_enable = true && that.is_set_enable();
+    if (this_present_enable || that_present_enable) {
+      if (!(this_present_enable && that_present_enable))
+        return false;
+      if (this.enable != that.enable)
+        return false;
+    }
+
+    boolean this_present_samplingpct = true && this.is_set_samplingpct();
+    boolean that_present_samplingpct = true && that.is_set_samplingpct();
+    if (this_present_samplingpct || that_present_samplingpct) {
+      if (!(this_present_samplingpct && that_present_samplingpct))
+        return false;
+      if (this.samplingpct != that.samplingpct)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_enable = true && (is_set_enable());
+    list.add(present_enable);
+    if (present_enable)
+      list.add(enable);
+
+    boolean present_samplingpct = true && (is_set_samplingpct());
+    list.add(present_samplingpct);
+    if (present_samplingpct)
+      list.add(samplingpct);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(DebugOptions other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_enable()).compareTo(other.is_set_enable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_enable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enable, other.enable);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_samplingpct()).compareTo(other.is_set_samplingpct());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_samplingpct()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.samplingpct, other.samplingpct);
+      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("DebugOptions(");
+    boolean first = true;
+
+    if (is_set_enable()) {
+      sb.append("enable:");
+      sb.append(this.enable);
+      first = false;
+    }
+    if (is_set_samplingpct()) {
+      if (!first) sb.append(", ");
+      sb.append("samplingpct:");
+      sb.append(this.samplingpct);
+      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 DebugOptionsStandardSchemeFactory implements SchemeFactory {
+    public DebugOptionsStandardScheme getScheme() {
+      return new DebugOptionsStandardScheme();
+    }
+  }
+
+  private static class DebugOptionsStandardScheme extends StandardScheme<DebugOptions> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, DebugOptions 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: // ENABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.enable = iprot.readBool();
+              struct.set_enable_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SAMPLINGPCT
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.samplingpct = iprot.readDouble();
+              struct.set_samplingpct_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, DebugOptions struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_enable()) {
+        oprot.writeFieldBegin(ENABLE_FIELD_DESC);
+        oprot.writeBool(struct.enable);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_samplingpct()) {
+        oprot.writeFieldBegin(SAMPLINGPCT_FIELD_DESC);
+        oprot.writeDouble(struct.samplingpct);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class DebugOptionsTupleSchemeFactory implements SchemeFactory {
+    public DebugOptionsTupleScheme getScheme() {
+      return new DebugOptionsTupleScheme();
+    }
+  }
+
+  private static class DebugOptionsTupleScheme extends TupleScheme<DebugOptions> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, DebugOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_enable()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_samplingpct()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_enable()) {
+        oprot.writeBool(struct.enable);
+      }
+      if (struct.is_set_samplingpct()) {
+        oprot.writeDouble(struct.samplingpct);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, DebugOptions struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.enable = iprot.readBool();
+        struct.set_enable_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.samplingpct = iprot.readDouble();
+        struct.set_samplingpct_isSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index 830d95e..c5d5c0a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
@@ -74,12 +74,15 @@ public class Nimbus {
      * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
      * and if null or empty, the debug flag will apply to the entire topology.
      * 
+     * If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
+     * 
      * 
      * @param name
      * @param component
      * @param enable
+     * @param samplingPercentage
      */
-    public void debug(String name, String component, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+    public void debug(String name, String component, boolean enable, double samplingPercentage) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
@@ -135,7 +138,7 @@ public class Nimbus {
 
     public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void debug(String name, String component, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+    public void debug(String name, String component, boolean enable, double samplingPercentage, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
@@ -385,18 +388,19 @@ public class Nimbus {
       return;
     }
 
-    public void debug(String name, String component, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    public void debug(String name, String component, boolean enable, double samplingPercentage) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
-      send_debug(name, component, enable);
+      send_debug(name, component, enable, samplingPercentage);
       recv_debug();
     }
 
-    public void send_debug(String name, String component, boolean enable) throws org.apache.thrift.TException
+    public void send_debug(String name, String component, boolean enable, double samplingPercentage) throws org.apache.thrift.TException
     {
       debug_args args = new debug_args();
       args.set_name(name);
       args.set_component(component);
       args.set_enable(enable);
+      args.set_samplingPercentage(samplingPercentage);
       sendBase("debug", args);
     }
 
@@ -1032,9 +1036,9 @@ public class Nimbus {
       }
     }
 
-    public void debug(String name, String component, boolean enable, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+    public void debug(String name, String component, boolean enable, double samplingPercentage, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
-      debug_call method_call = new debug_call(name, component, enable, resultHandler, this, ___protocolFactory, ___transport);
+      debug_call method_call = new debug_call(name, component, enable, samplingPercentage, resultHandler, this, ___protocolFactory, ___transport);
       this.___currentMethod = method_call;
       ___manager.call(method_call);
     }
@@ -1043,11 +1047,13 @@ public class Nimbus {
       private String name;
       private String component;
       private boolean enable;
-      public debug_call(String name, String component, boolean enable, 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 {
+      private double samplingPercentage;
+      public debug_call(String name, String component, boolean enable, double samplingPercentage, 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.name = name;
         this.component = component;
         this.enable = enable;
+        this.samplingPercentage = samplingPercentage;
       }
 
       public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
@@ -1056,6 +1062,7 @@ public class Nimbus {
         args.set_name(name);
         args.set_component(component);
         args.set_enable(enable);
+        args.set_samplingPercentage(samplingPercentage);
         args.write(prot);
         prot.writeMessageEnd();
       }
@@ -1727,7 +1734,7 @@ public class Nimbus {
       public debug_result getResult(I iface, debug_args args) throws org.apache.thrift.TException {
         debug_result result = new debug_result();
         try {
-          iface.debug(args.name, args.component, args.enable);
+          iface.debug(args.name, args.component, args.enable, args.samplingPercentage);
         } catch (NotAliveException e) {
           result.e = e;
         } catch (AuthorizationException aze) {
@@ -2599,7 +2606,7 @@ public class Nimbus {
       }
 
       public void start(I iface, debug_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.debug(args.name, args.component, args.enable,resultHandler);
+        iface.debug(args.name, args.component, args.enable, args.samplingPercentage,resultHandler);
       }
     }
 
@@ -10431,6 +10438,7 @@ public class Nimbus {
     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)1);
     private static final org.apache.thrift.protocol.TField COMPONENT_FIELD_DESC = new org.apache.thrift.protocol.TField("component", org.apache.thrift.protocol.TType.STRING, (short)2);
     private static final org.apache.thrift.protocol.TField ENABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("enable", org.apache.thrift.protocol.TType.BOOL, (short)3);
+    private static final org.apache.thrift.protocol.TField SAMPLING_PERCENTAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("samplingPercentage", org.apache.thrift.protocol.TType.DOUBLE, (short)4);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
@@ -10441,12 +10449,14 @@ public class Nimbus {
     private String name; // required
     private String component; // required
     private boolean enable; // required
+    private double samplingPercentage; // 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 {
       NAME((short)1, "name"),
       COMPONENT((short)2, "component"),
-      ENABLE((short)3, "enable");
+      ENABLE((short)3, "enable"),
+      SAMPLING_PERCENTAGE((short)4, "samplingPercentage");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -10467,6 +10477,8 @@ public class Nimbus {
             return COMPONENT;
           case 3: // ENABLE
             return ENABLE;
+          case 4: // SAMPLING_PERCENTAGE
+            return SAMPLING_PERCENTAGE;
           default:
             return null;
         }
@@ -10508,6 +10520,7 @@ public class Nimbus {
 
     // isset id assignments
     private static final int __ENABLE_ISSET_ID = 0;
+    private static final int __SAMPLINGPERCENTAGE_ISSET_ID = 1;
     private byte __isset_bitfield = 0;
     public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
     static {
@@ -10518,6 +10531,8 @@ public class Nimbus {
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
       tmpMap.put(_Fields.ENABLE, new org.apache.thrift.meta_data.FieldMetaData("enable", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      tmpMap.put(_Fields.SAMPLING_PERCENTAGE, new org.apache.thrift.meta_data.FieldMetaData("samplingPercentage", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
       org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(debug_args.class, metaDataMap);
     }
@@ -10528,13 +10543,16 @@ public class Nimbus {
     public debug_args(
       String name,
       String component,
-      boolean enable)
+      boolean enable,
+      double samplingPercentage)
     {
       this();
       this.name = name;
       this.component = component;
       this.enable = enable;
       set_enable_isSet(true);
+      this.samplingPercentage = samplingPercentage;
+      set_samplingPercentage_isSet(true);
     }
 
     /**
@@ -10549,6 +10567,7 @@ public class Nimbus {
         this.component = other.component;
       }
       this.enable = other.enable;
+      this.samplingPercentage = other.samplingPercentage;
     }
 
     public debug_args deepCopy() {
@@ -10561,6 +10580,8 @@ public class Nimbus {
       this.component = null;
       set_enable_isSet(false);
       this.enable = false;
+      set_samplingPercentage_isSet(false);
+      this.samplingPercentage = 0.0;
     }
 
     public String get_name() {
@@ -10631,6 +10652,28 @@ public class Nimbus {
       __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ENABLE_ISSET_ID, value);
     }
 
+    public double get_samplingPercentage() {
+      return this.samplingPercentage;
+    }
+
+    public void set_samplingPercentage(double samplingPercentage) {
+      this.samplingPercentage = samplingPercentage;
+      set_samplingPercentage_isSet(true);
+    }
+
+    public void unset_samplingPercentage() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SAMPLINGPERCENTAGE_ISSET_ID);
+    }
+
+    /** Returns true if field samplingPercentage is set (has been assigned a value) and false otherwise */
+    public boolean is_set_samplingPercentage() {
+      return EncodingUtils.testBit(__isset_bitfield, __SAMPLINGPERCENTAGE_ISSET_ID);
+    }
+
+    public void set_samplingPercentage_isSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SAMPLINGPERCENTAGE_ISSET_ID, value);
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
       case NAME:
@@ -10657,6 +10700,14 @@ public class Nimbus {
         }
         break;
 
+      case SAMPLING_PERCENTAGE:
+        if (value == null) {
+          unset_samplingPercentage();
+        } else {
+          set_samplingPercentage((Double)value);
+        }
+        break;
+
       }
     }
 
@@ -10671,6 +10722,9 @@ public class Nimbus {
       case ENABLE:
         return Boolean.valueOf(is_enable());
 
+      case SAMPLING_PERCENTAGE:
+        return Double.valueOf(get_samplingPercentage());
+
       }
       throw new IllegalStateException();
     }
@@ -10688,6 +10742,8 @@ public class Nimbus {
         return is_set_component();
       case ENABLE:
         return is_set_enable();
+      case SAMPLING_PERCENTAGE:
+        return is_set_samplingPercentage();
       }
       throw new IllegalStateException();
     }
@@ -10732,6 +10788,15 @@ public class Nimbus {
           return false;
       }
 
+      boolean this_present_samplingPercentage = true;
+      boolean that_present_samplingPercentage = true;
+      if (this_present_samplingPercentage || that_present_samplingPercentage) {
+        if (!(this_present_samplingPercentage && that_present_samplingPercentage))
+          return false;
+        if (this.samplingPercentage != that.samplingPercentage)
+          return false;
+      }
+
       return true;
     }
 
@@ -10754,6 +10819,11 @@ public class Nimbus {
       if (present_enable)
         list.add(enable);
 
+      boolean present_samplingPercentage = true;
+      list.add(present_samplingPercentage);
+      if (present_samplingPercentage)
+        list.add(samplingPercentage);
+
       return list.hashCode();
     }
 
@@ -10795,6 +10865,16 @@ public class Nimbus {
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(is_set_samplingPercentage()).compareTo(other.is_set_samplingPercentage());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_samplingPercentage()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.samplingPercentage, other.samplingPercentage);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -10834,6 +10914,10 @@ public class Nimbus {
       sb.append("enable:");
       sb.append(this.enable);
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("samplingPercentage:");
+      sb.append(this.samplingPercentage);
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -10903,6 +10987,14 @@ public class Nimbus {
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 4: // SAMPLING_PERCENTAGE
+              if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+                struct.samplingPercentage = iprot.readDouble();
+                struct.set_samplingPercentage_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -10929,6 +11021,9 @@ public class Nimbus {
         oprot.writeFieldBegin(ENABLE_FIELD_DESC);
         oprot.writeBool(struct.enable);
         oprot.writeFieldEnd();
+        oprot.writeFieldBegin(SAMPLING_PERCENTAGE_FIELD_DESC);
+        oprot.writeDouble(struct.samplingPercentage);
+        oprot.writeFieldEnd();
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
@@ -10956,7 +11051,10 @@ public class Nimbus {
         if (struct.is_set_enable()) {
           optionals.set(2);
         }
-        oprot.writeBitSet(optionals, 3);
+        if (struct.is_set_samplingPercentage()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
         if (struct.is_set_name()) {
           oprot.writeString(struct.name);
         }
@@ -10966,12 +11064,15 @@ public class Nimbus {
         if (struct.is_set_enable()) {
           oprot.writeBool(struct.enable);
         }
+        if (struct.is_set_samplingPercentage()) {
+          oprot.writeDouble(struct.samplingPercentage);
+        }
       }
 
       @Override
       public void read(org.apache.thrift.protocol.TProtocol prot, debug_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(3);
+        BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           struct.name = iprot.readString();
           struct.set_name_isSet(true);
@@ -10984,6 +11085,10 @@ public class Nimbus {
           struct.enable = iprot.readBool();
           struct.set_enable_isSet(true);
         }
+        if (incoming.get(3)) {
+          struct.samplingPercentage = iprot.readDouble();
+          struct.set_samplingPercentage_isSet(true);
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/jvm/backtype/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
index 342a1c7..f3c1fb2 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
 public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
 
@@ -79,7 +79,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   private String owner; // optional
   private TopologyActionOptions topology_action_options; // optional
   private TopologyStatus prev_status; // optional
-  private Map<String,Boolean> component_debug; // optional
+  private Map<String,DebugOptions> component_debug; // 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 {
@@ -200,7 +200,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", 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.BOOL))));
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormBase.class, metaDataMap);
   }
@@ -247,7 +247,18 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       this.prev_status = other.prev_status;
     }
     if (other.is_set_component_debug()) {
-      Map<String,Boolean> __this__component_debug = new HashMap<String,Boolean>(other.component_debug);
+      Map<String,DebugOptions> __this__component_debug = new HashMap<String,DebugOptions>(other.component_debug.size());
+      for (Map.Entry<String, DebugOptions> other_element : other.component_debug.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        DebugOptions other_element_value = other_element.getValue();
+
+        String __this__component_debug_copy_key = other_element_key;
+
+        DebugOptions __this__component_debug_copy_value = new DebugOptions(other_element_value);
+
+        __this__component_debug.put(__this__component_debug_copy_key, __this__component_debug_copy_value);
+      }
       this.component_debug = __this__component_debug;
     }
   }
@@ -484,18 +495,18 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     return (this.component_debug == null) ? 0 : this.component_debug.size();
   }
 
-  public void put_to_component_debug(String key, boolean val) {
+  public void put_to_component_debug(String key, DebugOptions val) {
     if (this.component_debug == null) {
-      this.component_debug = new HashMap<String,Boolean>();
+      this.component_debug = new HashMap<String,DebugOptions>();
     }
     this.component_debug.put(key, val);
   }
 
-  public Map<String,Boolean> get_component_debug() {
+  public Map<String,DebugOptions> get_component_debug() {
     return this.component_debug;
   }
 
-  public void set_component_debug(Map<String,Boolean> component_debug) {
+  public void set_component_debug(Map<String,DebugOptions> component_debug) {
     this.component_debug = component_debug;
   }
 
@@ -584,7 +595,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (value == null) {
         unset_component_debug();
       } else {
-        set_component_debug((Map<String,Boolean>)value);
+        set_component_debug((Map<String,DebugOptions>)value);
       }
       break;
 
@@ -1133,13 +1144,14 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
                 org.apache.thrift.protocol.TMap _map430 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,Boolean>(2*_map430.size);
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map430.size);
                 String _key431;
-                boolean _val432;
+                DebugOptions _val432;
                 for (int _i433 = 0; _i433 < _map430.size; ++_i433)
                 {
                   _key431 = iprot.readString();
-                  _val432 = iprot.readBool();
+                  _val432 = new DebugOptions();
+                  _val432.read(iprot);
                   struct.component_debug.put(_key431, _val432);
                 }
                 iprot.readMapEnd();
@@ -1220,11 +1232,11 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         if (struct.is_set_component_debug()) {
           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.BOOL, struct.component_debug.size()));
-            for (Map.Entry<String, Boolean> _iter435 : struct.component_debug.entrySet())
+            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<String, DebugOptions> _iter435 : struct.component_debug.entrySet())
             {
               oprot.writeString(_iter435.getKey());
-              oprot.writeBool(_iter435.getValue());
+              _iter435.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1296,10 +1308,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, Boolean> _iter437 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter437 : struct.component_debug.entrySet())
           {
             oprot.writeString(_iter437.getKey());
-            oprot.writeBool(_iter437.getValue());
+            _iter437.getValue().write(oprot);
           }
         }
       }
@@ -1349,14 +1361,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map442 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BOOL, iprot.readI32());
-          struct.component_debug = new HashMap<String,Boolean>(2*_map442.size);
+          org.apache.thrift.protocol.TMap _map442 = 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<String,DebugOptions>(2*_map442.size);
           String _key443;
-          boolean _val444;
+          DebugOptions _val444;
           for (int _i445 = 0; _i445 < _map442.size; ++_i445)
           {
             _key443 = iprot.readString();
-            _val444 = iprot.readBool();
+            _val444 = new DebugOptions();
+            _val444.read(iprot);
             struct.component_debug.put(_key443, _val444);
           }
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index d9b7c8d..f0107af 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -77,7 +77,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private List<ExecutorSummary> executors; // required
   private String status; // required
   private Map<String,List<ErrorInfo>> errors; // required
-  private Map<String,Boolean> component_debug; // optional
+  private Map<String,DebugOptions> component_debug; // optional
   private String sched_status; // optional
   private String owner; // optional
 
@@ -189,7 +189,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", 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.BOOL))));
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT            , "DebugOptions"))));
     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.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -260,7 +260,18 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       this.errors = __this__errors;
     }
     if (other.is_set_component_debug()) {
-      Map<String,Boolean> __this__component_debug = new HashMap<String,Boolean>(other.component_debug);
+      Map<String,DebugOptions> __this__component_debug = new HashMap<String,DebugOptions>(other.component_debug.size());
+      for (Map.Entry<String, DebugOptions> other_element : other.component_debug.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        DebugOptions other_element_value = other_element.getValue();
+
+        String __this__component_debug_copy_key = other_element_key;
+
+        DebugOptions __this__component_debug_copy_value = other_element_value;
+
+        __this__component_debug.put(__this__component_debug_copy_key, __this__component_debug_copy_value);
+      }
       this.component_debug = __this__component_debug;
     }
     if (other.is_set_sched_status()) {
@@ -456,18 +467,18 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     return (this.component_debug == null) ? 0 : this.component_debug.size();
   }
 
-  public void put_to_component_debug(String key, boolean val) {
+  public void put_to_component_debug(String key, DebugOptions val) {
     if (this.component_debug == null) {
-      this.component_debug = new HashMap<String,Boolean>();
+      this.component_debug = new HashMap<String,DebugOptions>();
     }
     this.component_debug.put(key, val);
   }
 
-  public Map<String,Boolean> get_component_debug() {
+  public Map<String,DebugOptions> get_component_debug() {
     return this.component_debug;
   }
 
-  public void set_component_debug(Map<String,Boolean> component_debug) {
+  public void set_component_debug(Map<String,DebugOptions> component_debug) {
     this.component_debug = component_debug;
   }
 
@@ -586,7 +597,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       if (value == null) {
         unset_component_debug();
       } else {
-        set_component_debug((Map<String,Boolean>)value);
+        set_component_debug((Map<String,DebugOptions>)value);
       }
       break;
 
@@ -1166,13 +1177,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
                 org.apache.thrift.protocol.TMap _map300 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,Boolean>(2*_map300.size);
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map300.size);
                 String _key301;
-                boolean _val302;
+                DebugOptions _val302;
                 for (int _i303 = 0; _i303 < _map300.size; ++_i303)
                 {
                   _key301 = iprot.readString();
-                  _val302 = iprot.readBool();
+                  _val302 = new DebugOptions();
+                  _val302.read(iprot);
                   struct.component_debug.put(_key301, _val302);
                 }
                 iprot.readMapEnd();
@@ -1265,11 +1277,11 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         if (struct.is_set_component_debug()) {
           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.BOOL, struct.component_debug.size()));
-            for (Map.Entry<String, Boolean> _iter307 : struct.component_debug.entrySet())
+            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<String, DebugOptions> _iter307 : struct.component_debug.entrySet())
             {
               oprot.writeString(_iter307.getKey());
-              oprot.writeBool(_iter307.getValue());
+              _iter307.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1346,10 +1358,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, Boolean> _iter311 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter311 : struct.component_debug.entrySet())
           {
             oprot.writeString(_iter311.getKey());
-            oprot.writeBool(_iter311.getValue());
+            _iter311.getValue().write(oprot);
           }
         }
       }
@@ -1410,14 +1422,15 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map322 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BOOL, iprot.readI32());
-          struct.component_debug = new HashMap<String,Boolean>(2*_map322.size);
+          org.apache.thrift.protocol.TMap _map322 = 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<String,DebugOptions>(2*_map322.size);
           String _key323;
-          boolean _val324;
+          DebugOptions _val324;
           for (int _i325 = 0; _i325 < _map322.size; ++_i325)
           {
             _key323 = iprot.readString();
-            _val324 = iprot.readBool();
+            _val324 = new DebugOptions();
+            _val324.read(iprot);
             struct.component_debug.put(_key323, _val324);
           }
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/py/storm/Nimbus-remote
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote
index f58c325..fff63f9 100644
--- a/storm-core/src/py/storm/Nimbus-remote
+++ b/storm-core/src/py/storm/Nimbus-remote
@@ -49,7 +49,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void activate(string name)')
   print('  void deactivate(string name)')
   print('  void rebalance(string name, RebalanceOptions options)')
-  print('  void debug(string name, string component, bool enable)')
+  print('  void debug(string name, string component, bool enable, double samplingPercentage)')
   print('  void uploadNewCredentials(string name, Credentials creds)')
   print('  string beginFileUpload()')
   print('  void uploadChunk(string location, string chunk)')
@@ -162,10 +162,10 @@ elif cmd == 'rebalance':
   pp.pprint(client.rebalance(args[0],eval(args[1]),))
 
 elif cmd == 'debug':
-  if len(args) != 3:
-    print('debug requires 3 args')
+  if len(args) != 4:
+    print('debug requires 4 args')
     sys.exit(1)
-  pp.pprint(client.debug(args[0],args[1],eval(args[2]),))
+  pp.pprint(client.debug(args[0],args[1],eval(args[2]),eval(args[3]),))
 
 elif cmd == 'uploadNewCredentials':
   if len(args) != 2:

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index 8bbe8d0..cccb0b3 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -94,16 +94,19 @@ class Iface:
     """
     pass
 
-  def debug(self, name, component, enable):
+  def debug(self, name, component, enable, samplingPercentage):
     """
     Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
     and if null or empty, the debug flag will apply to the entire topology.
 
+    If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
+
 
     Parameters:
      - name
      - component
      - enable
+     - samplingPercentage
     """
     pass
 
@@ -456,26 +459,30 @@ class Client(Iface):
       raise result.aze
     return
 
-  def debug(self, name, component, enable):
+  def debug(self, name, component, enable, samplingPercentage):
     """
     Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
     and if null or empty, the debug flag will apply to the entire topology.
 
+    If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
+
 
     Parameters:
      - name
      - component
      - enable
+     - samplingPercentage
     """
-    self.send_debug(name, component, enable)
+    self.send_debug(name, component, enable, samplingPercentage)
     self.recv_debug()
 
-  def send_debug(self, name, component, enable):
+  def send_debug(self, name, component, enable, samplingPercentage):
     self._oprot.writeMessageBegin('debug', TMessageType.CALL, self._seqid)
     args = debug_args()
     args.name = name
     args.component = component
     args.enable = enable
+    args.samplingPercentage = samplingPercentage
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
     self._oprot.trans.flush()
@@ -1095,7 +1102,7 @@ class Processor(Iface, TProcessor):
     iprot.readMessageEnd()
     result = debug_result()
     try:
-      self._handler.debug(args.name, args.component, args.enable)
+      self._handler.debug(args.name, args.component, args.enable, args.samplingPercentage)
     except NotAliveException, e:
       result.e = e
     except AuthorizationException, aze:
@@ -2489,6 +2496,7 @@ class debug_args:
    - name
    - component
    - enable
+   - samplingPercentage
   """
 
   thrift_spec = (
@@ -2496,12 +2504,14 @@ class debug_args:
     (1, TType.STRING, 'name', None, None, ), # 1
     (2, TType.STRING, 'component', None, None, ), # 2
     (3, TType.BOOL, 'enable', None, None, ), # 3
+    (4, TType.DOUBLE, 'samplingPercentage', None, None, ), # 4
   )
 
-  def __init__(self, name=None, component=None, enable=None,):
+  def __init__(self, name=None, component=None, enable=None, samplingPercentage=None,):
     self.name = name
     self.component = component
     self.enable = enable
+    self.samplingPercentage = samplingPercentage
 
   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:
@@ -2527,6 +2537,11 @@ class debug_args:
           self.enable = iprot.readBool();
         else:
           iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.DOUBLE:
+          self.samplingPercentage = iprot.readDouble();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2549,6 +2564,10 @@ class debug_args:
       oprot.writeFieldBegin('enable', TType.BOOL, 3)
       oprot.writeBool(self.enable)
       oprot.writeFieldEnd()
+    if self.samplingPercentage is not None:
+      oprot.writeFieldBegin('samplingPercentage', TType.DOUBLE, 4)
+      oprot.writeDouble(self.samplingPercentage)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2561,6 +2580,7 @@ class debug_args:
     value = (value * 31) ^ hash(self.name)
     value = (value * 31) ^ hash(self.component)
     value = (value * 31) ^ hash(self.enable)
+    value = (value * 31) ^ hash(self.samplingPercentage)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index 2784dad..9b77461 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -3579,7 +3579,7 @@ class TopologyInfo:
     (4, TType.LIST, 'executors', (TType.STRUCT,(ExecutorSummary, ExecutorSummary.thrift_spec)), None, ), # 4
     (5, TType.STRING, 'status', None, None, ), # 5
     (6, TType.MAP, 'errors', (TType.STRING,None,TType.LIST,(TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec))), None, ), # 6
-    (7, TType.MAP, 'component_debug', (TType.STRING,None,TType.BOOL,None), None, ), # 7
+    (7, TType.MAP, 'component_debug', (TType.STRING,None,TType.STRUCT,(DebugOptions, DebugOptions.thrift_spec)), None, ), # 7
     None, # 8
     None, # 9
     None, # 10
@@ -4163,7 +4163,8 @@ class TopologyInfo:
           (_ktype280, _vtype281, _size279 ) = iprot.readMapBegin()
           for _i283 in xrange(_size279):
             _key284 = iprot.readString().decode('utf-8')
-            _val285 = iprot.readBool();
+            _val285 = DebugOptions()
+            _val285.read(iprot)
             self.component_debug[_key284] = _val285
           iprot.readMapEnd()
         else:
@@ -4224,10 +4225,10 @@ class TopologyInfo:
       oprot.writeFieldEnd()
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 7)
-      oprot.writeMapBegin(TType.STRING, TType.BOOL, len(self.component_debug))
+      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
       for kiter290,viter291 in self.component_debug.items():
         oprot.writeString(kiter290.encode('utf-8'))
-        oprot.writeBool(viter291)
+        viter291.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.sched_status is not None:
@@ -4281,6 +4282,84 @@ class TopologyInfo:
   def __ne__(self, other):
     return not (self == other)
 
+class DebugOptions:
+  """
+  Attributes:
+   - enable
+   - samplingpct
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.BOOL, 'enable', None, None, ), # 1
+    (2, TType.DOUBLE, 'samplingpct', None, None, ), # 2
+  )
+
+  def __init__(self, enable=None, samplingpct=None,):
+    self.enable = enable
+    self.samplingpct = samplingpct
+
+  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.BOOL:
+          self.enable = iprot.readBool();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.DOUBLE:
+          self.samplingpct = iprot.readDouble();
+        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('DebugOptions')
+    if self.enable is not None:
+      oprot.writeFieldBegin('enable', TType.BOOL, 1)
+      oprot.writeBool(self.enable)
+      oprot.writeFieldEnd()
+    if self.samplingpct is not None:
+      oprot.writeFieldBegin('samplingpct', TType.DOUBLE, 2)
+      oprot.writeDouble(self.samplingpct)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.enable)
+    value = (value * 31) ^ hash(self.samplingpct)
+    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:
@@ -5150,7 +5229,7 @@ class StormBase:
     (6, TType.STRING, 'owner', None, None, ), # 6
     (7, TType.STRUCT, 'topology_action_options', (TopologyActionOptions, TopologyActionOptions.thrift_spec), None, ), # 7
     (8, TType.I32, 'prev_status', None, None, ), # 8
-    (9, TType.MAP, 'component_debug', (TType.STRING,None,TType.BOOL,None), None, ), # 9
+    (9, TType.MAP, 'component_debug', (TType.STRING,None,TType.STRUCT,(DebugOptions, DebugOptions.thrift_spec)), None, ), # 9
   )
 
   def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None, component_debug=None,):
@@ -5226,7 +5305,8 @@ class StormBase:
           (_ktype389, _vtype390, _size388 ) = iprot.readMapBegin()
           for _i392 in xrange(_size388):
             _key393 = iprot.readString().decode('utf-8')
-            _val394 = iprot.readBool();
+            _val394 = DebugOptions()
+            _val394.read(iprot)
             self.component_debug[_key393] = _val394
           iprot.readMapEnd()
         else:
@@ -5279,10 +5359,10 @@ class StormBase:
       oprot.writeFieldEnd()
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 9)
-      oprot.writeMapBegin(TType.STRING, TType.BOOL, len(self.component_debug))
+      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
       for kiter397,viter398 in self.component_debug.items():
         oprot.writeString(kiter397.encode('utf-8'))
-        oprot.writeBool(viter398)
+        viter398.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 8547b30..2556fbe 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -218,11 +218,16 @@ struct TopologyInfo {
   4: required list<ExecutorSummary> executors;
   5: required string status;
   6: required map<string, list<ErrorInfo>> errors;
-  7: optional map<string, bool> component_debug;
+  7: optional map<string, DebugOptions> component_debug;
 513: optional string sched_status;
 514: optional string owner;
 }
 
+struct DebugOptions {
+  1: optional bool enable
+  2: optional double samplingpct
+}
+
 struct KillOptions {
   1: optional i32 wait_secs;
 }
@@ -290,7 +295,7 @@ struct StormBase {
     6: optional string owner;
     7: optional TopologyActionOptions topology_action_options;
     8: optional TopologyStatus prev_status;//currently only used during rebalance action.
-    9: optional map<string, bool> component_debug; // topology/component level debug flags.
+    9: optional map<string, DebugOptions> component_debug; // topology/component level debug option.
 }
 
 struct ClusterWorkerHeartbeat {
@@ -354,8 +359,10 @@ service Nimbus {
   /**
   * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
   * and if null or empty, the debug flag will apply to the entire topology.
+  *
+  * If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
   **/
-  void debug(1: string name, 2: string component, 3: bool enable) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  void debug(1: string name, 2: string component, 3: bool enable, 4: double samplingPercentage) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
 
   // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs

http://git-wip-us.apache.org/repos/asf/storm/blob/f9a4b010/storm-core/src/ui/public/js/script.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js
index d707530..fe8643d 100644
--- a/storm-core/src/ui/public/js/script.js
+++ b/storm-core/src/ui/public/js/script.js
@@ -81,7 +81,7 @@ function ensureInt(n) {
     return isInt;
 }
 
-function confirmComponentAction(topologyId, componentId, componentName, action, actionText) {
+function confirmComponentAction(topologyId, componentId, componentName, action, param, defaultParamValue, paramText, actionText) {
     var opts = {
         type:'POST',
         url:'/api/v1/topology/' + topologyId + '/component/' + componentId + '/' + action
@@ -89,8 +89,22 @@ function confirmComponentAction(topologyId, componentId, componentName, action,
     if (actionText === undefined) {
         actionText = action;
     }
-    if (!confirm('Do you really want to ' + actionText + ' component "' + componentName + '"?')) {
-        return false;
+    if (param) {
+        var paramValue = prompt('Do you really want to ' + actionText + ' component "' + componentName + '"? ' +
+                                  'If yes, please, specify ' + paramText + ':',
+                                  defaultParamValue);
+        if (paramValue != null && paramValue != "" && ensureInt(paramValue)) {
+            opts.url += '/' + paramValue;
+        } else {
+            return false;
+        }
+    } else {
+        if (typeof defaultParamValue !== 'undefined') {
+            opts.url +=  '/' + defaultParamValue;
+        }
+        if (!confirm('Do you really want to ' + actionText + ' component "' + componentName + '"?')) {
+            return false;
+        }
     }
 
     $("input[type=button]").attr("disabled", "disabled");
@@ -103,7 +117,7 @@ function confirmComponentAction(topologyId, componentId, componentName, action,
     return false;
 }
 
-function confirmAction(id, name, action, wait, defaultWait, actionText) {
+function confirmAction(id, name, action, param, defaultParamValue, paramText, actionText) {
     var opts = {
         type:'POST',
         url:'/api/v1/topology/' + id + '/' + action
@@ -111,18 +125,23 @@ function confirmAction(id, name, action, wait, defaultWait, actionText) {
     if (actionText === undefined) {
         actionText = action;
     }
-    if (wait) {
-        var waitSecs = prompt('Do you really want to ' + actionText + ' topology "' + name + '"? ' +
-                              'If yes, please, specify wait time in seconds:',
-                              defaultWait);
+    if (param) {
+        var paramValue = prompt('Do you really want to ' + actionText + ' topology "' + name + '"? ' +
+                              'If yes, please, specify ' + paramText + ':',
+                              defaultParamValue);
 
-        if (waitSecs != null && waitSecs != "" && ensureInt(waitSecs)) {
-            opts.url += '/' + waitSecs;
+        if (paramValue != null && paramValue != "" && ensureInt(paramValue)) {
+            opts.url += '/' + paramValue;
         } else {
             return false;
         }
-    } else if (!confirm('Do you really want to ' + actionText + ' topology "' + name + '"?')) {
-        return false;
+    } else {
+        if (typeof defaultParamValue !== 'undefined') {
+            opts.url +=  '/' + defaultParamValue;
+        }
+        if (!confirm('Do you really want to ' + actionText + ' topology "' + name + '"?')) {
+            return false;
+        }
     }
 
     $("input[type=button]").attr("disabled", "disabled");
@@ -171,7 +190,7 @@ function renderToggleSys(div) {
     }
 }
 
-function topologyActionJson(id, encodedId, name, status, msgTimeout, debug) {
+function topologyActionJson(id, encodedId, name, status, msgTimeout, debug, samplingPct) {
     var jsonData = {};
     jsonData["id"] = id;
     jsonData["encodedId"] = encodedId;
@@ -183,16 +202,18 @@ function topologyActionJson(id, encodedId, name, status, msgTimeout, debug) {
     jsonData["killStatus"] = (status !== "KILLED") ? "enabled" : "disabled";
     jsonData["startDebugStatus"] = (status === "ACTIVE" && !debug) ? "enabled" : "disabled";
     jsonData["stopDebugStatus"] = (status === "ACTIVE" && debug) ? "enabled" : "disabled";
+    jsonData["currentSamplingPct"] = samplingPct;
     return jsonData;
 }
 
-function componentActionJson(encodedTopologyId, encodedId, componentName, status, debug) {
+function componentActionJson(encodedTopologyId, encodedId, componentName, status, debug, samplingPct) {
     var jsonData = {};
     jsonData["encodedTopologyId"] = encodedTopologyId;
     jsonData["encodedId"] = encodedId;
     jsonData["componentName"] = componentName;
     jsonData["startDebugStatus"] = (status === "ACTIVE" && !debug) ? "enabled" : "disabled";
     jsonData["stopDebugStatus"] = (status === "ACTIVE" && debug) ? "enabled" : "disabled";
+    jsonData["currentSamplingPct"] = samplingPct;
     return jsonData;
 }
 


[02/24] storm git commit: Nimbus debug api first cut

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/26b02706/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index f9502c2..a8a3acd 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -17,11 +17,12 @@
   (:use [clojure test])
   (:require [backtype.storm [util :as util] [stats :as stats]])
   (:require [backtype.storm.daemon [nimbus :as nimbus]])
+  (:require [backtype.storm [converter :as converter]])
   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount
             TestAggregatesCounter TestPlannerSpout TestPlannerBolt])
   (:import [backtype.storm.scheduler INimbus])
   (:import [backtype.storm.generated Credentials NotAliveException SubmitOptions
-            TopologyInitialStatus AlreadyAliveException KillOptions RebalanceOptions
+            TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
             InvalidTopologyException AuthorizationException])
   (:import [java.util HashMap])
   (:import [java.io File])
@@ -1180,3 +1181,11 @@
         (is (thrown-cause? InvalidTopologyException
           (submit-local-topology-with-opts nimbus "test" bad-config topology
                                            (SubmitOptions.))))))))
+
+(deftest test-debug-on
+  (with-local-cluster [cluster]
+    (let [nimbus (:nimbus cluster)]
+      (stubbing [converter/clojurify-storm-base (backtype.storm.daemon.common.StormBase. "storm-name" 100
+                                                  {:type :active} 1 nil nil nil nil nil)
+                 cluster/maybe-deserialize nil]
+        (.debug nimbus "test" true)))))


[20/24] storm git commit: Sampling pct and Fields fixes

Posted by sr...@apache.org.
Sampling pct and Fields fixes

1. Changed UI default sampling pct to 10 pct
2. Fixes to fields logged by EventLogger


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

Branch: refs/heads/master
Commit: ea868a4f0a5af6ad43941ad0ecc09aaa0d0ec498
Parents: cec7ff9
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Thu Sep 10 00:17:39 2015 -0700
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Thu Sep 10 00:17:39 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/converter.clj              | 2 +-
 storm-core/src/clj/backtype/storm/daemon/common.clj          | 2 +-
 storm-core/src/clj/backtype/storm/daemon/executor.clj        | 8 ++++----
 storm-core/src/clj/backtype/storm/ui/core.clj                | 4 ++--
 storm-core/src/jvm/backtype/storm/generated/Nimbus.java      | 2 +-
 .../src/jvm/backtype/storm/metric/EventLoggerBolt.java       | 4 +++-
 storm-core/src/py/storm/Nimbus.py                            | 3 ++-
 storm-core/src/storm.thrift                                  | 2 +-
 8 files changed, 15 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index 253a406..c571fa1 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -154,7 +154,7 @@
 (defn thriftify-debugoptions [options]
   (doto (DebugOptions.)
     (.set_enable (get options :enable false))
-    (.set_samplingpct (get options :samplingpct 100))))
+    (.set_samplingpct (get options :samplingpct 10))))
 
 (defn thriftify-storm-base [storm-base]
   (doto (StormBase.)

http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/clj/backtype/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index 173cd5a..9da0e4f 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -295,7 +295,7 @@
 
 ;; return the fields that event logger bolt expects
 (defn eventlogger-bolt-fields []
-  [(EventLoggerBolt/FIELD_COMPONENT_ID)  (EventLoggerBolt/FIELD_TS) (EventLoggerBolt/FIELD_VALUES)]
+  [(EventLoggerBolt/FIELD_COMPONENT_ID) (EventLoggerBolt/FIELD_MESSAGE_ID)  (EventLoggerBolt/FIELD_TS) (EventLoggerBolt/FIELD_VALUES)]
   )
 
 (defn add-eventlogger! [storm-conf ^StormTopology ret]

http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index 6a6d7f0..829a0b4 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -448,7 +448,7 @@
 
 ;; Send sampled data to the eventlogger if the global or component level
 ;; debug flag is set (via nimbus api).
-(defn send-to-eventlogger [executor-data task-data values overflow-buffer component-id random]
+(defn send-to-eventlogger [executor-data task-data values overflow-buffer component-id message-id random]
     (let [c->d @(:storm-component->debug-atom executor-data)
           options (get c->d component-id (get c->d (:storm-id executor-data)))
           spct    (if (and (not-nil? options) (:enable options)) (:samplingpct options) 0)]
@@ -458,7 +458,7 @@
         (task/send-unanchored
           task-data
           EVENTLOGGER-STREAM-ID
-          [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
+          [component-id message-id (System/currentTimeMillis) values]
           overflow-buffer))))
 
 (defmethod mk-threads :spout [executor-data task-datas initial-credentials]
@@ -549,7 +549,7 @@
                                                                         overflow-buffer)
                                                            ))
                                          (if has-eventloggers?
-                                           (send-to-eventlogger executor-data task-data values overflow-buffer component-id rand))
+                                           (send-to-eventlogger executor-data task-data values overflow-buffer component-id message-id rand))
                                          (if (and rooted?
                                                   (not (.isEmpty out-ids)))
                                            (do
@@ -757,7 +757,7 @@
                                                                                (MessageId/makeId anchors-to-ids))
                                                                    overflow-buffer)))
                                     (if has-eventloggers?
-                                      (send-to-eventlogger executor-data task-data values overflow-buffer component-id rand))
+                                      (send-to-eventlogger executor-data task-data values overflow-buffer component-id nil rand))
                                     (or out-tasks [])))]]
           (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
           (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials)) 

http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index f8b0390..414bfb1 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -758,7 +758,7 @@
        "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) 100)
+       "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]
@@ -987,7 +987,7 @@
          "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) 100)
+         "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 10)
          "eventLogLink" (event-log-link topology-id summ topology component secure?)}
        spec errors))))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index c5d5c0a..3a5d5df 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
@@ -74,7 +74,7 @@ public class Nimbus {
      * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
      * and if null or empty, the debug flag will apply to the entire topology.
      * 
-     * If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
+     * The 'samplingPercentage' will limit loggging to a percentage of generated tuples.
      * 
      * 
      * @param name

http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java b/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
index 19ff8c5..17ebbd1 100644
--- a/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
+++ b/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
@@ -28,6 +28,7 @@ public class EventLoggerBolt implements IBolt {
     public static final String FIELD_TS = "ts";
     public static final String FIELD_VALUES = "values";
     public static final String FIELD_COMPONENT_ID = "component-id";
+    public static final String FIELD_MESSAGE_ID = "message-id";
 
     private IEventLogger eventLogger;
 
@@ -42,8 +43,9 @@ public class EventLoggerBolt implements IBolt {
     public void execute(Tuple input) {
         LOG.debug("** EventLoggerBolt got tuple from sourceComponent {}, with values {}", input.getSourceComponent(), input.getValues());
 
+        Object msgId = input.getValueByField(FIELD_MESSAGE_ID);
         EventInfo eventInfo = new EventInfo(input.getValueByField(FIELD_TS).toString(), input.getSourceComponent(),
-                                            String.valueOf(input.getSourceTask()), input.getMessageId().toString(),
+                                            String.valueOf(input.getSourceTask()), msgId == null ? "" : msgId.toString(),
                                             input.getValueByField(FIELD_VALUES).toString());
 
         eventLogger.log(eventInfo);

http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index cccb0b3..0caeed0 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -99,7 +99,8 @@ class Iface:
     Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
     and if null or empty, the debug flag will apply to the entire topology.
 
-    If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
+    The 'samplingPercentage' will limit loggging to a percentage of generated tuples.
+
 
 
     Parameters:

http://git-wip-us.apache.org/repos/asf/storm/blob/ea868a4f/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 20dbbbe..f687681 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -370,7 +370,7 @@ service Nimbus {
   * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
   * and if null or empty, the debug flag will apply to the entire topology.
   *
-  * If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
+  * The 'samplingPercentage' will limit loggging to a percentage of generated tuples.
   **/
   void debug(1: string name, 2: string component, 3: bool enable, 4: double samplingPercentage) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);


[06/24] storm git commit: nimbus and stormbase changes for component level debug flag

Posted by sr...@apache.org.
nimbus and stormbase changes for component level debug flag


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

Branch: refs/heads/master
Commit: 5914944f6ac7ad637a8727ab04fed0c1a5e2be1e
Parents: d1e5ec9
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Sun Aug 9 18:17:19 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 10 14:22:09 2015 +0530

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   |  4 ++-
 storm-core/src/clj/backtype/storm/converter.clj |  4 +--
 .../src/clj/backtype/storm/daemon/common.clj    |  2 +-
 .../src/clj/backtype/storm/daemon/executor.clj  | 28 +++++++++++---------
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  8 +++---
 .../src/clj/backtype/storm/daemon/worker.clj    |  6 ++---
 storm-core/src/clj/backtype/storm/ui/core.clj   | 26 ++++++++++++------
 7 files changed, 47 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5914944f/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index 1070237..5d854d6 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -412,7 +412,9 @@
         [this storm-id new-elems]
         (let [base (storm-base this storm-id nil)
               executors (:component->executors base)
-              new-elems (update new-elems :component->executors (partial merge executors))]
+              component->debug (:component->debug base)
+              new-elems (update new-elems :component->executors (partial merge executors))
+              new-elems (update new-elems :component->debug (partial merge component->debug))]
           (set-data cluster-state (storm-path storm-id)
                     (-> base
                         (merge new-elems)

http://git-wip-us.apache.org/repos/asf/storm/blob/5914944f/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index a429c51..dbda8a9 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -152,7 +152,7 @@
     (.set_owner (:owner storm-base))
     (.set_topology_action_options (thriftify-topology-action-options storm-base))
     (.set_prev_status (convert-to-status-from-symbol (:prev-status storm-base)))
-    (.set_debug (:debug storm-base))))
+    (.set_component_debug (map-val boolean (:component->debug storm-base)))))
 
 (defn clojurify-storm-base [^StormBase storm-base]
   (if storm-base
@@ -165,7 +165,7 @@
       (.get_owner storm-base)
       (clojurify-topology-action-options (.get_topology_action_options storm-base))
       (convert-to-symbol-from-status (.get_prev_status storm-base))
-      (.is_debug storm-base))))
+      (into {} (.get_component_debug storm-base)))))
 
 (defn thriftify-stats [stats]
   (if stats

http://git-wip-us.apache.org/repos/asf/storm/blob/5914944f/storm-core/src/clj/backtype/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index 55cecee..359c4b6 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -55,7 +55,7 @@
 
 
 ;; component->executors is a map from spout/bolt id to number of executors for that component
-(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner topology-action-options prev-status debug])
+(defrecord StormBase [storm-name launch-time-secs status num-workers component->executors owner topology-action-options prev-status component->debug])
 
 (defrecord SupervisorInfo [time-secs hostname assignment-id used-ports meta scheduler-meta uptime-secs version])
 

http://git-wip-us.apache.org/repos/asf/storm/blob/5914944f/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index b3e356c..e8a42f8 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -240,7 +240,7 @@
      :conf (:conf worker)
      :shared-executor-data (HashMap.)
      :storm-active-atom (:storm-active-atom worker)
-     :storm-debug-atom (:storm-debug-atom worker)
+     :storm-component->debug-atom (:storm-component->debug-atom worker)
      :batch-transfer-queue batch-transfer->worker
      :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
      :suicide-fn (:suicide-fn worker)
@@ -534,12 +534,14 @@
                                                                         overflow-buffer)
                                                            ))
                                          ; Send data to the eventlogger.
-                                         (if (and has-eventloggers? @(:storm-debug-atom executor-data))
-                                           (task/send-unanchored
-                                             task-data
-                                             EVENTLOGGER-STREAM-ID
-                                             [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
-                                             overflow-buffer))
+                                         (if has-eventloggers?
+                                           (let [c->d @(:storm-component->debug-atom executor-data)]
+                                             (if (get c->d component-id (get c->d (:storm-id executor-data) false))
+                                               (task/send-unanchored
+                                                 task-data
+                                                 EVENTLOGGER-STREAM-ID
+                                                 [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
+                                                 overflow-buffer))))
                                          (if (and rooted?
                                                   (not (.isEmpty out-ids)))
                                            (do
@@ -747,11 +749,13 @@
                                                                                (MessageId/makeId anchors-to-ids))
                                                                    overflow-buffer)))
                                     ; send the data to the eventlogger
-                                    (if (and has-eventloggers? @(:storm-debug-atom executor-data))
-                                      (task/send-unanchored task-data
-                                        EVENTLOGGER-STREAM-ID
-                                        [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
-                                        overflow-buffer))
+                                    (if has-eventloggers?
+                                      (let [c->d @(:storm-component->debug-atom executor-data)]
+                                        (if (get c->d component-id (get c->d (:storm-id executor-data) false))
+                                          (task/send-unanchored task-data
+                                            EVENTLOGGER-STREAM-ID
+                                            [component-id (System/currentTimeMillis) values] ;TODO: add more metadata to the vector
+                                            overflow-buffer))))
                                     (or out-tasks [])))]]
           (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
           (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials)) 

http://git-wip-us.apache.org/repos/asf/storm/blob/5914944f/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 3e39d44..c9e80c7 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -755,7 +755,7 @@
                                   (storm-conf TOPOLOGY-SUBMITTER-USER)
                                   nil
                                   nil
-                                  false))))
+                                  {}))))
 
 ;; Master:
 ;; job submit:
@@ -1162,10 +1162,10 @@
         (transition-name! nimbus storm-name :inactivate true))
 
       ;; TODO
-      (debug [this storm-name enable?]
+      (debug [this storm-name component-id enable?]
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
-              storm-base-updates {:debug enable?}]
+              storm-base-updates (assoc {} :component->debug (if (empty? component-id) {storm-id enable?} {component-id enable?}))]
 ;;          (check-authorization! nimbus storm-name topology-conf "debug")
 ;;          (when-not storm-id
 ;;            (throw (NotAliveException. storm-name)))
@@ -1352,7 +1352,7 @@
                            executor-summaries
                            (extract-status-str base)
                            errors
-                           (:debug base)
+                           (map-val boolean (:component->debug base))
                            )]
             (when-let [owner (:owner base)] (.set_owner topo-info owner))
             (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))

http://git-wip-us.apache.org/repos/asf/storm/blob/5914944f/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index f947637..69eddfb 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -223,7 +223,7 @@
       ;; and spout and bolt will be activated.
       :worker-active-flag (atom false)
       :storm-active-atom (atom false)
-      :storm-debug-atom (atom false)
+      :storm-component->debug-atom (atom {})
       :executors executors
       :task-ids (->> receive-queue-map keys (map int) sort)
       :storm-conf storm-conf
@@ -328,8 +328,8 @@
       (reset!
         (:storm-active-atom worker)
         (and (= :active (-> base :status :type)) @(:worker-active-flag worker)))
-      (reset! (:storm-debug-atom worker) (-> base :debug))
-      (log-message "debug flag is " @(:storm-debug-atom worker)))))
+      (reset! (:storm-component->debug-atom worker) (-> base :component->debug))
+      (log-message "debug options " @(:storm-component->debug-atom worker)))))
 
 ;; TODO: consider having a max batch size besides what disruptor does automagically to prevent latency issues
 (defn mk-transfer-tuples-handler [worker]

http://git-wip-us.apache.org/repos/asf/storm/blob/5914944f/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index 1b5e5b4..a0e9171 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -714,8 +714,10 @@
 (defn topology-summary [^TopologyInfo summ]
   (let [executors (.get_executors summ)
         workers (set (for [^ExecutorSummary e executors]
-                       [(.get_host e) (.get_port e)]))]
-      {"id" (.get_id summ)
+                       [(.get_host e) (.get_port e)]))
+        topology-id (.get_id summ)
+        component->debug (.get_component_debug summ)]
+      {"id" topology-id
        "encodedId" (url-encode (.get_id summ))
        "owner" (.get_owner summ)
        "name" (.get_name summ)
@@ -725,7 +727,7 @@
        "workersTotal" (count workers)
        "executorsTotal" (count executors)
        "schedulerInfo" (.get_sched_status summ)
-       "debug" (.is_debug summ)}))
+       "debug" (get component->debug topology-id false)}))
 
 (defn spout-summary-json [topology-id id stats window]
   (let [times (stats-times (:emitted stats))
@@ -934,7 +936,8 @@
           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?)]
+          errors (component-errors (get (.get_errors summ) component) topology-id secure?)
+          component->debug (.get_component_debug summ)]
       (merge
         {"user" user
          "id" component
@@ -947,7 +950,7 @@
          "window" window
          "componentType" (name type)
          "windowHint" (window-hint window)
-         "debug" (.is_debug summ)
+         "debug" (get component->debug component (get component->debug topology-id false))
          "eventLogLink" (event-log-link topology-id summ topology component secure?)}
        spec errors))))
 
@@ -961,6 +964,13 @@
    "status" "success"
    })
 
+(defn component-op-response [topology-id component-id op]
+  {"topologyOperation" op,
+   "topologyId" topology-id,
+   "componentId" component-id,
+   "status" "success"
+   })
+
 (defn check-include-sys?
   [sys?]
   (if (or (nil? sys?) (= "false" sys?)) false true))
@@ -1040,7 +1050,7 @@
                    (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
             name (.get_name tplg)
             enable? (= "enable" action)]
-        (.debug nimbus name enable?)
+        (.debug nimbus name "" enable?)
         (log-message "Debug topology [" name "] action [" action "]")))
     (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/component/:component/debug/:action" [:as {:keys [cookies servlet-request]} id component action & m]
@@ -1052,9 +1062,9 @@
                    (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
             name (.get_name tplg)
             enable? (= "enable" action)]
-        (.debug nimbus name enable?) ;; TODO: include component id in the nimbus api
+        (.debug nimbus name component enable?)
         (log-message "Debug topology [" name "] component [" component "] action [" action "]")))
-    (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
+    (json-response (component-op-response id component (str "/debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
     (assert-authorized-user servlet-request "rebalance" (topology-config id))
     (with-nimbus nimbus


[04/24] storm git commit: Checking in the py files that were modifed by genthrift

Posted by sr...@apache.org.
Checking in the py files that were modifed by genthrift


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

Branch: refs/heads/master
Commit: f8c4c5bf428c812ffa51eaabed0c8c1d04793cb4
Parents: 26b0270
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Fri Aug 7 11:15:42 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 10 14:21:51 2015 +0530

----------------------------------------------------------------------
 storm-core/src/py/storm/Nimbus-remote |   7 +
 storm-core/src/py/storm/Nimbus.py     | 218 +++++++++++++++++++++++++++++
 storm-core/src/py/storm/ttypes.py     |  39 +++++-
 3 files changed, 257 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f8c4c5bf/storm-core/src/py/storm/Nimbus-remote
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote
index 9af5ead..d012659 100644
--- a/storm-core/src/py/storm/Nimbus-remote
+++ b/storm-core/src/py/storm/Nimbus-remote
@@ -49,6 +49,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void activate(string name)')
   print('  void deactivate(string name)')
   print('  void rebalance(string name, RebalanceOptions options)')
+  print('  void debug(string name, bool enable)')
   print('  void uploadNewCredentials(string name, Credentials creds)')
   print('  string beginFileUpload()')
   print('  void uploadChunk(string location, string chunk)')
@@ -160,6 +161,12 @@ elif cmd == 'rebalance':
     sys.exit(1)
   pp.pprint(client.rebalance(args[0],eval(args[1]),))
 
+elif cmd == 'debug':
+  if len(args) != 2:
+    print('debug requires 2 args')
+    sys.exit(1)
+  pp.pprint(client.debug(args[0],eval(args[1]),))
+
 elif cmd == 'uploadNewCredentials':
   if len(args) != 2:
     print('uploadNewCredentials requires 2 args')

http://git-wip-us.apache.org/repos/asf/storm/blob/f8c4c5bf/storm-core/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index e9636c2..9e6923d 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -94,6 +94,14 @@ class Iface:
     """
     pass
 
+  def debug(self, name, enable):
+    """
+    Parameters:
+     - name
+     - enable
+    """
+    pass
+
   def uploadNewCredentials(self, name, creds):
     """
     Parameters:
@@ -443,6 +451,41 @@ class Client(Iface):
       raise result.aze
     return
 
+  def debug(self, name, enable):
+    """
+    Parameters:
+     - name
+     - enable
+    """
+    self.send_debug(name, enable)
+    self.recv_debug()
+
+  def send_debug(self, name, enable):
+    self._oprot.writeMessageBegin('debug', TMessageType.CALL, self._seqid)
+    args = debug_args()
+    args.name = name
+    args.enable = enable
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_debug(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = debug_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    return
+
   def uploadNewCredentials(self, name, creds):
     """
     Parameters:
@@ -887,6 +930,7 @@ class Processor(Iface, TProcessor):
     self._processMap["activate"] = Processor.process_activate
     self._processMap["deactivate"] = Processor.process_deactivate
     self._processMap["rebalance"] = Processor.process_rebalance
+    self._processMap["debug"] = Processor.process_debug
     self._processMap["uploadNewCredentials"] = Processor.process_uploadNewCredentials
     self._processMap["beginFileUpload"] = Processor.process_beginFileUpload
     self._processMap["uploadChunk"] = Processor.process_uploadChunk
@@ -1034,6 +1078,22 @@ class Processor(Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_debug(self, seqid, iprot, oprot):
+    args = debug_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = debug_result()
+    try:
+      self._handler.debug(args.name, args.enable)
+    except NotAliveException, e:
+      result.e = e
+    except AuthorizationException, aze:
+      result.aze = aze
+    oprot.writeMessageBegin("debug", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_uploadNewCredentials(self, seqid, iprot, oprot):
     args = uploadNewCredentials_args()
     args.read(iprot)
@@ -2412,6 +2472,164 @@ class rebalance_result:
   def __ne__(self, other):
     return not (self == other)
 
+class debug_args:
+  """
+  Attributes:
+   - name
+   - enable
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.BOOL, 'enable', None, None, ), # 2
+  )
+
+  def __init__(self, name=None, enable=None,):
+    self.name = name
+    self.enable = enable
+
+  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.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.BOOL:
+          self.enable = 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('debug_args')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.enable is not None:
+      oprot.writeFieldBegin('enable', TType.BOOL, 2)
+      oprot.writeBool(self.enable)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.enable)
+    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 debug_result:
+  """
+  Attributes:
+   - e
+   - aze
+  """
+
+  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, e=None, aze=None,):
+    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 == 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('debug_result')
+    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.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 uploadNewCredentials_args:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/storm/blob/f8c4c5bf/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index a06af92..46e64ed 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -2365,10 +2365,10 @@ class SupervisorSummary:
     (3, TType.I32, 'num_workers', None, None, ), # 3
     (4, TType.I32, 'num_used_workers', None, None, ), # 4
     (5, TType.STRING, 'supervisor_id', None, None, ), # 5
-    (6, TType.STRING, 'version', None, None, ), # 6
+    (6, TType.STRING, 'version', None, "VERSION_NOT_PROVIDED", ), # 6
   )
 
-  def __init__(self, host=None, uptime_secs=None, num_workers=None, num_used_workers=None, supervisor_id=None, version=None,):
+  def __init__(self, host=None, uptime_secs=None, num_workers=None, num_used_workers=None, supervisor_id=None, version=thrift_spec[6][4],):
     self.host = host
     self.uptime_secs = uptime_secs
     self.num_workers = num_workers
@@ -2463,8 +2463,6 @@ class SupervisorSummary:
       raise TProtocol.TProtocolException(message='Required field num_used_workers is unset!')
     if self.supervisor_id is None:
       raise TProtocol.TProtocolException(message='Required field supervisor_id is unset!')
-    if self.version is None:
-      raise TProtocol.TProtocolException(message='Required field version is unset!')
     return
 
 
@@ -3568,6 +3566,7 @@ class TopologyInfo:
    - executors
    - status
    - errors
+   - debug
    - sched_status
    - owner
   """
@@ -3580,7 +3579,7 @@ class TopologyInfo:
     (4, TType.LIST, 'executors', (TType.STRUCT,(ExecutorSummary, ExecutorSummary.thrift_spec)), None, ), # 4
     (5, TType.STRING, 'status', None, None, ), # 5
     (6, TType.MAP, 'errors', (TType.STRING,None,TType.LIST,(TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec))), None, ), # 6
-    None, # 7
+    (7, TType.BOOL, 'debug', None, None, ), # 7
     None, # 8
     None, # 9
     None, # 10
@@ -4090,13 +4089,14 @@ class TopologyInfo:
     (514, TType.STRING, 'owner', None, None, ), # 514
   )
 
-  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, sched_status=None, owner=None,):
+  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, debug=None, sched_status=None, owner=None,):
     self.id = id
     self.name = name
     self.uptime_secs = uptime_secs
     self.executors = executors
     self.status = status
     self.errors = errors
+    self.debug = debug
     self.sched_status = sched_status
     self.owner = owner
 
@@ -4157,6 +4157,11 @@ class TopologyInfo:
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.BOOL:
+          self.debug = iprot.readBool();
+        else:
+          iprot.skip(ftype)
       elif fid == 513:
         if ftype == TType.STRING:
           self.sched_status = iprot.readString().decode('utf-8')
@@ -4211,6 +4216,10 @@ class TopologyInfo:
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
+    if self.debug is not None:
+      oprot.writeFieldBegin('debug', TType.BOOL, 7)
+      oprot.writeBool(self.debug)
+      oprot.writeFieldEnd()
     if self.sched_status is not None:
       oprot.writeFieldBegin('sched_status', TType.STRING, 513)
       oprot.writeString(self.sched_status.encode('utf-8'))
@@ -4235,6 +4244,8 @@ class TopologyInfo:
       raise TProtocol.TProtocolException(message='Required field status is unset!')
     if self.errors is None:
       raise TProtocol.TProtocolException(message='Required field errors is unset!')
+    if self.debug is None:
+      raise TProtocol.TProtocolException(message='Required field debug is unset!')
     return
 
 
@@ -4246,6 +4257,7 @@ class TopologyInfo:
     value = (value * 31) ^ hash(self.executors)
     value = (value * 31) ^ hash(self.status)
     value = (value * 31) ^ hash(self.errors)
+    value = (value * 31) ^ hash(self.debug)
     value = (value * 31) ^ hash(self.sched_status)
     value = (value * 31) ^ hash(self.owner)
     return value
@@ -5117,6 +5129,7 @@ class StormBase:
    - owner
    - topology_action_options
    - prev_status
+   - debug
   """
 
   thrift_spec = (
@@ -5129,9 +5142,10 @@ class StormBase:
     (6, TType.STRING, 'owner', None, None, ), # 6
     (7, TType.STRUCT, 'topology_action_options', (TopologyActionOptions, TopologyActionOptions.thrift_spec), None, ), # 7
     (8, TType.I32, 'prev_status', None, None, ), # 8
+    (9, TType.BOOL, 'debug', None, None, ), # 9
   )
 
-  def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None,):
+  def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None, debug=None,):
     self.name = name
     self.status = status
     self.num_workers = num_workers
@@ -5140,6 +5154,7 @@ class StormBase:
     self.owner = owner
     self.topology_action_options = topology_action_options
     self.prev_status = prev_status
+    self.debug = debug
 
   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:
@@ -5197,6 +5212,11 @@ class StormBase:
           self.prev_status = iprot.readI32();
         else:
           iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.BOOL:
+          self.debug = iprot.readBool();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -5243,6 +5263,10 @@ class StormBase:
       oprot.writeFieldBegin('prev_status', TType.I32, 8)
       oprot.writeI32(self.prev_status)
       oprot.writeFieldEnd()
+    if self.debug is not None:
+      oprot.writeFieldBegin('debug', TType.BOOL, 9)
+      oprot.writeBool(self.debug)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -5266,6 +5290,7 @@ class StormBase:
     value = (value * 31) ^ hash(self.owner)
     value = (value * 31) ^ hash(self.topology_action_options)
     value = (value * 31) ^ hash(self.prev_status)
+    value = (value * 31) ^ hash(self.debug)
     return value
 
   def __repr__(self):


[07/24] storm git commit: nimbus api and stormbase changes to support component level debug

Posted by sr...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
index a51ad45..342a1c7 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
 
@@ -63,7 +63,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   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)6);
   private static final org.apache.thrift.protocol.TField TOPOLOGY_ACTION_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_action_options", org.apache.thrift.protocol.TType.STRUCT, (short)7);
   private static final org.apache.thrift.protocol.TField PREV_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("prev_status", org.apache.thrift.protocol.TType.I32, (short)8);
-  private static final org.apache.thrift.protocol.TField DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("debug", org.apache.thrift.protocol.TType.BOOL, (short)9);
+  private static final org.apache.thrift.protocol.TField COMPONENT_DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("component_debug", org.apache.thrift.protocol.TType.MAP, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -79,7 +79,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   private String owner; // optional
   private TopologyActionOptions topology_action_options; // optional
   private TopologyStatus prev_status; // optional
-  private boolean debug; // optional
+  private Map<String,Boolean> component_debug; // 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 {
@@ -99,7 +99,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
      * @see TopologyStatus
      */
     PREV_STATUS((short)8, "prev_status"),
-    DEBUG((short)9, "debug");
+    COMPONENT_DEBUG((short)9, "component_debug");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -130,8 +130,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           return TOPOLOGY_ACTION_OPTIONS;
         case 8: // PREV_STATUS
           return PREV_STATUS;
-        case 9: // DEBUG
-          return DEBUG;
+        case 9: // COMPONENT_DEBUG
+          return COMPONENT_DEBUG;
         default:
           return null;
       }
@@ -174,9 +174,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
   // isset id assignments
   private static final int __NUM_WORKERS_ISSET_ID = 0;
   private static final int __LAUNCH_TIME_SECS_ISSET_ID = 1;
-  private static final int __DEBUG_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.COMPONENT_EXECUTORS,_Fields.LAUNCH_TIME_SECS,_Fields.OWNER,_Fields.TOPOLOGY_ACTION_OPTIONS,_Fields.PREV_STATUS,_Fields.DEBUG};
+  private static final _Fields optionals[] = {_Fields.COMPONENT_EXECUTORS,_Fields.LAUNCH_TIME_SECS,_Fields.OWNER,_Fields.TOPOLOGY_ACTION_OPTIONS,_Fields.PREV_STATUS,_Fields.COMPONENT_DEBUG};
   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);
@@ -198,8 +197,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyActionOptions.class)));
     tmpMap.put(_Fields.PREV_STATUS, new org.apache.thrift.meta_data.FieldMetaData("prev_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyStatus.class)));
-    tmpMap.put(_Fields.DEBUG, new org.apache.thrift.meta_data.FieldMetaData("debug", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", 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.BOOL))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormBase.class, metaDataMap);
   }
@@ -245,7 +246,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     if (other.is_set_prev_status()) {
       this.prev_status = other.prev_status;
     }
-    this.debug = other.debug;
+    if (other.is_set_component_debug()) {
+      Map<String,Boolean> __this__component_debug = new HashMap<String,Boolean>(other.component_debug);
+      this.component_debug = __this__component_debug;
+    }
   }
 
   public StormBase deepCopy() {
@@ -264,8 +268,7 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     this.owner = null;
     this.topology_action_options = null;
     this.prev_status = null;
-    set_debug_isSet(false);
-    this.debug = false;
+    this.component_debug = null;
   }
 
   public String get_name() {
@@ -477,26 +480,38 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     }
   }
 
-  public boolean is_debug() {
-    return this.debug;
+  public int get_component_debug_size() {
+    return (this.component_debug == null) ? 0 : this.component_debug.size();
   }
 
-  public void set_debug(boolean debug) {
-    this.debug = debug;
-    set_debug_isSet(true);
+  public void put_to_component_debug(String key, boolean val) {
+    if (this.component_debug == null) {
+      this.component_debug = new HashMap<String,Boolean>();
+    }
+    this.component_debug.put(key, val);
   }
 
-  public void unset_debug() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  public Map<String,Boolean> get_component_debug() {
+    return this.component_debug;
   }
 
-  /** Returns true if field debug is set (has been assigned a value) and false otherwise */
-  public boolean is_set_debug() {
-    return EncodingUtils.testBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  public void set_component_debug(Map<String,Boolean> component_debug) {
+    this.component_debug = component_debug;
   }
 
-  public void set_debug_isSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DEBUG_ISSET_ID, value);
+  public void unset_component_debug() {
+    this.component_debug = null;
+  }
+
+  /** Returns true if field component_debug is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_debug() {
+    return this.component_debug != null;
+  }
+
+  public void set_component_debug_isSet(boolean value) {
+    if (!value) {
+      this.component_debug = null;
+    }
   }
 
   public void setFieldValue(_Fields field, Object value) {
@@ -565,11 +580,11 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       break;
 
-    case DEBUG:
+    case COMPONENT_DEBUG:
       if (value == null) {
-        unset_debug();
+        unset_component_debug();
       } else {
-        set_debug((Boolean)value);
+        set_component_debug((Map<String,Boolean>)value);
       }
       break;
 
@@ -602,8 +617,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     case PREV_STATUS:
       return get_prev_status();
 
-    case DEBUG:
-      return Boolean.valueOf(is_debug());
+    case COMPONENT_DEBUG:
+      return get_component_debug();
 
     }
     throw new IllegalStateException();
@@ -632,8 +647,8 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       return is_set_topology_action_options();
     case PREV_STATUS:
       return is_set_prev_status();
-    case DEBUG:
-      return is_set_debug();
+    case COMPONENT_DEBUG:
+      return is_set_component_debug();
     }
     throw new IllegalStateException();
   }
@@ -723,12 +738,12 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         return false;
     }
 
-    boolean this_present_debug = true && this.is_set_debug();
-    boolean that_present_debug = true && that.is_set_debug();
-    if (this_present_debug || that_present_debug) {
-      if (!(this_present_debug && that_present_debug))
+    boolean this_present_component_debug = true && this.is_set_component_debug();
+    boolean that_present_component_debug = true && that.is_set_component_debug();
+    if (this_present_component_debug || that_present_component_debug) {
+      if (!(this_present_component_debug && that_present_component_debug))
         return false;
-      if (this.debug != that.debug)
+      if (!this.component_debug.equals(that.component_debug))
         return false;
     }
 
@@ -779,10 +794,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
     if (present_prev_status)
       list.add(prev_status.getValue());
 
-    boolean present_debug = true && (is_set_debug());
-    list.add(present_debug);
-    if (present_debug)
-      list.add(debug);
+    boolean present_component_debug = true && (is_set_component_debug());
+    list.add(present_component_debug);
+    if (present_component_debug)
+      list.add(component_debug);
 
     return list.hashCode();
   }
@@ -875,12 +890,12 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_debug()).compareTo(other.is_set_debug());
+    lastComparison = Boolean.valueOf(is_set_component_debug()).compareTo(other.is_set_component_debug());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (is_set_debug()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug, other.debug);
+    if (is_set_component_debug()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_debug, other.component_debug);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -970,10 +985,14 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       first = false;
     }
-    if (is_set_debug()) {
+    if (is_set_component_debug()) {
       if (!first) sb.append(", ");
-      sb.append("debug:");
-      sb.append(this.debug);
+      sb.append("component_debug:");
+      if (this.component_debug == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_debug);
+      }
       first = false;
     }
     sb.append(")");
@@ -1060,15 +1079,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 4: // COMPONENT_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map416 = iprot.readMapBegin();
-                struct.component_executors = new HashMap<String,Integer>(2*_map416.size);
-                String _key417;
-                int _val418;
-                for (int _i419 = 0; _i419 < _map416.size; ++_i419)
+                org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map426.size);
+                String _key427;
+                int _val428;
+                for (int _i429 = 0; _i429 < _map426.size; ++_i429)
                 {
-                  _key417 = iprot.readString();
-                  _val418 = iprot.readI32();
-                  struct.component_executors.put(_key417, _val418);
+                  _key427 = iprot.readString();
+                  _val428 = iprot.readI32();
+                  struct.component_executors.put(_key427, _val428);
                 }
                 iprot.readMapEnd();
               }
@@ -1110,10 +1129,22 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 9: // DEBUG
-            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-              struct.debug = iprot.readBool();
-              struct.set_debug_isSet(true);
+          case 9: // COMPONENT_DEBUG
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map430 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,Boolean>(2*_map430.size);
+                String _key431;
+                boolean _val432;
+                for (int _i433 = 0; _i433 < _map430.size; ++_i433)
+                {
+                  _key431 = iprot.readString();
+                  _val432 = iprot.readBool();
+                  struct.component_debug.put(_key431, _val432);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_component_debug_isSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
@@ -1149,10 +1180,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           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<String, Integer> _iter420 : struct.component_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter434 : struct.component_executors.entrySet())
             {
-              oprot.writeString(_iter420.getKey());
-              oprot.writeI32(_iter420.getValue());
+              oprot.writeString(_iter434.getKey());
+              oprot.writeI32(_iter434.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1185,10 +1216,20 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldEnd();
         }
       }
-      if (struct.is_set_debug()) {
-        oprot.writeFieldBegin(DEBUG_FIELD_DESC);
-        oprot.writeBool(struct.debug);
-        oprot.writeFieldEnd();
+      if (struct.component_debug != null) {
+        if (struct.is_set_component_debug()) {
+          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.BOOL, struct.component_debug.size()));
+            for (Map.Entry<String, Boolean> _iter435 : struct.component_debug.entrySet())
+            {
+              oprot.writeString(_iter435.getKey());
+              oprot.writeBool(_iter435.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
       }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
@@ -1226,17 +1267,17 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_prev_status()) {
         optionals.set(4);
       }
-      if (struct.is_set_debug()) {
+      if (struct.is_set_component_debug()) {
         optionals.set(5);
       }
       oprot.writeBitSet(optionals, 6);
       if (struct.is_set_component_executors()) {
         {
           oprot.writeI32(struct.component_executors.size());
-          for (Map.Entry<String, Integer> _iter421 : struct.component_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter436 : struct.component_executors.entrySet())
           {
-            oprot.writeString(_iter421.getKey());
-            oprot.writeI32(_iter421.getValue());
+            oprot.writeString(_iter436.getKey());
+            oprot.writeI32(_iter436.getValue());
           }
         }
       }
@@ -1252,8 +1293,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_prev_status()) {
         oprot.writeI32(struct.prev_status.getValue());
       }
-      if (struct.is_set_debug()) {
-        oprot.writeBool(struct.debug);
+      if (struct.is_set_component_debug()) {
+        {
+          oprot.writeI32(struct.component_debug.size());
+          for (Map.Entry<String, Boolean> _iter437 : struct.component_debug.entrySet())
+          {
+            oprot.writeString(_iter437.getKey());
+            oprot.writeBool(_iter437.getValue());
+          }
+        }
       }
     }
 
@@ -1269,15 +1317,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map422 = 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<String,Integer>(2*_map422.size);
-          String _key423;
-          int _val424;
-          for (int _i425 = 0; _i425 < _map422.size; ++_i425)
+          org.apache.thrift.protocol.TMap _map438 = 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<String,Integer>(2*_map438.size);
+          String _key439;
+          int _val440;
+          for (int _i441 = 0; _i441 < _map438.size; ++_i441)
           {
-            _key423 = iprot.readString();
-            _val424 = iprot.readI32();
-            struct.component_executors.put(_key423, _val424);
+            _key439 = iprot.readString();
+            _val440 = iprot.readI32();
+            struct.component_executors.put(_key439, _val440);
           }
         }
         struct.set_component_executors_isSet(true);
@@ -1300,8 +1348,19 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
         struct.set_prev_status_isSet(true);
       }
       if (incoming.get(5)) {
-        struct.debug = iprot.readBool();
-        struct.set_debug_isSet(true);
+        {
+          org.apache.thrift.protocol.TMap _map442 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BOOL, iprot.readI32());
+          struct.component_debug = new HashMap<String,Boolean>(2*_map442.size);
+          String _key443;
+          boolean _val444;
+          for (int _i445 = 0; _i445 < _map442.size; ++_i445)
+          {
+            _key443 = iprot.readString();
+            _val444 = iprot.readBool();
+            struct.component_debug.put(_key443, _val444);
+          }
+        }
+        struct.set_component_debug_isSet(true);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
index 9083b0d..e2b5ad1 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-20")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo");
 
@@ -990,13 +990,13 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 4: // USED_PORTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list336 = iprot.readListBegin();
-                struct.used_ports = new ArrayList<Long>(_list336.size);
-                long _elem337;
-                for (int _i338 = 0; _i338 < _list336.size; ++_i338)
+                org.apache.thrift.protocol.TList _list346 = iprot.readListBegin();
+                struct.used_ports = new ArrayList<Long>(_list346.size);
+                long _elem347;
+                for (int _i348 = 0; _i348 < _list346.size; ++_i348)
                 {
-                  _elem337 = iprot.readI64();
-                  struct.used_ports.add(_elem337);
+                  _elem347 = iprot.readI64();
+                  struct.used_ports.add(_elem347);
                 }
                 iprot.readListEnd();
               }
@@ -1008,13 +1008,13 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 5: // META
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list339 = iprot.readListBegin();
-                struct.meta = new ArrayList<Long>(_list339.size);
-                long _elem340;
-                for (int _i341 = 0; _i341 < _list339.size; ++_i341)
+                org.apache.thrift.protocol.TList _list349 = iprot.readListBegin();
+                struct.meta = new ArrayList<Long>(_list349.size);
+                long _elem350;
+                for (int _i351 = 0; _i351 < _list349.size; ++_i351)
                 {
-                  _elem340 = iprot.readI64();
-                  struct.meta.add(_elem340);
+                  _elem350 = iprot.readI64();
+                  struct.meta.add(_elem350);
                 }
                 iprot.readListEnd();
               }
@@ -1026,15 +1026,15 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 6: // SCHEDULER_META
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map342 = iprot.readMapBegin();
-                struct.scheduler_meta = new HashMap<String,String>(2*_map342.size);
-                String _key343;
-                String _val344;
-                for (int _i345 = 0; _i345 < _map342.size; ++_i345)
+                org.apache.thrift.protocol.TMap _map352 = iprot.readMapBegin();
+                struct.scheduler_meta = new HashMap<String,String>(2*_map352.size);
+                String _key353;
+                String _val354;
+                for (int _i355 = 0; _i355 < _map352.size; ++_i355)
                 {
-                  _key343 = iprot.readString();
-                  _val344 = iprot.readString();
-                  struct.scheduler_meta.put(_key343, _val344);
+                  _key353 = iprot.readString();
+                  _val354 = iprot.readString();
+                  struct.scheduler_meta.put(_key353, _val354);
                 }
                 iprot.readMapEnd();
               }
@@ -1092,9 +1092,9 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           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 _iter346 : struct.used_ports)
+            for (long _iter356 : struct.used_ports)
             {
-              oprot.writeI64(_iter346);
+              oprot.writeI64(_iter356);
             }
             oprot.writeListEnd();
           }
@@ -1106,9 +1106,9 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(META_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size()));
-            for (long _iter347 : struct.meta)
+            for (long _iter357 : struct.meta)
             {
-              oprot.writeI64(_iter347);
+              oprot.writeI64(_iter357);
             }
             oprot.writeListEnd();
           }
@@ -1120,10 +1120,10 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           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<String, String> _iter348 : struct.scheduler_meta.entrySet())
+            for (Map.Entry<String, String> _iter358 : struct.scheduler_meta.entrySet())
             {
-              oprot.writeString(_iter348.getKey());
-              oprot.writeString(_iter348.getValue());
+              oprot.writeString(_iter358.getKey());
+              oprot.writeString(_iter358.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1187,28 +1187,28 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       if (struct.is_set_used_ports()) {
         {
           oprot.writeI32(struct.used_ports.size());
-          for (long _iter349 : struct.used_ports)
+          for (long _iter359 : struct.used_ports)
           {
-            oprot.writeI64(_iter349);
+            oprot.writeI64(_iter359);
           }
         }
       }
       if (struct.is_set_meta()) {
         {
           oprot.writeI32(struct.meta.size());
-          for (long _iter350 : struct.meta)
+          for (long _iter360 : struct.meta)
           {
-            oprot.writeI64(_iter350);
+            oprot.writeI64(_iter360);
           }
         }
       }
       if (struct.is_set_scheduler_meta()) {
         {
           oprot.writeI32(struct.scheduler_meta.size());
-          for (Map.Entry<String, String> _iter351 : struct.scheduler_meta.entrySet())
+          for (Map.Entry<String, String> _iter361 : struct.scheduler_meta.entrySet())
           {
-            oprot.writeString(_iter351.getKey());
-            oprot.writeString(_iter351.getValue());
+            oprot.writeString(_iter361.getKey());
+            oprot.writeString(_iter361.getValue());
           }
         }
       }
@@ -1234,41 +1234,41 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list352 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.used_ports = new ArrayList<Long>(_list352.size);
-          long _elem353;
-          for (int _i354 = 0; _i354 < _list352.size; ++_i354)
+          org.apache.thrift.protocol.TList _list362 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.used_ports = new ArrayList<Long>(_list362.size);
+          long _elem363;
+          for (int _i364 = 0; _i364 < _list362.size; ++_i364)
           {
-            _elem353 = iprot.readI64();
-            struct.used_ports.add(_elem353);
+            _elem363 = iprot.readI64();
+            struct.used_ports.add(_elem363);
           }
         }
         struct.set_used_ports_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list355 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.meta = new ArrayList<Long>(_list355.size);
-          long _elem356;
-          for (int _i357 = 0; _i357 < _list355.size; ++_i357)
+          org.apache.thrift.protocol.TList _list365 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.meta = new ArrayList<Long>(_list365.size);
+          long _elem366;
+          for (int _i367 = 0; _i367 < _list365.size; ++_i367)
           {
-            _elem356 = iprot.readI64();
-            struct.meta.add(_elem356);
+            _elem366 = iprot.readI64();
+            struct.meta.add(_elem366);
           }
         }
         struct.set_meta_isSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map358 = 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<String,String>(2*_map358.size);
-          String _key359;
-          String _val360;
-          for (int _i361 = 0; _i361 < _map358.size; ++_i361)
+          org.apache.thrift.protocol.TMap _map368 = 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<String,String>(2*_map368.size);
+          String _key369;
+          String _val370;
+          for (int _i371 = 0; _i371 < _map368.size; ++_i371)
           {
-            _key359 = iprot.readString();
-            _val360 = iprot.readString();
-            struct.scheduler_meta.put(_key359, _val360);
+            _key369 = iprot.readString();
+            _val370 = iprot.readString();
+            struct.scheduler_meta.put(_key369, _val370);
           }
         }
         struct.set_scheduler_meta_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index 76b7c83..e050f24 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
 public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
 
@@ -61,7 +61,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)4);
   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)5);
   private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.MAP, (short)6);
-  private static final org.apache.thrift.protocol.TField DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("debug", org.apache.thrift.protocol.TType.BOOL, (short)7);
+  private static final org.apache.thrift.protocol.TField COMPONENT_DEBUG_FIELD_DESC = new org.apache.thrift.protocol.TField("component_debug", org.apache.thrift.protocol.TType.MAP, (short)7);
   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)513);
   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)514);
 
@@ -77,7 +77,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private List<ExecutorSummary> executors; // required
   private String status; // required
   private Map<String,List<ErrorInfo>> errors; // required
-  private boolean debug; // required
+  private Map<String,Boolean> component_debug; // required
   private String sched_status; // optional
   private String owner; // optional
 
@@ -89,7 +89,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     EXECUTORS((short)4, "executors"),
     STATUS((short)5, "status"),
     ERRORS((short)6, "errors"),
-    DEBUG((short)7, "debug"),
+    COMPONENT_DEBUG((short)7, "component_debug"),
     SCHED_STATUS((short)513, "sched_status"),
     OWNER((short)514, "owner");
 
@@ -118,8 +118,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           return STATUS;
         case 6: // ERRORS
           return ERRORS;
-        case 7: // DEBUG
-          return DEBUG;
+        case 7: // COMPONENT_DEBUG
+          return COMPONENT_DEBUG;
         case 513: // SCHED_STATUS
           return SCHED_STATUS;
         case 514: // OWNER
@@ -165,7 +165,6 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
 
   // isset id assignments
   private static final int __UPTIME_SECS_ISSET_ID = 0;
-  private static final int __DEBUG_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
   private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
@@ -187,8 +186,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             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.DEBUG, new org.apache.thrift.meta_data.FieldMetaData("debug", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        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.BOOL))));
     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.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
@@ -207,7 +208,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     List<ExecutorSummary> executors,
     String status,
     Map<String,List<ErrorInfo>> errors,
-    boolean debug)
+    Map<String,Boolean> component_debug)
   {
     this();
     this.id = id;
@@ -217,8 +218,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     this.executors = executors;
     this.status = status;
     this.errors = errors;
-    this.debug = debug;
-    set_debug_isSet(true);
+    this.component_debug = component_debug;
   }
 
   /**
@@ -261,7 +261,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       this.errors = __this__errors;
     }
-    this.debug = other.debug;
+    if (other.is_set_component_debug()) {
+      Map<String,Boolean> __this__component_debug = new HashMap<String,Boolean>(other.component_debug);
+      this.component_debug = __this__component_debug;
+    }
     if (other.is_set_sched_status()) {
       this.sched_status = other.sched_status;
     }
@@ -283,8 +286,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     this.executors = null;
     this.status = null;
     this.errors = null;
-    set_debug_isSet(false);
-    this.debug = false;
+    this.component_debug = null;
     this.sched_status = null;
     this.owner = null;
   }
@@ -452,26 +454,38 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     }
   }
 
-  public boolean is_debug() {
-    return this.debug;
+  public int get_component_debug_size() {
+    return (this.component_debug == null) ? 0 : this.component_debug.size();
   }
 
-  public void set_debug(boolean debug) {
-    this.debug = debug;
-    set_debug_isSet(true);
+  public void put_to_component_debug(String key, boolean val) {
+    if (this.component_debug == null) {
+      this.component_debug = new HashMap<String,Boolean>();
+    }
+    this.component_debug.put(key, val);
   }
 
-  public void unset_debug() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  public Map<String,Boolean> get_component_debug() {
+    return this.component_debug;
   }
 
-  /** Returns true if field debug is set (has been assigned a value) and false otherwise */
-  public boolean is_set_debug() {
-    return EncodingUtils.testBit(__isset_bitfield, __DEBUG_ISSET_ID);
+  public void set_component_debug(Map<String,Boolean> component_debug) {
+    this.component_debug = component_debug;
   }
 
-  public void set_debug_isSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DEBUG_ISSET_ID, value);
+  public void unset_component_debug() {
+    this.component_debug = null;
+  }
+
+  /** Returns true if field component_debug is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_debug() {
+    return this.component_debug != null;
+  }
+
+  public void set_component_debug_isSet(boolean value) {
+    if (!value) {
+      this.component_debug = null;
+    }
   }
 
   public String get_sched_status() {
@@ -570,11 +584,11 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       break;
 
-    case DEBUG:
+    case COMPONENT_DEBUG:
       if (value == null) {
-        unset_debug();
+        unset_component_debug();
       } else {
-        set_debug((Boolean)value);
+        set_component_debug((Map<String,Boolean>)value);
       }
       break;
 
@@ -617,8 +631,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     case ERRORS:
       return get_errors();
 
-    case DEBUG:
-      return Boolean.valueOf(is_debug());
+    case COMPONENT_DEBUG:
+      return get_component_debug();
 
     case SCHED_STATUS:
       return get_sched_status();
@@ -649,8 +663,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       return is_set_status();
     case ERRORS:
       return is_set_errors();
-    case DEBUG:
-      return is_set_debug();
+    case COMPONENT_DEBUG:
+      return is_set_component_debug();
     case SCHED_STATUS:
       return is_set_sched_status();
     case OWNER:
@@ -726,12 +740,12 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return false;
     }
 
-    boolean this_present_debug = true;
-    boolean that_present_debug = true;
-    if (this_present_debug || that_present_debug) {
-      if (!(this_present_debug && that_present_debug))
+    boolean this_present_component_debug = true && this.is_set_component_debug();
+    boolean that_present_component_debug = true && that.is_set_component_debug();
+    if (this_present_component_debug || that_present_component_debug) {
+      if (!(this_present_component_debug && that_present_component_debug))
         return false;
-      if (this.debug != that.debug)
+      if (!this.component_debug.equals(that.component_debug))
         return false;
     }
 
@@ -790,10 +804,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     if (present_errors)
       list.add(errors);
 
-    boolean present_debug = true;
-    list.add(present_debug);
-    if (present_debug)
-      list.add(debug);
+    boolean present_component_debug = true && (is_set_component_debug());
+    list.add(present_component_debug);
+    if (present_component_debug)
+      list.add(component_debug);
 
     boolean present_sched_status = true && (is_set_sched_status());
     list.add(present_sched_status);
@@ -876,12 +890,12 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_debug()).compareTo(other.is_set_debug());
+    lastComparison = Boolean.valueOf(is_set_component_debug()).compareTo(other.is_set_component_debug());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (is_set_debug()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug, other.debug);
+    if (is_set_component_debug()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_debug, other.component_debug);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -970,8 +984,12 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     }
     first = false;
     if (!first) sb.append(", ");
-    sb.append("debug:");
-    sb.append(this.debug);
+    sb.append("component_debug:");
+    if (this.component_debug == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.component_debug);
+    }
     first = false;
     if (is_set_sched_status()) {
       if (!first) sb.append(", ");
@@ -1023,8 +1041,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'errors' is unset! Struct:" + toString());
     }
 
-    if (!is_set_debug()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'debug' is unset! Struct:" + toString());
+    if (!is_set_component_debug()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_debug' is unset! Struct:" + toString());
     }
 
     // check for sub-struct validity
@@ -1148,10 +1166,22 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // DEBUG
-            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-              struct.debug = iprot.readBool();
-              struct.set_debug_isSet(true);
+          case 7: // COMPONENT_DEBUG
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map300 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,Boolean>(2*_map300.size);
+                String _key301;
+                boolean _val302;
+                for (int _i303 = 0; _i303 < _map300.size; ++_i303)
+                {
+                  _key301 = iprot.readString();
+                  _val302 = iprot.readBool();
+                  struct.component_debug.put(_key301, _val302);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_component_debug_isSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
@@ -1202,9 +1232,9 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorSummary _iter300 : struct.executors)
+          for (ExecutorSummary _iter304 : struct.executors)
           {
-            _iter300.write(oprot);
+            _iter304.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1219,14 +1249,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         oprot.writeFieldBegin(ERRORS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.errors.size()));
-          for (Map.Entry<String, List<ErrorInfo>> _iter301 : struct.errors.entrySet())
+          for (Map.Entry<String, List<ErrorInfo>> _iter305 : struct.errors.entrySet())
           {
-            oprot.writeString(_iter301.getKey());
+            oprot.writeString(_iter305.getKey());
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter301.getValue().size()));
-              for (ErrorInfo _iter302 : _iter301.getValue())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter305.getValue().size()));
+              for (ErrorInfo _iter306 : _iter305.getValue())
               {
-                _iter302.write(oprot);
+                _iter306.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -1235,9 +1265,19 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         }
         oprot.writeFieldEnd();
       }
-      oprot.writeFieldBegin(DEBUG_FIELD_DESC);
-      oprot.writeBool(struct.debug);
-      oprot.writeFieldEnd();
+      if (struct.component_debug != null) {
+        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.BOOL, struct.component_debug.size()));
+          for (Map.Entry<String, Boolean> _iter307 : struct.component_debug.entrySet())
+          {
+            oprot.writeString(_iter307.getKey());
+            oprot.writeBool(_iter307.getValue());
+          }
+          oprot.writeMapEnd();
+        }
+        oprot.writeFieldEnd();
+      }
       if (struct.sched_status != null) {
         if (struct.is_set_sched_status()) {
           oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
@@ -1274,27 +1314,34 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       oprot.writeI32(struct.uptime_secs);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorSummary _iter303 : struct.executors)
+        for (ExecutorSummary _iter308 : struct.executors)
         {
-          _iter303.write(oprot);
+          _iter308.write(oprot);
         }
       }
       oprot.writeString(struct.status);
       {
         oprot.writeI32(struct.errors.size());
-        for (Map.Entry<String, List<ErrorInfo>> _iter304 : struct.errors.entrySet())
+        for (Map.Entry<String, List<ErrorInfo>> _iter309 : struct.errors.entrySet())
         {
-          oprot.writeString(_iter304.getKey());
+          oprot.writeString(_iter309.getKey());
           {
-            oprot.writeI32(_iter304.getValue().size());
-            for (ErrorInfo _iter305 : _iter304.getValue())
+            oprot.writeI32(_iter309.getValue().size());
+            for (ErrorInfo _iter310 : _iter309.getValue())
             {
-              _iter305.write(oprot);
+              _iter310.write(oprot);
             }
           }
         }
       }
-      oprot.writeBool(struct.debug);
+      {
+        oprot.writeI32(struct.component_debug.size());
+        for (Map.Entry<String, Boolean> _iter311 : struct.component_debug.entrySet())
+        {
+          oprot.writeString(_iter311.getKey());
+          oprot.writeBool(_iter311.getValue());
+        }
+      }
       BitSet optionals = new BitSet();
       if (struct.is_set_sched_status()) {
         optionals.set(0);
@@ -1321,44 +1368,55 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       struct.uptime_secs = iprot.readI32();
       struct.set_uptime_secs_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list306 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorSummary>(_list306.size);
-        ExecutorSummary _elem307;
-        for (int _i308 = 0; _i308 < _list306.size; ++_i308)
+        org.apache.thrift.protocol.TList _list312 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorSummary>(_list312.size);
+        ExecutorSummary _elem313;
+        for (int _i314 = 0; _i314 < _list312.size; ++_i314)
         {
-          _elem307 = new ExecutorSummary();
-          _elem307.read(iprot);
-          struct.executors.add(_elem307);
+          _elem313 = new ExecutorSummary();
+          _elem313.read(iprot);
+          struct.executors.add(_elem313);
         }
       }
       struct.set_executors_isSet(true);
       struct.status = iprot.readString();
       struct.set_status_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map309 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-        struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map309.size);
-        String _key310;
-        List<ErrorInfo> _val311;
-        for (int _i312 = 0; _i312 < _map309.size; ++_i312)
+        org.apache.thrift.protocol.TMap _map315 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+        struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map315.size);
+        String _key316;
+        List<ErrorInfo> _val317;
+        for (int _i318 = 0; _i318 < _map315.size; ++_i318)
         {
-          _key310 = iprot.readString();
+          _key316 = iprot.readString();
           {
-            org.apache.thrift.protocol.TList _list313 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            _val311 = new ArrayList<ErrorInfo>(_list313.size);
-            ErrorInfo _elem314;
-            for (int _i315 = 0; _i315 < _list313.size; ++_i315)
+            org.apache.thrift.protocol.TList _list319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val317 = new ArrayList<ErrorInfo>(_list319.size);
+            ErrorInfo _elem320;
+            for (int _i321 = 0; _i321 < _list319.size; ++_i321)
             {
-              _elem314 = new ErrorInfo();
-              _elem314.read(iprot);
-              _val311.add(_elem314);
+              _elem320 = new ErrorInfo();
+              _elem320.read(iprot);
+              _val317.add(_elem320);
             }
           }
-          struct.errors.put(_key310, _val311);
+          struct.errors.put(_key316, _val317);
         }
       }
       struct.set_errors_isSet(true);
-      struct.debug = iprot.readBool();
-      struct.set_debug_isSet(true);
+      {
+        org.apache.thrift.protocol.TMap _map322 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BOOL, iprot.readI32());
+        struct.component_debug = new HashMap<String,Boolean>(2*_map322.size);
+        String _key323;
+        boolean _val324;
+        for (int _i325 = 0; _i325 < _map322.size; ++_i325)
+        {
+          _key323 = iprot.readString();
+          _val324 = iprot.readBool();
+          struct.component_debug.put(_key323, _val324);
+        }
+      }
+      struct.set_component_debug_isSet(true);
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.sched_status = iprot.readString();

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/py/storm/Nimbus-remote
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote
index d012659..f58c325 100644
--- a/storm-core/src/py/storm/Nimbus-remote
+++ b/storm-core/src/py/storm/Nimbus-remote
@@ -49,7 +49,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void activate(string name)')
   print('  void deactivate(string name)')
   print('  void rebalance(string name, RebalanceOptions options)')
-  print('  void debug(string name, bool enable)')
+  print('  void debug(string name, string component, bool enable)')
   print('  void uploadNewCredentials(string name, Credentials creds)')
   print('  string beginFileUpload()')
   print('  void uploadChunk(string location, string chunk)')
@@ -162,10 +162,10 @@ elif cmd == 'rebalance':
   pp.pprint(client.rebalance(args[0],eval(args[1]),))
 
 elif cmd == 'debug':
-  if len(args) != 2:
-    print('debug requires 2 args')
+  if len(args) != 3:
+    print('debug requires 3 args')
     sys.exit(1)
-  pp.pprint(client.debug(args[0],eval(args[1]),))
+  pp.pprint(client.debug(args[0],args[1],eval(args[2]),))
 
 elif cmd == 'uploadNewCredentials':
   if len(args) != 2:

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index 9e6923d..2aff10d 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -94,10 +94,11 @@ class Iface:
     """
     pass
 
-  def debug(self, name, enable):
+  def debug(self, name, component, enable):
     """
     Parameters:
      - name
+     - component
      - enable
     """
     pass
@@ -451,19 +452,21 @@ class Client(Iface):
       raise result.aze
     return
 
-  def debug(self, name, enable):
+  def debug(self, name, component, enable):
     """
     Parameters:
      - name
+     - component
      - enable
     """
-    self.send_debug(name, enable)
+    self.send_debug(name, component, enable)
     self.recv_debug()
 
-  def send_debug(self, name, enable):
+  def send_debug(self, name, component, enable):
     self._oprot.writeMessageBegin('debug', TMessageType.CALL, self._seqid)
     args = debug_args()
     args.name = name
+    args.component = component
     args.enable = enable
     args.write(self._oprot)
     self._oprot.writeMessageEnd()
@@ -1084,7 +1087,7 @@ class Processor(Iface, TProcessor):
     iprot.readMessageEnd()
     result = debug_result()
     try:
-      self._handler.debug(args.name, args.enable)
+      self._handler.debug(args.name, args.component, args.enable)
     except NotAliveException, e:
       result.e = e
     except AuthorizationException, aze:
@@ -2476,17 +2479,20 @@ class debug_args:
   """
   Attributes:
    - name
+   - component
    - enable
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRING, 'name', None, None, ), # 1
-    (2, TType.BOOL, 'enable', None, None, ), # 2
+    (2, TType.STRING, 'component', None, None, ), # 2
+    (3, TType.BOOL, 'enable', None, None, ), # 3
   )
 
-  def __init__(self, name=None, enable=None,):
+  def __init__(self, name=None, component=None, enable=None,):
     self.name = name
+    self.component = component
     self.enable = enable
 
   def read(self, iprot):
@@ -2504,6 +2510,11 @@ class debug_args:
         else:
           iprot.skip(ftype)
       elif fid == 2:
+        if ftype == TType.STRING:
+          self.component = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
         if ftype == TType.BOOL:
           self.enable = iprot.readBool();
         else:
@@ -2522,8 +2533,12 @@ class debug_args:
       oprot.writeFieldBegin('name', TType.STRING, 1)
       oprot.writeString(self.name.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.component is not None:
+      oprot.writeFieldBegin('component', TType.STRING, 2)
+      oprot.writeString(self.component.encode('utf-8'))
+      oprot.writeFieldEnd()
     if self.enable is not None:
-      oprot.writeFieldBegin('enable', TType.BOOL, 2)
+      oprot.writeFieldBegin('enable', TType.BOOL, 3)
       oprot.writeBool(self.enable)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -2536,6 +2551,7 @@ class debug_args:
   def __hash__(self):
     value = 17
     value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.component)
     value = (value * 31) ^ hash(self.enable)
     return value
 

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index 46e64ed..fe9663e 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -3566,7 +3566,7 @@ class TopologyInfo:
    - executors
    - status
    - errors
-   - debug
+   - component_debug
    - sched_status
    - owner
   """
@@ -3579,7 +3579,7 @@ class TopologyInfo:
     (4, TType.LIST, 'executors', (TType.STRUCT,(ExecutorSummary, ExecutorSummary.thrift_spec)), None, ), # 4
     (5, TType.STRING, 'status', None, None, ), # 5
     (6, TType.MAP, 'errors', (TType.STRING,None,TType.LIST,(TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec))), None, ), # 6
-    (7, TType.BOOL, 'debug', None, None, ), # 7
+    (7, TType.MAP, 'component_debug', (TType.STRING,None,TType.BOOL,None), None, ), # 7
     None, # 8
     None, # 9
     None, # 10
@@ -4089,14 +4089,14 @@ class TopologyInfo:
     (514, TType.STRING, 'owner', None, None, ), # 514
   )
 
-  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, debug=None, sched_status=None, owner=None,):
+  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, component_debug=None, sched_status=None, owner=None,):
     self.id = id
     self.name = name
     self.uptime_secs = uptime_secs
     self.executors = executors
     self.status = status
     self.errors = errors
-    self.debug = debug
+    self.component_debug = component_debug
     self.sched_status = sched_status
     self.owner = owner
 
@@ -4158,8 +4158,14 @@ class TopologyInfo:
         else:
           iprot.skip(ftype)
       elif fid == 7:
-        if ftype == TType.BOOL:
-          self.debug = iprot.readBool();
+        if ftype == TType.MAP:
+          self.component_debug = {}
+          (_ktype280, _vtype281, _size279 ) = iprot.readMapBegin()
+          for _i283 in xrange(_size279):
+            _key284 = iprot.readString().decode('utf-8')
+            _val285 = iprot.readBool();
+            self.component_debug[_key284] = _val285
+          iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 513:
@@ -4197,8 +4203,8 @@ class TopologyInfo:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter279 in self.executors:
-        iter279.write(oprot)
+      for iter286 in self.executors:
+        iter286.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.status is not None:
@@ -4208,17 +4214,21 @@ class TopologyInfo:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.LIST, len(self.errors))
-      for kiter280,viter281 in self.errors.items():
-        oprot.writeString(kiter280.encode('utf-8'))
-        oprot.writeListBegin(TType.STRUCT, len(viter281))
-        for iter282 in viter281:
-          iter282.write(oprot)
+      for kiter287,viter288 in self.errors.items():
+        oprot.writeString(kiter287.encode('utf-8'))
+        oprot.writeListBegin(TType.STRUCT, len(viter288))
+        for iter289 in viter288:
+          iter289.write(oprot)
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
-    if self.debug is not None:
-      oprot.writeFieldBegin('debug', TType.BOOL, 7)
-      oprot.writeBool(self.debug)
+    if self.component_debug is not None:
+      oprot.writeFieldBegin('component_debug', TType.MAP, 7)
+      oprot.writeMapBegin(TType.STRING, TType.BOOL, len(self.component_debug))
+      for kiter290,viter291 in self.component_debug.items():
+        oprot.writeString(kiter290.encode('utf-8'))
+        oprot.writeBool(viter291)
+      oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.sched_status is not None:
       oprot.writeFieldBegin('sched_status', TType.STRING, 513)
@@ -4244,8 +4254,8 @@ class TopologyInfo:
       raise TProtocol.TProtocolException(message='Required field status is unset!')
     if self.errors is None:
       raise TProtocol.TProtocolException(message='Required field errors is unset!')
-    if self.debug is None:
-      raise TProtocol.TProtocolException(message='Required field debug is unset!')
+    if self.component_debug is None:
+      raise TProtocol.TProtocolException(message='Required field component_debug is unset!')
     return
 
 
@@ -4257,7 +4267,7 @@ class TopologyInfo:
     value = (value * 31) ^ hash(self.executors)
     value = (value * 31) ^ hash(self.status)
     value = (value * 31) ^ hash(self.errors)
-    value = (value * 31) ^ hash(self.debug)
+    value = (value * 31) ^ hash(self.component_debug)
     value = (value * 31) ^ hash(self.sched_status)
     value = (value * 31) ^ hash(self.owner)
     return value
@@ -4380,11 +4390,11 @@ class RebalanceOptions:
       elif fid == 3:
         if ftype == TType.MAP:
           self.num_executors = {}
-          (_ktype284, _vtype285, _size283 ) = iprot.readMapBegin()
-          for _i287 in xrange(_size283):
-            _key288 = iprot.readString().decode('utf-8')
-            _val289 = iprot.readI32();
-            self.num_executors[_key288] = _val289
+          (_ktype293, _vtype294, _size292 ) = iprot.readMapBegin()
+          for _i296 in xrange(_size292):
+            _key297 = iprot.readString().decode('utf-8')
+            _val298 = iprot.readI32();
+            self.num_executors[_key297] = _val298
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4409,9 +4419,9 @@ class RebalanceOptions:
     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 kiter290,viter291 in self.num_executors.items():
-        oprot.writeString(kiter290.encode('utf-8'))
-        oprot.writeI32(viter291)
+      for kiter299,viter300 in self.num_executors.items():
+        oprot.writeString(kiter299.encode('utf-8'))
+        oprot.writeI32(viter300)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4465,11 +4475,11 @@ class Credentials:
       if fid == 1:
         if ftype == TType.MAP:
           self.creds = {}
-          (_ktype293, _vtype294, _size292 ) = iprot.readMapBegin()
-          for _i296 in xrange(_size292):
-            _key297 = iprot.readString().decode('utf-8')
-            _val298 = iprot.readString().decode('utf-8')
-            self.creds[_key297] = _val298
+          (_ktype302, _vtype303, _size301 ) = iprot.readMapBegin()
+          for _i305 in xrange(_size301):
+            _key306 = iprot.readString().decode('utf-8')
+            _val307 = iprot.readString().decode('utf-8')
+            self.creds[_key306] = _val307
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4486,9 +4496,9 @@ class Credentials:
     if self.creds is not None:
       oprot.writeFieldBegin('creds', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds))
-      for kiter299,viter300 in self.creds.items():
-        oprot.writeString(kiter299.encode('utf-8'))
-        oprot.writeString(viter300.encode('utf-8'))
+      for kiter308,viter309 in self.creds.items():
+        oprot.writeString(kiter308.encode('utf-8'))
+        oprot.writeString(viter309.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4659,31 +4669,31 @@ class SupervisorInfo:
       elif fid == 4:
         if ftype == TType.LIST:
           self.used_ports = []
-          (_etype304, _size301) = iprot.readListBegin()
-          for _i305 in xrange(_size301):
-            _elem306 = iprot.readI64();
-            self.used_ports.append(_elem306)
+          (_etype313, _size310) = iprot.readListBegin()
+          for _i314 in xrange(_size310):
+            _elem315 = iprot.readI64();
+            self.used_ports.append(_elem315)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.meta = []
-          (_etype310, _size307) = iprot.readListBegin()
-          for _i311 in xrange(_size307):
-            _elem312 = iprot.readI64();
-            self.meta.append(_elem312)
+          (_etype319, _size316) = iprot.readListBegin()
+          for _i320 in xrange(_size316):
+            _elem321 = iprot.readI64();
+            self.meta.append(_elem321)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.MAP:
           self.scheduler_meta = {}
-          (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin()
-          for _i317 in xrange(_size313):
-            _key318 = iprot.readString().decode('utf-8')
-            _val319 = iprot.readString().decode('utf-8')
-            self.scheduler_meta[_key318] = _val319
+          (_ktype323, _vtype324, _size322 ) = iprot.readMapBegin()
+          for _i326 in xrange(_size322):
+            _key327 = iprot.readString().decode('utf-8')
+            _val328 = iprot.readString().decode('utf-8')
+            self.scheduler_meta[_key327] = _val328
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4722,23 +4732,23 @@ class SupervisorInfo:
     if self.used_ports is not None:
       oprot.writeFieldBegin('used_ports', TType.LIST, 4)
       oprot.writeListBegin(TType.I64, len(self.used_ports))
-      for iter320 in self.used_ports:
-        oprot.writeI64(iter320)
+      for iter329 in self.used_ports:
+        oprot.writeI64(iter329)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.meta is not None:
       oprot.writeFieldBegin('meta', TType.LIST, 5)
       oprot.writeListBegin(TType.I64, len(self.meta))
-      for iter321 in self.meta:
-        oprot.writeI64(iter321)
+      for iter330 in self.meta:
+        oprot.writeI64(iter330)
       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 kiter322,viter323 in self.scheduler_meta.items():
-        oprot.writeString(kiter322.encode('utf-8'))
-        oprot.writeString(viter323.encode('utf-8'))
+      for kiter331,viter332 in self.scheduler_meta.items():
+        oprot.writeString(kiter331.encode('utf-8'))
+        oprot.writeString(viter332.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.uptime_secs is not None:
@@ -4817,10 +4827,10 @@ class NodeInfo:
       elif fid == 2:
         if ftype == TType.SET:
           self.port = set()
-          (_etype327, _size324) = iprot.readSetBegin()
-          for _i328 in xrange(_size324):
-            _elem329 = iprot.readI64();
-            self.port.add(_elem329)
+          (_etype336, _size333) = iprot.readSetBegin()
+          for _i337 in xrange(_size333):
+            _elem338 = iprot.readI64();
+            self.port.add(_elem338)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -4841,8 +4851,8 @@ class NodeInfo:
     if self.port is not None:
       oprot.writeFieldBegin('port', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.port))
-      for iter330 in self.port:
-        oprot.writeI64(iter330)
+      for iter339 in self.port:
+        oprot.writeI64(iter339)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4925,44 +4935,44 @@ class Assignment:
       elif fid == 2:
         if ftype == TType.MAP:
           self.node_host = {}
-          (_ktype332, _vtype333, _size331 ) = iprot.readMapBegin()
-          for _i335 in xrange(_size331):
-            _key336 = iprot.readString().decode('utf-8')
-            _val337 = iprot.readString().decode('utf-8')
-            self.node_host[_key336] = _val337
+          (_ktype341, _vtype342, _size340 ) = iprot.readMapBegin()
+          for _i344 in xrange(_size340):
+            _key345 = iprot.readString().decode('utf-8')
+            _val346 = iprot.readString().decode('utf-8')
+            self.node_host[_key345] = _val346
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.executor_node_port = {}
-          (_ktype339, _vtype340, _size338 ) = iprot.readMapBegin()
-          for _i342 in xrange(_size338):
-            _key343 = []
-            (_etype348, _size345) = iprot.readListBegin()
-            for _i349 in xrange(_size345):
-              _elem350 = iprot.readI64();
-              _key343.append(_elem350)
+          (_ktype348, _vtype349, _size347 ) = iprot.readMapBegin()
+          for _i351 in xrange(_size347):
+            _key352 = []
+            (_etype357, _size354) = iprot.readListBegin()
+            for _i358 in xrange(_size354):
+              _elem359 = iprot.readI64();
+              _key352.append(_elem359)
             iprot.readListEnd()
-            _val344 = NodeInfo()
-            _val344.read(iprot)
-            self.executor_node_port[_key343] = _val344
+            _val353 = NodeInfo()
+            _val353.read(iprot)
+            self.executor_node_port[_key352] = _val353
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.executor_start_time_secs = {}
-          (_ktype352, _vtype353, _size351 ) = iprot.readMapBegin()
-          for _i355 in xrange(_size351):
-            _key356 = []
-            (_etype361, _size358) = iprot.readListBegin()
-            for _i362 in xrange(_size358):
-              _elem363 = iprot.readI64();
-              _key356.append(_elem363)
+          (_ktype361, _vtype362, _size360 ) = iprot.readMapBegin()
+          for _i364 in xrange(_size360):
+            _key365 = []
+            (_etype370, _size367) = iprot.readListBegin()
+            for _i371 in xrange(_size367):
+              _elem372 = iprot.readI64();
+              _key365.append(_elem372)
             iprot.readListEnd()
-            _val357 = iprot.readI64();
-            self.executor_start_time_secs[_key356] = _val357
+            _val366 = iprot.readI64();
+            self.executor_start_time_secs[_key365] = _val366
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4983,31 +4993,31 @@ class Assignment:
     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 kiter364,viter365 in self.node_host.items():
-        oprot.writeString(kiter364.encode('utf-8'))
-        oprot.writeString(viter365.encode('utf-8'))
+      for kiter373,viter374 in self.node_host.items():
+        oprot.writeString(kiter373.encode('utf-8'))
+        oprot.writeString(viter374.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 kiter366,viter367 in self.executor_node_port.items():
-        oprot.writeListBegin(TType.I64, len(kiter366))
-        for iter368 in kiter366:
-          oprot.writeI64(iter368)
+      for kiter375,viter376 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter375))
+        for iter377 in kiter375:
+          oprot.writeI64(iter377)
         oprot.writeListEnd()
-        viter367.write(oprot)
+        viter376.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 kiter369,viter370 in self.executor_start_time_secs.items():
-        oprot.writeListBegin(TType.I64, len(kiter369))
-        for iter371 in kiter369:
-          oprot.writeI64(iter371)
+      for kiter378,viter379 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter378))
+        for iter380 in kiter378:
+          oprot.writeI64(iter380)
         oprot.writeListEnd()
-        oprot.writeI64(viter370)
+        oprot.writeI64(viter379)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5129,7 +5139,7 @@ class StormBase:
    - owner
    - topology_action_options
    - prev_status
-   - debug
+   - component_debug
   """
 
   thrift_spec = (
@@ -5142,10 +5152,10 @@ class StormBase:
     (6, TType.STRING, 'owner', None, None, ), # 6
     (7, TType.STRUCT, 'topology_action_options', (TopologyActionOptions, TopologyActionOptions.thrift_spec), None, ), # 7
     (8, TType.I32, 'prev_status', None, None, ), # 8
-    (9, TType.BOOL, 'debug', None, None, ), # 9
+    (9, TType.MAP, 'component_debug', (TType.STRING,None,TType.BOOL,None), None, ), # 9
   )
 
-  def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None, debug=None,):
+  def __init__(self, name=None, status=None, num_workers=None, component_executors=None, launch_time_secs=None, owner=None, topology_action_options=None, prev_status=None, component_debug=None,):
     self.name = name
     self.status = status
     self.num_workers = num_workers
@@ -5154,7 +5164,7 @@ class StormBase:
     self.owner = owner
     self.topology_action_options = topology_action_options
     self.prev_status = prev_status
-    self.debug = debug
+    self.component_debug = component_debug
 
   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:
@@ -5183,11 +5193,11 @@ class StormBase:
       elif fid == 4:
         if ftype == TType.MAP:
           self.component_executors = {}
-          (_ktype373, _vtype374, _size372 ) = iprot.readMapBegin()
-          for _i376 in xrange(_size372):
-            _key377 = iprot.readString().decode('utf-8')
-            _val378 = iprot.readI32();
-            self.component_executors[_key377] = _val378
+          (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin()
+          for _i385 in xrange(_size381):
+            _key386 = iprot.readString().decode('utf-8')
+            _val387 = iprot.readI32();
+            self.component_executors[_key386] = _val387
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5213,8 +5223,14 @@ class StormBase:
         else:
           iprot.skip(ftype)
       elif fid == 9:
-        if ftype == TType.BOOL:
-          self.debug = iprot.readBool();
+        if ftype == TType.MAP:
+          self.component_debug = {}
+          (_ktype389, _vtype390, _size388 ) = iprot.readMapBegin()
+          for _i392 in xrange(_size388):
+            _key393 = iprot.readString().decode('utf-8')
+            _val394 = iprot.readBool();
+            self.component_debug[_key393] = _val394
+          iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       else:
@@ -5242,9 +5258,9 @@ class StormBase:
     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 kiter379,viter380 in self.component_executors.items():
-        oprot.writeString(kiter379.encode('utf-8'))
-        oprot.writeI32(viter380)
+      for kiter395,viter396 in self.component_executors.items():
+        oprot.writeString(kiter395.encode('utf-8'))
+        oprot.writeI32(viter396)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.launch_time_secs is not None:
@@ -5263,9 +5279,13 @@ class StormBase:
       oprot.writeFieldBegin('prev_status', TType.I32, 8)
       oprot.writeI32(self.prev_status)
       oprot.writeFieldEnd()
-    if self.debug is not None:
-      oprot.writeFieldBegin('debug', TType.BOOL, 9)
-      oprot.writeBool(self.debug)
+    if self.component_debug is not None:
+      oprot.writeFieldBegin('component_debug', TType.MAP, 9)
+      oprot.writeMapBegin(TType.STRING, TType.BOOL, len(self.component_debug))
+      for kiter397,viter398 in self.component_debug.items():
+        oprot.writeString(kiter397.encode('utf-8'))
+        oprot.writeBool(viter398)
+      oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -5290,7 +5310,7 @@ class StormBase:
     value = (value * 31) ^ hash(self.owner)
     value = (value * 31) ^ hash(self.topology_action_options)
     value = (value * 31) ^ hash(self.prev_status)
-    value = (value * 31) ^ hash(self.debug)
+    value = (value * 31) ^ hash(self.component_debug)
     return value
 
   def __repr__(self):
@@ -5344,13 +5364,13 @@ class ClusterWorkerHeartbeat:
       elif fid == 2:
         if ftype == TType.MAP:
           self.executor_stats = {}
-          (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin()
-          for _i385 in xrange(_size381):
-            _key386 = ExecutorInfo()
-            _key386.read(iprot)
-            _val387 = ExecutorStats()
-            _val387.read(iprot)
-            self.executor_stats[_key386] = _val387
+          (_ktype400, _vtype401, _size399 ) = iprot.readMapBegin()
+          for _i403 in xrange(_size399):
+            _key404 = ExecutorInfo()
+            _key404.read(iprot)
+            _val405 = ExecutorStats()
+            _val405.read(iprot)
+            self.executor_stats[_key404] = _val405
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5381,9 +5401,9 @@ class ClusterWorkerHeartbeat:
     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 kiter388,viter389 in self.executor_stats.items():
-        kiter388.write(oprot)
-        viter389.write(oprot)
+      for kiter406,viter407 in self.executor_stats.items():
+        kiter406.write(oprot)
+        viter407.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.time_secs is not None:
@@ -5536,12 +5556,12 @@ class LocalStateData:
       if fid == 1:
         if ftype == TType.MAP:
           self.serialized_parts = {}
-          (_ktype391, _vtype392, _size390 ) = iprot.readMapBegin()
-          for _i394 in xrange(_size390):
-            _key395 = iprot.readString().decode('utf-8')
-            _val396 = ThriftSerializedObject()
-            _val396.read(iprot)
-            self.serialized_parts[_key395] = _val396
+          (_ktype409, _vtype410, _size408 ) = iprot.readMapBegin()
+          for _i412 in xrange(_size408):
+            _key413 = iprot.readString().decode('utf-8')
+            _val414 = ThriftSerializedObject()
+            _val414.read(iprot)
+            self.serialized_parts[_key413] = _val414
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5558,9 +5578,9 @@ class LocalStateData:
     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 kiter397,viter398 in self.serialized_parts.items():
-        oprot.writeString(kiter397.encode('utf-8'))
-        viter398.write(oprot)
+      for kiter415,viter416 in self.serialized_parts.items():
+        oprot.writeString(kiter415.encode('utf-8'))
+        viter416.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5622,11 +5642,11 @@ class LocalAssignment:
       elif fid == 2:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype402, _size399) = iprot.readListBegin()
-          for _i403 in xrange(_size399):
-            _elem404 = ExecutorInfo()
-            _elem404.read(iprot)
-            self.executors.append(_elem404)
+          (_etype420, _size417) = iprot.readListBegin()
+          for _i421 in xrange(_size417):
+            _elem422 = ExecutorInfo()
+            _elem422.read(iprot)
+            self.executors.append(_elem422)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5647,8 +5667,8 @@ class LocalAssignment:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter405 in self.executors:
-        iter405.write(oprot)
+      for iter423 in self.executors:
+        iter423.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5772,11 +5792,11 @@ class LSApprovedWorkers:
       if fid == 1:
         if ftype == TType.MAP:
           self.approved_workers = {}
-          (_ktype407, _vtype408, _size406 ) = iprot.readMapBegin()
-          for _i410 in xrange(_size406):
-            _key411 = iprot.readString().decode('utf-8')
-            _val412 = iprot.readI32();
-            self.approved_workers[_key411] = _val412
+          (_ktype425, _vtype426, _size424 ) = iprot.readMapBegin()
+          for _i428 in xrange(_size424):
+            _key429 = iprot.readString().decode('utf-8')
+            _val430 = iprot.readI32();
+            self.approved_workers[_key429] = _val430
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5793,9 +5813,9 @@ class LSApprovedWorkers:
     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 kiter413,viter414 in self.approved_workers.items():
-        oprot.writeString(kiter413.encode('utf-8'))
-        oprot.writeI32(viter414)
+      for kiter431,viter432 in self.approved_workers.items():
+        oprot.writeString(kiter431.encode('utf-8'))
+        oprot.writeI32(viter432)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5849,12 +5869,12 @@ class LSSupervisorAssignments:
       if fid == 1:
         if ftype == TType.MAP:
           self.assignments = {}
-          (_ktype416, _vtype417, _size415 ) = iprot.readMapBegin()
-          for _i419 in xrange(_size415):
-            _key420 = iprot.readI32();
-            _val421 = LocalAssignment()
-            _val421.read(iprot)
-            self.assignments[_key420] = _val421
+          (_ktype434, _vtype435, _size433 ) = iprot.readMapBegin()
+          for _i437 in xrange(_size433):
+            _key438 = iprot.readI32();
+            _val439 = LocalAssignment()
+            _val439.read(iprot)
+            self.assignments[_key438] = _val439
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5871,9 +5891,9 @@ class LSSupervisorAssignments:
     if self.assignments is not None:
       oprot.writeFieldBegin('assignments', TType.MAP, 1)
       oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments))
-      for kiter422,viter423 in self.assignments.items():
-        oprot.writeI32(kiter422)
-        viter423.write(oprot)
+      for kiter440,viter441 in self.assignments.items():
+        oprot.writeI32(kiter440)
+        viter441.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5946,11 +5966,11 @@ class LSWorkerHeartbeat:
       elif fid == 3:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype427, _size424) = iprot.readListBegin()
-          for _i428 in xrange(_size424):
-            _elem429 = ExecutorInfo()
-            _elem429.read(iprot)
-            self.executors.append(_elem429)
+          (_etype445, _size442) = iprot.readListBegin()
+          for _i446 in xrange(_size442):
+            _elem447 = ExecutorInfo()
+            _elem447.read(iprot)
+            self.executors.append(_elem447)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5980,8 +6000,8 @@ class LSWorkerHeartbeat:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter430 in self.executors:
-        iter430.write(oprot)
+      for iter448 in self.executors:
+        iter448.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.port is not None:

http://git-wip-us.apache.org/repos/asf/storm/blob/306ec57d/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index e4436be..c4417d4 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -218,7 +218,7 @@ struct TopologyInfo {
   4: required list<ExecutorSummary> executors;
   5: required string status;
   6: required map<string, list<ErrorInfo>> errors;
-  7: required bool debug;
+  7: required map<string, bool> component_debug;
 513: optional string sched_status;
 514: optional string owner;
 }
@@ -290,7 +290,7 @@ struct StormBase {
     6: optional string owner;
     7: optional TopologyActionOptions topology_action_options;
     8: optional TopologyStatus prev_status;//currently only used during rebalance action.
-    9: optional bool debug;
+    9: optional map<string, bool> component_debug; // topology/component level debug flags.
 }
 
 struct ClusterWorkerHeartbeat {
@@ -351,7 +351,7 @@ service Nimbus {
   void activate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void deactivate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
-  void debug(1: string name, 2: bool enable) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  void debug(1: string name, 2: string component, 3: bool enable) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
 
   // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs


[15/24] storm git commit: Fixes

Posted by sr...@apache.org.
Fixes

1. Correctly handle null values in thriftify-debugoptions
2. replace samplingPct correctly in component page template.


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

Branch: refs/heads/master
Commit: e5265d8519158b41b8ee2490ecdd2eaf41d50c68
Parents: f9a4b01
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Wed Aug 26 15:02:03 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Wed Aug 26 15:02:03 2015 +0530

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/converter.clj | 4 ++--
 storm-core/src/ui/public/component.html         | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e5265d85/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index b3d0cc6..253a406 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -153,8 +153,8 @@
 
 (defn thriftify-debugoptions [options]
   (doto (DebugOptions.)
-    (.set_enable (if options (:enable options) false))
-    (.set_samplingpct (if options (:samplingpct options) 0))))
+    (.set_enable (get options :enable false))
+    (.set_samplingpct (get options :samplingpct 100))))
 
 (defn thriftify-storm-base [storm-base]
   (doto (StormBase.)

http://git-wip-us.apache.org/repos/asf/storm/blob/e5265d85/storm-core/src/ui/public/component.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html
index 5749a57..4be5860 100644
--- a/storm-core/src/ui/public/component.html
+++ b/storm-core/src/ui/public/component.html
@@ -117,7 +117,7 @@ $(document).ready(function() {
         var componentSummary = $("#component-summary");
         var componentActions = $("#component-actions");
         var buttonJsonData = componentActionJson(response["encodedTopologyId"], response["encodedId"], response["id"],
-                                                 response["topologyStatus"], response["debug"]);
+                                                 response["topologyStatus"], response["debug"], response["samplingPct"]);
         var componentStatsDetail = $("#component-stats-detail")
         var inputStats = $("#component-input-stats");
         var outputStats = $("#component-output-stats");


[22/24] storm git commit: fixed unit test failure

Posted by sr...@apache.org.
fixed unit test failure


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

Branch: refs/heads/master
Commit: 0e680b36decffaacf5683e5564b8542cbbfec5d4
Parents: dc6d559
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Sat Sep 19 23:32:47 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Sat Sep 19 23:32:47 2015 +0530

----------------------------------------------------------------------
 storm-core/test/clj/backtype/storm/nimbus_test.clj | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0e680b36/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index 5956128..c1dca18 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -1329,7 +1329,7 @@
                      {"spout" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
                      {})]
         (submit-local-topology nimbus "t1" {TOPOLOGY-WORKERS 1} topology)
-        (.debug nimbus "t1" "spout" true))))
+        (.debug nimbus "t1" "spout" true 100))))
 
 (deftest test-debug-on-global
   (with-local-cluster [cluster]
@@ -1338,4 +1338,4 @@
                      {"spout" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
                      {})]
       (submit-local-topology nimbus "t1" {TOPOLOGY-WORKERS 1} topology)
-      (.debug nimbus "t1" "" true))))
+      (.debug nimbus "t1" "" true 100))))


[09/24] storm git commit: Merging upstream/master

Posted by sr...@apache.org.
Merging upstream/master


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

Branch: refs/heads/master
Commit: 7940d0c57c07c6b168617a0daaf182895cec8028
Parents: 306ec57 544e55c
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Mon Aug 10 15:08:10 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 10 15:08:10 2015 +0530

----------------------------------------------------------------------
 CHANGELOG.md                                    |  16 ++-
 DEVELOPER.md                                    |   7 +-
 README.markdown                                 |   1 +
 external/storm-elasticsearch/README.md          |  72 ++++++++++
 external/storm-elasticsearch/pom.xml            |  95 +++++++++++++
 .../elasticsearch/bolt/AbstractEsBolt.java      |  81 +++++++++++
 .../storm/elasticsearch/bolt/EsIndexBolt.java   |  68 ++++++++++
 .../elasticsearch/bolt/EsPercolateBolt.java     |  78 +++++++++++
 .../storm/elasticsearch/common/EsConfig.java    |  54 ++++++++
 .../storm/elasticsearch/trident/EsState.java    | 117 ++++++++++++++++
 .../elasticsearch/trident/EsStateFactory.java   |  50 +++++++
 .../storm/elasticsearch/trident/EsUpdater.java  |  31 +++++
 .../elasticsearch/bolt/AbstractEsBoltTest.java  |  81 +++++++++++
 .../elasticsearch/bolt/EsIndexBoltTest.java     |  68 ++++++++++
 .../elasticsearch/bolt/EsIndexTopology.java     | 120 +++++++++++++++++
 .../elasticsearch/bolt/EsPercolateBoltTest.java |  65 +++++++++
 .../storm/elasticsearch/common/EsConstants.java |  22 +++
 .../storm/elasticsearch/common/EsTestUtil.java  |  70 ++++++++++
 .../trident/TridentEsTopology.java              | 135 +++++++++++++++++++
 external/storm-hive/README.md                   |   1 +
 .../org/apache/storm/hive/bolt/HiveBolt.java    |  44 ++++--
 .../apache/storm/hive/common/HiveWriter.java    |   5 +-
 .../apache/storm/hive/bolt/TestHiveBolt.java    | 100 ++++++++++++--
 .../src/jvm/storm/kafka/ZkCoordinator.java      |   2 +-
 external/storm-redis/README.md                  |   1 +
 log4j2/cluster.xml                              |   6 +-
 log4j2/worker.xml                               |   6 +-
 pom.xml                                         |   1 +
 storm-core/src/clj/backtype/storm/cluster.clj   |   2 +-
 .../src/clj/backtype/storm/daemon/task.clj      |   1 -
 storm-core/src/clj/backtype/storm/tuple.clj     |   6 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj |  26 ++--
 storm-core/src/jvm/backtype/storm/Config.java   |   2 +-
 .../jvm/backtype/storm/ConfigValidation.java    |  15 +--
 .../src/jvm/backtype/storm/drpc/DRPCSpout.java  |   6 +-
 .../storm/grouping/PartialKeyGrouping.java      |  27 +++-
 .../storm/testing/TestWordBytesCounter.java     |  27 ++++
 .../backtype/storm/testing/TestWordCounter.java |   6 +-
 .../src/jvm/storm/trident/TridentTopology.java  |  13 +-
 .../test/clj/backtype/storm/config_test.clj     |  19 ++-
 .../test/clj/backtype/storm/grouping_test.clj   |  43 +++---
 .../test/clj/backtype/storm/nimbus_test.clj     |  30 +++++
 storm-dist/binary/src/main/assembly/binary.xml  |  14 ++
 43 files changed, 1532 insertions(+), 102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7940d0c5/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/7940d0c5/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/7940d0c5/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/nimbus_test.clj
index a8a3acd,00fb6d6..10d135c
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@@ -1182,10 -1181,31 +1182,40 @@@
            (submit-local-topology-with-opts nimbus "test" bad-config topology
                                             (SubmitOptions.))))))))
  
+ (deftest test-stateless-with-scheduled-topology-to-be-killed
+   ; tests regression of STORM-856
+   (with-inprocess-zookeeper zk-port
+     (with-local-tmp [nimbus-dir]
+       (letlocals
+         (bind conf (merge (read-storm-config)
+                      {STORM-ZOOKEEPER-SERVERS ["localhost"]
+                       STORM-CLUSTER-MODE "local"
+                       STORM-ZOOKEEPER-PORT zk-port
+                       STORM-LOCAL-DIR nimbus-dir}))
+         (bind cluster-state (cluster/mk-storm-cluster-state conf))
+         (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
+         (bind topology (thrift/mk-topology
+                          {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                          {}))
+         (submit-local-topology nimbus "t1" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 30} topology)
+         ; make transition for topology t1 to be killed -> nimbus applies this event to cluster state
+         (.killTopology nimbus "t1")
+         ; shutdown nimbus immediately to achieve nimbus doesn't handle event right now
+         (.shutdown nimbus)
+ 
+         ; in startup of nimbus it reads cluster state and take proper actions
+         ; in this case nimbus registers topology transition event to scheduler again
+         ; before applying STORM-856 nimbus was killed with NPE
+         (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
+         (.shutdown nimbus)
+         (.disconnect cluster-state)
 -        ))))
++        ))))
++
 +(deftest test-debug-on
 +  (with-local-cluster [cluster]
 +    (let [nimbus (:nimbus cluster)]
 +      (stubbing [converter/clojurify-storm-base (backtype.storm.daemon.common.StormBase. "storm-name" 100
 +                                                  {:type :active} 1 nil nil nil nil nil)
 +                 cluster/maybe-deserialize nil]
 +        (.debug nimbus "test" true)))))
++


[21/24] storm git commit: merging from upstream/master

Posted by sr...@apache.org.
merging from upstream/master


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

Branch: refs/heads/master
Commit: dc6d559b4957898ea96cca3df2910faea0e296a1
Parents: ea868a4 2bb41a9
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Sat Sep 19 16:35:37 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Sat Sep 19 16:35:37 2015 +0530

----------------------------------------------------------------------
 CHANGELOG.md                                    |  17 +-
 DEVELOPER.md                                    |  21 +++
 README.markdown                                 |   4 +
 conf/defaults.yaml                              |   5 +
 docs/about/multi-language.md                    |   2 +-
 .../documentation/Setting-up-a-Storm-cluster.md |  10 +-
 docs/documentation/Tutorial.md                  |   2 +-
 .../trident/TridentEsTopology.java              |   2 +-
 .../trident/OpaqueTridentEventHubSpout.java     |   2 +-
 .../TransactionalTridentEventHubSpout.java      |   2 +-
 .../org/apache/storm/hbase/common/Utils.java    |  10 +-
 .../storm/hdfs/trident/FixedBatchSpout.java     |   2 +-
 .../storm/hive/trident/TridentHiveTopology.java |   2 +-
 external/storm-kafka/README.md                  |  46 +++---
 storm-core/pom.xml                              |   1 +
 storm-core/src/clj/backtype/storm/cluster.clj   |  46 +++++-
 storm-core/src/clj/backtype/storm/config.clj    |  13 +-
 .../backtype/storm/daemon/builtin_metrics.clj   |  25 ++-
 .../src/clj/backtype/storm/daemon/executor.clj  | 154 ++++++++++++-------
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   2 +
 .../clj/backtype/storm/daemon/supervisor.clj    |   3 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |  78 +++++++++-
 storm-core/src/clj/backtype/storm/disruptor.clj |  19 ++-
 storm-core/src/clj/backtype/storm/util.clj      |   4 +
 storm-core/src/jvm/backtype/storm/Config.java   |  23 +++
 .../coordination/BatchSubtopologyBuilder.java   |   4 +-
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |   7 +-
 .../backtype/storm/messaging/netty/Client.java  |  37 +++--
 .../topology/BaseConfigurationDeclarer.java     |   2 +-
 .../ComponentConfigurationDeclarer.java         |   2 +-
 .../storm/topology/TopologyBuilder.java         |   2 +-
 .../TransactionalTopologyBuilder.java           |  10 +-
 .../src/jvm/backtype/storm/tuple/Fields.java    |  21 ++-
 .../src/jvm/backtype/storm/tuple/ITuple.java    | 126 ++++++++++++---
 .../utils/DisruptorBackpressureCallback.java    |  27 ++++
 .../backtype/storm/utils/DisruptorQueue.java    | 101 +++++++++++-
 .../jvm/backtype/storm/utils/RateTracker.java   | 119 ++++++++++++++
 .../storm/utils/WorkerBackpressureCallback.java |  26 ++++
 .../storm/utils/WorkerBackpressureThread.java   |  59 +++++++
 .../storm/trident/spout/BatchSpoutExecutor.java |   2 +-
 .../jvm/storm/trident/spout/IBatchSpout.java    |   2 +-
 .../spout/IOpaquePartitionedTridentSpout.java   |   2 +-
 .../trident/spout/IPartitionedTridentSpout.java |   2 +-
 .../jvm/storm/trident/spout/ITridentSpout.java  |   2 +-
 .../trident/spout/RichSpoutBatchExecutor.java   |   2 +-
 .../storm/trident/testing/FeederBatchSpout.java |   2 +-
 .../testing/FeederCommitterBatchSpout.java      |   2 +-
 .../storm/trident/testing/FixedBatchSpout.java  |   2 +-
 .../topology/TridentTopologyBuilder.java        |  18 +--
 .../test/clj/backtype/storm/config_test.clj     |  28 +++-
 .../test/clj/backtype/storm/supervisor_test.clj |   2 +
 .../utils/DisruptorQueueBackpressureTest.java   | 115 ++++++++++++++
 .../backtype/storm/utils/RateTrackerTest.java   |  62 ++++++++
 53 files changed, 1091 insertions(+), 190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d559b/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d559b/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d559b/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/executor.clj
index 829a0b4,0683f38..d7a68be
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@@ -504,18 -532,8 +548,9 @@@
          receive-queue (:receive-queue executor-data)
          event-handler (mk-task-receiver executor-data tuple-action-fn)
          has-ackers? (has-ackers? storm-conf)
 +        has-eventloggers? (has-eventloggers? storm-conf)
          emitted-count (MutableLong. 0)
-         empty-emit-streak (MutableLong. 0)
-         
-         ;; the overflow buffer is used to ensure that spouts never block when emitting
-         ;; this ensures that the spout can always clear the incoming buffer (acks and fails), which
-         ;; prevents deadlock from occuring across the topology (e.g. Spout -> Bolt -> Acker -> Spout, and all
-         ;; buffers filled up)
-         ;; when the overflow buffer is full, spouts stop calling nextTuple until it's able to clear the overflow buffer
-         ;; this limits the size of the overflow buffer to however many tuples a spout emits in one call of nextTuple, 
-         ;; preventing memory issues
-         overflow-buffer (ConcurrentLinkedQueue.)]
+         empty-emit-streak (MutableLong. 0)]
     
      [(async-loop
        (fn []
@@@ -710,17 -750,7 +769,8 @@@
                                      (stats/bolt-execute-tuple! executor-stats
                                                                 (.getSourceComponent tuple)
                                                                 (.getSourceStreamId tuple)
 -                                                               delta)))))))]
 +                                                               delta)))))))
- 
-         ;; the overflow buffer is used to ensure that bolts do not block when emitting
-         ;; this ensures that the bolt can always clear the incoming messages, which
-         ;; prevents deadlock from occurs across the topology
-         ;; (e.g. Spout -> BoltA -> Splitter -> BoltB -> BoltA, and all
-         ;; buffers filled up)
-         ;; the overflow buffer is might gradually fill degrading the performance gradually
-         ;; eventually running out of memory, but at least prevent live-locks/deadlocks.
-         overflow-buffer (if (storm-conf TOPOLOGY-BOLTS-OUTGOING-OVERFLOW-BUFFER-ENABLE) (ConcurrentLinkedQueue.) nil)
 +        has-eventloggers? (has-eventloggers? storm-conf)]
      
      ;; TODO: can get any SubscribedState objects out of the context now
  

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

http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d559b/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/worker.clj
index b542348,f795daa..781a959
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@@ -137,11 -170,11 +170,11 @@@
                          (.add remote (TaskMessage. task (.serialize serializer tuple)))
                          (log-warn "Can't transfer tuple - task value is nil. tuple type: " (pr-str (type tuple)) " and information: " (pr-str tuple)))
                       ))))
-                 (local-transfer local)
-                 (disruptor/publish transfer-queue remoteMap)
-               ))]
+ 
+               (local-transfer local)
+               (disruptor/publish transfer-queue remoteMap)))]
      (if try-serialize-local
 -      (do 
 +      (do
          (log-warn "WILL TRY TO SERIALIZE ALL TUPLES (Turn off " TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE " for production)")
          (fn [^KryoTupleSerializer serializer tuple-batch]
            (assert-can-serialize serializer tuple-batch)

http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d559b/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/dc6d559b/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------


[05/24] storm git commit: Debug options support for components

Posted by sr...@apache.org.
Debug options support for components


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

Branch: refs/heads/master
Commit: d1e5ec98732e77409e6303b1ec3708a44e6a7bb9
Parents: f8c4c5b
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Fri Aug 7 14:28:14 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 10 14:22:00 2015 +0530

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/ui/core.clj   | 15 ++++++++-
 storm-core/src/ui/public/component.html         |  7 +++++
 storm-core/src/ui/public/js/script.js           | 32 ++++++++++++++++++++
 .../templates/component-page-template.html      |  7 +++++
 4 files changed, 60 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d1e5ec98/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index ea7b048..1b5e5b4 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -947,6 +947,7 @@
          "window" window
          "componentType" (name type)
          "windowHint" (window-hint window)
+         "debug" (.is_debug summ)
          "eventLogLink" (event-log-link topology-id summ topology component secure?)}
        spec errors))))
 
@@ -1040,7 +1041,19 @@
             name (.get_name tplg)
             enable? (= "enable" action)]
         (.debug nimbus name enable?)
-        (log-message "Debug topology '" name "' [" action "]")))
+        (log-message "Debug topology [" name "] action [" action "]")))
+    (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
+  (POST "/api/v1/topology/:id/component/:component/debug/:action" [:as {:keys [cookies servlet-request]} id component action & m]
+    (assert-authorized-user servlet-request "debug" (topology-config id))
+    (with-nimbus nimbus
+      (let [tplg (->> (doto
+                        (GetInfoOptions.)
+                        (.set_num_err_choice NumErrorsChoice/NONE))
+                   (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
+            name (.get_name tplg)
+            enable? (= "enable" action)]
+        (.debug nimbus name enable?) ;; TODO: include component id in the nimbus api
+        (log-message "Debug topology [" name "] component [" component "] action [" action "]")))
     (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
     (assert-authorized-user servlet-request "rebalance" (topology-config id))

http://git-wip-us.apache.org/repos/asf/storm/blob/d1e5ec98/storm-core/src/ui/public/component.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html
index a42c1a7..3b4b10d 100644
--- a/storm-core/src/ui/public/component.html
+++ b/storm-core/src/ui/public/component.html
@@ -46,6 +46,9 @@
     <div id="component-summary" class="col-md-12"></div>
   </div>
   <div class="row">
+    <div id="component-actions" class="col-md-12"></div>
+  </div>
+  <div class="row">
     <div id="component-stats-detail" class="col-md-12"></div>
   </div>
   <div class="row">
@@ -112,6 +115,8 @@ $(document).ready(function() {
         });
 
         var componentSummary = $("#component-summary");
+        var componentActions = $("#component-actions");
+        var buttonJsonData = componentActionJson(response["encodedTopologyId"], response["encodedId"], response["id"], response["debug"]);
         var componentStatsDetail = $("#component-stats-detail")
         var inputStats = $("#component-input-stats");
         var outputStats = $("#component-output-stats");
@@ -119,6 +124,7 @@ $(document).ready(function() {
         var componentErrors = $("#component-errors");
         $.get("/templates/component-page-template.html", function(template) {
             componentSummary.append(Mustache.render($(template).filter("#component-summary-template").html(),response));
+            componentActions.append(Mustache.render($(template).filter("#component-actions-template").html(),buttonJsonData));
             if(response["componentType"] == "spout") {
                 componentStatsDetail.append(Mustache.render($(template).filter("#spout-stats-detail-template").html(),response));
                 //window, emitted, transferred, complete latency, acked, failed
@@ -197,6 +203,7 @@ $(document).ready(function() {
               }
             }
             $('#component-summary [data-toggle="tooltip"]').tooltip();
+            $('#component-actions [data-toggle="tooltip"]').tooltip();
             $('#component-stats-detail [data-toggle="tooltip"]').tooltip();
             $('#component-input-stats [data-toggle="tooltip"]').tooltip();
             $('#component-output-stats [data-toggle="tooltip"]').tooltip();

http://git-wip-us.apache.org/repos/asf/storm/blob/d1e5ec98/storm-core/src/ui/public/js/script.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js
index fda17f2..2173b43 100644
--- a/storm-core/src/ui/public/js/script.js
+++ b/storm-core/src/ui/public/js/script.js
@@ -81,6 +81,28 @@ function ensureInt(n) {
     return isInt;
 }
 
+function confirmComponentAction(topologyId, componentId, componentName, action, actionText) {
+    var opts = {
+        type:'POST',
+        url:'/api/v1/topology/' + topologyId + '/component/' + componentId + '/' + action
+    };
+    if (actionText === undefined) {
+        actionText = action;
+    }
+    if (!confirm('Do you really want to ' + actionText + ' component "' + componentName + '"?')) {
+        return false;
+    }
+
+    $("input[type=button]").attr("disabled", "disabled");
+    $.ajax(opts).always(function () {
+        window.location.reload();
+    }).fail(function () {
+        alert("Error while communicating with Nimbus.");
+    });
+
+    return false;
+}
+
 function confirmAction(id, name, action, wait, defaultWait, actionText) {
     var opts = {
         type:'POST',
@@ -164,6 +186,16 @@ function topologyActionJson(id, encodedId, name, status, msgTimeout, debug) {
     return jsonData;
 }
 
+function componentActionJson(encodedTopologyId, encodedId, componentName, debug) {
+    var jsonData = {};
+    jsonData["encodedTopologyId"] = encodedTopologyId;
+    jsonData["encodedId"] = encodedId;
+    jsonData["componentName"] = componentName;
+    jsonData["startDebugStatus"] = (!debug) ? "enabled" : "disabled";
+    jsonData["stopDebugStatus"] = debug ? "enabled" : "disabled";
+    return jsonData;
+}
+
 function topologyActionButton(id,name,status,actionLabel,command,wait,defaultWait) {
     var buttonData = {};
     buttonData["buttonStatus"] = status ;

http://git-wip-us.apache.org/repos/asf/storm/blob/d1e5ec98/storm-core/src/ui/public/templates/component-page-template.html
----------------------------------------------------------------------
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 7bf9026..01b19ce 100644
--- a/storm-core/src/ui/public/templates/component-page-template.html
+++ b/storm-core/src/ui/public/templates/component-page-template.html
@@ -492,3 +492,10 @@
     </tbody>
   </table>
 </script>
+<script id="component-actions-template" type="text/html">
+  <h2>Component actions</h2>
+  <p id="component-actions">
+    <input {{startDebugStatus}} onclick="confirmComponentAction('{{encodedTopologyId}}', '{{encodedId}}', '{{componentName}}', 'debug/enable', 'debug')" type="button" value="Debug" class="btn btn-default">
+    <input {{stopDebugStatus}} onclick="confirmComponentAction('{{encodedTopologyId}}', '{{encodedId}}', '{{componentName}}', 'debug/disable', 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
+  </p>
+</script>


[19/24] storm git commit: fixing a few issues post merge

Posted by sr...@apache.org.
fixing a few issues post merge


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

Branch: refs/heads/master
Commit: cec7ff9be618951ed91f3b8183c52e6bc6a6bd28
Parents: 0151b44
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Thu Sep 3 13:31:53 2015 -0700
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Thu Sep 3 13:31:53 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/ui/core.clj | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/cec7ff9b/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index 7b44619..f8b0390 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -758,7 +758,7 @@
        "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) 100)}))
+       "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 100)
        "replicationCount" (.get_replication_count summ)}))
 
 (defn spout-summary-json [topology-id id stats window]
@@ -1083,7 +1083,7 @@
     (json-response (topology-op-response id "deactivate") (m "callback")))
   (POST "/api/v1/topology/:id/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id action spct & m]
     (assert-authorized-user servlet-request "debug" (topology-config id))
-    (with-nimbus nimbus
+    (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (->> (doto
                         (GetInfoOptions.)
                         (.set_num_err_choice NumErrorsChoice/NONE))
@@ -1095,7 +1095,7 @@
     (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/component/:component/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id component action spct & m]
     (assert-authorized-user servlet-request "debug" (topology-config id))
-    (with-nimbus nimbus
+    (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (->> (doto
                         (GetInfoOptions.)
                         (.set_num_err_choice NumErrorsChoice/NONE))


[12/24] storm git commit: Removed component debug flags from TopologyInfo ctor since its now optional

Posted by sr...@apache.org.
Removed component debug flags from TopologyInfo ctor since its now optional


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

Branch: refs/heads/master
Commit: 788db9d9e9dd3913e82732ca631223e0b2d39927
Parents: 1764b86
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Mon Aug 24 13:46:47 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 24 13:46:47 2015 +0530

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/daemon/nimbus.clj | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/788db9d9/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 9fe3928..67ad239 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1353,10 +1353,10 @@
                            executor-summaries
                            (extract-status-str base)
                            errors
-                           (map-val boolean (:component->debug base))
                            )]
             (when-let [owner (:owner base)] (.set_owner topo-info owner))
             (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))
+            (when-let [component->debug (:component->debug base)] (.set_component_debug topo-info (map-val boolean component->debug)))
             topo-info
           ))
 


[11/24] storm git commit: Addressed PR feedbacks

Posted by sr...@apache.org.
Addressed PR feedbacks


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

Branch: refs/heads/master
Commit: 1764b867b80b0e228e2f2b867d761c1e91155673
Parents: 35990fa
Author: Arun Mahadevan <ai...@hortonworks.com>
Authored: Mon Aug 24 13:24:46 2015 +0530
Committer: Arun Mahadevan <ai...@hortonworks.com>
Committed: Mon Aug 24 13:24:46 2015 +0530

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   |   3 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   3 +-
 .../jvm/backtype/storm/generated/Nimbus.java    |  11 +-
 .../backtype/storm/generated/TopologyInfo.java  | 105 ++++++++++---------
 storm-core/src/py/storm/Nimbus.py               |   8 ++
 storm-core/src/py/storm/ttypes.py               |   2 -
 storm-core/src/storm.thrift                     |   6 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |  10 +-
 8 files changed, 90 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index dcb4fd5..d96cb98 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -240,8 +240,7 @@
     (when cb
       (cb id))))
 
-;; public for stubbing in nimbus_test
-(defn maybe-deserialize
+(defn- maybe-deserialize
   [ser clazz]
   (when ser
     (Utils/deserialize ser clazz)))

http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index c568e9e..9fe3928 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1169,7 +1169,8 @@
           (check-authorization! nimbus storm-name topology-conf "debug")
           (when-not storm-id
             (throw (NotAliveException. storm-name)))
-          (log-message "Nimbus setting debug to " enable? " for storm-name " storm-name " storm-id " storm-id)
+          (log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "'"
+            (if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
           (locking (:submit-lock nimbus)
             (.update-storm! storm-cluster-state storm-id storm-base-updates))))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index 1c2b2f0..830d95e 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.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
 public class Nimbus {
 
   public interface Iface {
@@ -70,6 +70,15 @@ public class Nimbus {
 
     public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
+    /**
+     * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
+     * and if null or empty, the debug flag will apply to the entire topology.
+     * 
+     * 
+     * @param name
+     * @param component
+     * @param enable
+     */
     public void debug(String name, String component, boolean enable) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;

http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index e050f24..d9b7c8d 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-7")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
 public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
 
@@ -77,7 +77,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private List<ExecutorSummary> executors; // required
   private String status; // required
   private Map<String,List<ErrorInfo>> errors; // required
-  private Map<String,Boolean> component_debug; // required
+  private Map<String,Boolean> component_debug; // optional
   private String sched_status; // optional
   private String owner; // optional
 
@@ -166,7 +166,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   // isset id assignments
   private static final int __UPTIME_SECS_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER};
+  private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER};
   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);
@@ -186,7 +186,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             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.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+    tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", 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.BOOL))));
@@ -207,8 +207,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     int uptime_secs,
     List<ExecutorSummary> executors,
     String status,
-    Map<String,List<ErrorInfo>> errors,
-    Map<String,Boolean> component_debug)
+    Map<String,List<ErrorInfo>> errors)
   {
     this();
     this.id = id;
@@ -218,7 +217,6 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     this.executors = executors;
     this.status = status;
     this.errors = errors;
-    this.component_debug = component_debug;
   }
 
   /**
@@ -983,14 +981,16 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       sb.append(this.errors);
     }
     first = false;
-    if (!first) sb.append(", ");
-    sb.append("component_debug:");
-    if (this.component_debug == null) {
-      sb.append("null");
-    } else {
-      sb.append(this.component_debug);
+    if (is_set_component_debug()) {
+      if (!first) sb.append(", ");
+      sb.append("component_debug:");
+      if (this.component_debug == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_debug);
+      }
+      first = false;
     }
-    first = false;
     if (is_set_sched_status()) {
       if (!first) sb.append(", ");
       sb.append("sched_status:");
@@ -1041,10 +1041,6 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'errors' is unset! Struct:" + toString());
     }
 
-    if (!is_set_component_debug()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_debug' is unset! Struct:" + toString());
-    }
-
     // check for sub-struct validity
   }
 
@@ -1266,17 +1262,19 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         oprot.writeFieldEnd();
       }
       if (struct.component_debug != null) {
-        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.BOOL, struct.component_debug.size()));
-          for (Map.Entry<String, Boolean> _iter307 : struct.component_debug.entrySet())
+        if (struct.is_set_component_debug()) {
+          oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC);
           {
-            oprot.writeString(_iter307.getKey());
-            oprot.writeBool(_iter307.getValue());
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BOOL, struct.component_debug.size()));
+            for (Map.Entry<String, Boolean> _iter307 : struct.component_debug.entrySet())
+            {
+              oprot.writeString(_iter307.getKey());
+              oprot.writeBool(_iter307.getValue());
+            }
+            oprot.writeMapEnd();
           }
-          oprot.writeMapEnd();
+          oprot.writeFieldEnd();
         }
-        oprot.writeFieldEnd();
       }
       if (struct.sched_status != null) {
         if (struct.is_set_sched_status()) {
@@ -1334,22 +1332,27 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           }
         }
       }
-      {
-        oprot.writeI32(struct.component_debug.size());
-        for (Map.Entry<String, Boolean> _iter311 : struct.component_debug.entrySet())
-        {
-          oprot.writeString(_iter311.getKey());
-          oprot.writeBool(_iter311.getValue());
-        }
-      }
       BitSet optionals = new BitSet();
-      if (struct.is_set_sched_status()) {
+      if (struct.is_set_component_debug()) {
         optionals.set(0);
       }
-      if (struct.is_set_owner()) {
+      if (struct.is_set_sched_status()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_owner()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_component_debug()) {
+        {
+          oprot.writeI32(struct.component_debug.size());
+          for (Map.Entry<String, Boolean> _iter311 : struct.component_debug.entrySet())
+          {
+            oprot.writeString(_iter311.getKey());
+            oprot.writeBool(_iter311.getValue());
+          }
+        }
+      }
       if (struct.is_set_sched_status()) {
         oprot.writeString(struct.sched_status);
       }
@@ -1404,25 +1407,27 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         }
       }
       struct.set_errors_isSet(true);
-      {
-        org.apache.thrift.protocol.TMap _map322 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BOOL, iprot.readI32());
-        struct.component_debug = new HashMap<String,Boolean>(2*_map322.size);
-        String _key323;
-        boolean _val324;
-        for (int _i325 = 0; _i325 < _map322.size; ++_i325)
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
         {
-          _key323 = iprot.readString();
-          _val324 = iprot.readBool();
-          struct.component_debug.put(_key323, _val324);
+          org.apache.thrift.protocol.TMap _map322 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.BOOL, iprot.readI32());
+          struct.component_debug = new HashMap<String,Boolean>(2*_map322.size);
+          String _key323;
+          boolean _val324;
+          for (int _i325 = 0; _i325 < _map322.size; ++_i325)
+          {
+            _key323 = iprot.readString();
+            _val324 = iprot.readBool();
+            struct.component_debug.put(_key323, _val324);
+          }
         }
+        struct.set_component_debug_isSet(true);
       }
-      struct.set_component_debug_isSet(true);
-      BitSet incoming = iprot.readBitSet(2);
-      if (incoming.get(0)) {
+      if (incoming.get(1)) {
         struct.sched_status = iprot.readString();
         struct.set_sched_status_isSet(true);
       }
-      if (incoming.get(1)) {
+      if (incoming.get(2)) {
         struct.owner = iprot.readString();
         struct.set_owner_isSet(true);
       }

http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/src/py/storm/Nimbus.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index 2aff10d..8bbe8d0 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -96,6 +96,10 @@ class Iface:
 
   def debug(self, name, component, enable):
     """
+    Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
+    and if null or empty, the debug flag will apply to the entire topology.
+
+
     Parameters:
      - name
      - component
@@ -454,6 +458,10 @@ class Client(Iface):
 
   def debug(self, name, component, enable):
     """
+    Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
+    and if null or empty, the debug flag will apply to the entire topology.
+
+
     Parameters:
      - name
      - component

http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index fe9663e..2784dad 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -4254,8 +4254,6 @@ class TopologyInfo:
       raise TProtocol.TProtocolException(message='Required field status is unset!')
     if self.errors is None:
       raise TProtocol.TProtocolException(message='Required field errors is unset!')
-    if self.component_debug is None:
-      raise TProtocol.TProtocolException(message='Required field component_debug is unset!')
     return
 
 

http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index c4417d4..8547b30 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -218,7 +218,7 @@ struct TopologyInfo {
   4: required list<ExecutorSummary> executors;
   5: required string status;
   6: required map<string, list<ErrorInfo>> errors;
-  7: required map<string, bool> component_debug;
+  7: optional map<string, bool> component_debug;
 513: optional string sched_status;
 514: optional string owner;
 }
@@ -351,6 +351,10 @@ service Nimbus {
   void activate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void deactivate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
+  /**
+  * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
+  * and if null or empty, the debug flag will apply to the entire topology.
+  **/
   void debug(1: string name, 2: string component, 3: bool enable) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
 

http://git-wip-us.apache.org/repos/asf/storm/blob/1764b867/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index 1f7258b..3d0978c 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -1224,7 +1224,7 @@
         (.disconnect cluster-state)
         ))))
 
-(deftest test-debug-on
+(deftest test-debug-on-component
   (with-local-cluster [cluster]
     (let [nimbus (:nimbus cluster)
           topology (thrift/mk-topology
@@ -1233,3 +1233,11 @@
         (submit-local-topology nimbus "t1" {TOPOLOGY-WORKERS 1} topology)
         (.debug nimbus "t1" "spout" true))))
 
+(deftest test-debug-on-global
+  (with-local-cluster [cluster]
+    (let [nimbus (:nimbus cluster)
+          topology (thrift/mk-topology
+                     {"spout" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                     {})]
+      (submit-local-topology nimbus "t1" {TOPOLOGY-WORKERS 1} topology)
+      (.debug nimbus "t1" "" true))))