You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2015/08/24 15:51:46 UTC

[01/50] [abbrv] storm git commit: STORM-166: some refactoring so host:port parsing happens in centralized location.

Repository: storm
Updated Branches:
  refs/heads/master cc276087b -> d4db41054


STORM-166: some refactoring so host:port parsing happens in centralized location.


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

Branch: refs/heads/master
Commit: d0bb3830a3e6fccbfb4d17ed170ed8f4e6056df0
Parents: 50fdcee
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 11:26:26 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 11:26:26 2014 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   |  7 ++++---
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 20 +++++++++-----------
 .../jvm/backtype/storm/nimbus/NimbusInfo.java   | 17 ++++++++++++++++-
 3 files changed, 29 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d0bb3830/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 572fc96..6078852 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -20,6 +20,7 @@
   (:import [backtype.storm.utils Utils])
   (:import [java.security MessageDigest])
   (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
+  (:import [backtype.storm.nimbus NimbusInfo])
   (:use [backtype.storm util log config])
   (:require [backtype.storm [zookeeper :as zk]])
   (:require [backtype.storm.daemon [common :as common]]))
@@ -328,7 +329,7 @@
 
       (code-distributor-info
         [this storm-id]
-        (get-children cluster-state (code-distributor-path storm-id) false))
+        (map (fn [nimbus-info] (NimbusInfo/parse nimbus-info)) (get-children cluster-state (code-distributor-path storm-id) false)))
 
       (active-storms
         [this]
@@ -431,9 +432,9 @@
         (set-data cluster-state (assignment-path storm-id) (Utils/serialize info) acls))
 
       (setup-code-distributor!
-        [this storm-id info]
+        [this storm-id nimbusInfo]
         (mkdirs cluster-state (code-distributor-path storm-id) acls)
-        (mkdirs cluster-state (str (code-distributor-path storm-id) "/" info) acls))
+        (mkdirs cluster-state (str (code-distributor-path storm-id) "/" (.toHostPortString nimbusInfo)) acls))
 
       (remove-storm!
         [this storm-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/d0bb3830/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 e991de3..c652b41 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -25,6 +25,7 @@
             Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
   (:use [backtype.storm bootstrap util zookeeper])
   (:import [backtype.storm.generated AuthorizationException])
+  (:import [backtype.storm.nimbus NimbusInfo])
   (:use [backtype.storm bootstrap util])
   (:use [backtype.storm.config :only [validate-configs-with-schemas]])
   (:use [backtype.storm.daemon common])
@@ -78,7 +79,7 @@
 (defn nimbus-data [conf inimbus]
   (let [forced-scheduler (.getForcedScheduler inimbus)]
     {:conf conf
-     :host-port-info (str (.getCanonicalHostName (InetAddress/getLocalHost)) ":" (conf NIMBUS-THRIFT-PORT))
+     :nimbus-host-port-info (NimbusInfo. (.getCanonicalHostName (InetAddress/getLocalHost)) (conf NIMBUS-THRIFT-PORT) false)
      :inimbus inimbus
      :authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
      :submitted-count (atom 0)
@@ -1107,8 +1108,7 @@
                                 (dissoc storm-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-SCHEME STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
                 total-storm-conf (merge conf storm-conf)
                 topology (normalize-topology total-storm-conf topology)
-                storm-cluster-state (:storm-cluster-state nimbus)
-                host-port-info (:host-port-info nimbus) ]
+                storm-cluster-state (:storm-cluster-state nimbus)]
             (when credentials (doseq [nimbus-autocred-plugin (:nimbus-autocred-plugins nimbus)]
               (.populateCredentials nimbus-autocred-plugin credentials (Collections/unmodifiableMap storm-conf))))
             (if (and (conf SUPERVISOR-RUN-WORKER-AS-USER) (or (nil? submitter-user) (.isEmpty (.trim submitter-user)))) 
@@ -1125,7 +1125,7 @@
               ;;cred-update-lock is not needed here because creds are being added for the first time.
               (.set-credentials! storm-cluster-state storm-id credentials storm-conf)
               (setup-storm-code nimbus conf storm-id uploadedJarLocation storm-conf topology)
-              (.setup-code-distributor! storm-cluster-state storm-id host-port-info)
+              (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus))
               (wait-for-desired-code-replication nimbus conf storm-id)
               (.setup-heartbeats! storm-cluster-state storm-id)
               (let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive
@@ -1380,7 +1380,6 @@
 (defn download-code [conf nimbus storm-id host port]
   (let [tmp-root (str (master-tmp-dir conf) file-path-separator (uuid))
         storm-cluster-state (:storm-cluster-state nimbus)
-        host-port-info (:host-port-info nimbus)
         storm-root (master-stormdist-root conf storm-id)
         remote-meta-file-path (master-storm-metafile-path storm-root)
         local-meta-file-path (master-storm-metafile-path tmp-root)]
@@ -1390,7 +1389,7 @@
       (.download (:bt-tracker nimbus) storm-id (File. local-meta-file-path)))
     (if (.exists (File. storm-root)) (FileUtils/forceDelete (File. storm-root)))
     (FileUtils/moveDirectory (File. tmp-root) (File. storm-root))
-    (.setup-code-distributor! storm-cluster-state storm-id host-port-info)))
+    (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus))))
 
 (defmethod sync-code :distributed [conf nimbus]
   (let [storm-cluster-state (:storm-cluster-state nimbus)
@@ -1405,11 +1404,10 @@
           (let [nimbuses-with-missing (.code-distributor-info storm-cluster-state missing)]
             (log-message "trying to download missing topology code from " (clojure.string/join "," nimbuses-with-missing))
             (doseq [nimbus-host-port nimbuses-with-missing]
-              (let [[host port] (clojure.string/split nimbus-host-port #":")]
-                (when-not (contains? (code-ids (:conf nimbus)) missing)
-                  (try
-                    (download-code conf nimbus missing host (Integer/parseInt port))
-                    (catch Exception e (log-error e "Exception while trying to syn-code for missing topology" missing))))))))))
+              (when-not (contains? (code-ids (:conf nimbus)) missing)
+                (try
+                  (download-code conf nimbus missing (.getHost nimbus-host-port) (.getPort nimbus-host-port))
+                  (catch Exception e (log-error e "Exception while trying to syn-code for missing topology" missing)))))))))
     (.addToLeaderLockQueue (:leader-elector nimbus))
     (log-message "local disk is completely in sync with zk code-distributor.")))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d0bb3830/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
index 1baa461..e31090f 100644
--- a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
+++ b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
@@ -3,9 +3,11 @@ package backtype.storm.nimbus;
 import java.io.Serializable;
 
 public class NimbusInfo implements Serializable {
+    private static final String DELIM = ":";
+
     private String host;
     private int port;
-    private transient boolean isLeader;
+    private boolean isLeader;
 
     public NimbusInfo(String host, int port, boolean isLeader) {
         this.host = host;
@@ -13,6 +15,19 @@ public class NimbusInfo implements Serializable {
         this.isLeader = isLeader;
     }
 
+    public static NimbusInfo parse(String nimbusInfo) {
+        String[] hostAndPort = nimbusInfo.split(DELIM);
+        if(hostAndPort != null && hostAndPort.length == 2) {
+            return new NimbusInfo(hostAndPort[0], Integer.parseInt(hostAndPort[1]), false);
+        } else {
+            throw new RuntimeException("nimbusInfo should have format of host:port, invalid string " + nimbusInfo);
+        }
+    }
+
+    public String toHostPortString() {
+        return String.format("%s%s%s",host,DELIM,port);
+    }
+
     public boolean isLeader() {
         return isLeader;
     }


[41/50] [abbrv] storm git commit: Changed the code-distributor entries to ephemeral nodes. Added code to ensure nimbus sets up the correct code-distributor entries on startup.

Posted by bo...@apache.org.
Changed the code-distributor entries to ephemeral nodes. Added code to ensure nimbus sets up the correct code-distributor entries on startup.


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

Branch: refs/heads/master
Commit: 3f66ffd73086681c43a7f16342cab5cd944f25f4
Parents: 3e20823
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Apr 8 16:52:12 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 09:28:27 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj       |  5 ++++-
 storm-core/src/clj/backtype/storm/daemon/nimbus.clj | 11 +++++++++++
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3f66ffd7/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 f75648a..15af1b0 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -483,8 +483,11 @@
 
       (setup-code-distributor!
         [this storm-id nimbusInfo]
+        (let [path (str (code-distributor-path storm-id) "/" (.toHostPortString nimbusInfo))]
         (mkdirs cluster-state (code-distributor-path storm-id) acls)
-        (mkdirs cluster-state (str (code-distributor-path storm-id) "/" (.toHostPortString nimbusInfo)) acls))
+        ;we delete the node first to ensure the node gets created as part of this session only.
+        (delete-node cluster-state path)
+        (set-ephemeral-node cluster-state path nil acls)))
 
       (remove-storm!
         [this storm-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/3f66ffd7/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 35154d3..5d9a038 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -957,6 +957,15 @@
         )))
   (log-message "not a leader, skipping cleanup-corrupt-topologies"))
 
+;;setsup code distributor entries for all current topologies for which code is available locally.
+(defn setup-code-distributor [nimbus]
+  (let [storm-cluster-state (:storm-cluster-state nimbus)
+        locally-available-storm-ids (set (code-ids (:conf nimbus)))
+        active-topologies (set (.active-storms storm-cluster-state))
+        locally-available-active-storm-ids (set/intersection locally-available-storm-ids active-topologies)]
+    (doseq [storm-id locally-available-active-storm-ids]
+      (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus)))))
+
 (defn- get-errors [storm-cluster-state storm-id component-id]
   (->> (.errors storm-cluster-state storm-id component-id)
        (map #(doto (ErrorInfo. (:error %) (:time-secs %))
@@ -1074,6 +1083,8 @@
 
     (.addToLeaderLockQueue (:leader-elector nimbus))
     (cleanup-corrupt-topologies! nimbus)
+    (setup-code-distributor nimbus)
+
     ;register call back for code-distributor
     (.code-distributor (:storm-cluster-state nimbus) (fn [] (sync-code conf nimbus)))
     (when (is-leader nimbus :throw-exception false)


[25/50] [abbrv] storm git commit: Removing the cache variable for now.

Posted by bo...@apache.org.
Removing the cache variable for now.


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

Branch: refs/heads/master
Commit: 0b454e8ebfebdf144f204169f6c75ae4f8fb0a63
Parents: 7cae523
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 15:00:30 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 15:00:30 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/storm/blob/0b454e8e/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 2059c0d..26bd591 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -105,7 +105,6 @@
      :id->sched-status (atom {})
      :cred-renewers (AuthUtils/GetCredentialRenewers conf)
      :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)
-     :nimbuses-cache (atom {}) ;;TODO need to figure out how to keep the cache upto date, one more thread
      }))
 
 (defn inbox [nimbus]


[07/50] [abbrv] storm git commit: STORM-166 removing no longer required test.

Posted by bo...@apache.org.
STORM-166 removing no longer required test.


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

Branch: refs/heads/master
Commit: 2954eae77c898174bdf6b5970d8095ed442e672d
Parents: c82482d
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 13:55:32 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 13:55:32 2014 -0800

----------------------------------------------------------------------
 storm-core/test/clj/backtype/storm/utils_test.clj | 11 -----------
 1 file changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/2954eae7/storm-core/test/clj/backtype/storm/utils_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/utils_test.clj b/storm-core/test/clj/backtype/storm/utils_test.clj
index 2b0bbd5..74d97b7 100644
--- a/storm-core/test/clj/backtype/storm/utils_test.clj
+++ b/storm-core/test/clj/backtype/storm/utils_test.clj
@@ -42,17 +42,6 @@
   )
 )
 
-(deftest test-getConfiguredClient-throws-RunTimeException-on-bad-config
-  (let [storm-conf (merge (read-storm-config)
-                     {STORM-THRIFT-TRANSPORT-PLUGIN
-                       "backtype.storm.security.auth.SimpleTransportPlugin"
-                      Config/NIMBUS_THRIFT_PORT ""
-                     })]
-    (is (thrown-cause? RuntimeException
-      (NimbusClient/getConfiguredClient storm-conf)))
-  )
-)
-
 (deftest test-getConfiguredClient-throws-RunTimeException-on-bad-args
   (let [storm-conf (merge
                     (read-storm-config)


[22/50] [abbrv] storm git commit: Adding nimbus summary info to zookeeper.

Posted by bo...@apache.org.
Adding nimbus summary info to zookeeper.


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

Branch: refs/heads/master
Commit: 4502bffbe3f9b4cd3674a56afbda1bb115cec239
Parents: 1b6491f
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 11:27:50 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 11:27:50 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   |  24 +-
 storm-core/src/clj/backtype/storm/config.clj    |  10 +
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  27 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  28 +-
 .../storm/generated/ClusterSummary.java         | 232 +++---
 .../backtype/storm/generated/NimbusSummary.java | 723 +++++++++++++++++++
 storm-core/src/py/storm/ttypes.py               | 577 +++++++++------
 storm-core/src/storm.thrift                     |  10 +-
 .../public/templates/index-page-template.html   |  26 +-
 9 files changed, 1297 insertions(+), 360 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/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 2c58510..3bf6628 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -146,6 +146,12 @@
   (code-distributor [this callback])
   ;returns lits of nimbusinfos under /stormroot/code-distributor/storm-id
   (code-distributor-info [this storm-id])
+
+  ;returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
+  (nimbuses [this])
+  ;adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
+  (add-nimbus-host! [this nimbus-id nimbus-summary])
+
   (active-storms [this])
   (storm-base [this storm-id callback])
   (get-worker-heartbeat [this storm-id node port])
@@ -180,14 +186,17 @@
 (def WORKERBEATS-ROOT "workerbeats")
 (def ERRORS-ROOT "errors")
 (def CODE-DISTRIBUTOR-ROOT "code-distributor")
+(def NIMBUSES-ROOT "nimbuses")
 (def CREDENTIALS-ROOT "credentials")
 
+
 (def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
 (def STORMS-SUBTREE (str "/" STORMS-ROOT))
 (def SUPERVISORS-SUBTREE (str "/" SUPERVISORS-ROOT))
 (def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
 (def ERRORS-SUBTREE (str "/" ERRORS-ROOT))
 (def CODE-DISTRIBUTOR-SUBTREE (str "/" CODE-DISTRIBUTOR-ROOT))
+(def NIMBUSES-SUBTREE (str "/" NIMBUSES-ROOT))
 (def CREDENTIALS-SUBTREE (str "/" CREDENTIALS-ROOT))
 
 (defn supervisor-path
@@ -202,6 +211,10 @@
   [id]
   (str CODE-DISTRIBUTOR-SUBTREE "/" id))
 
+(defn nimbus-path
+  [id]
+  (str NIMBUSES-SUBTREE "/" id))
+
 (defn storm-path
   [id]
   (str STORMS-SUBTREE "/" id))
@@ -292,7 +305,7 @@
                          CREDENTIALS-ROOT (issue-map-callback! credentials-callback (first args))
                          ;; this should never happen
                          (exit-process! 30 "Unknown callback for subtree " subtree args)))))]
-    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE CODE-DISTRIBUTOR-SUBTREE]]
+    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE CODE-DISTRIBUTOR-SUBTREE NIMBUSES-SUBTREE]]
       (mkdirs cluster-state p acls))
     (reify
       StormClusterState
@@ -330,6 +343,15 @@
           (reset! code-distributor-callback callback))
         (get-children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback)))
 
+      (nimbuses
+        [this]
+        (map #(maybe-deserialize (get-data cluster-state (nimbus-path %1) false))
+          (get-children cluster-state NIMBUSES-SUBTREE false)))
+
+      (add-nimbus-host!
+        [this nimbus-id nimbus-summary]
+        (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
+
       (code-distributor-info
         [this storm-id]
         (map (fn [nimbus-info] (NimbusInfo/parse nimbus-info)) (get-children cluster-state (code-distributor-path storm-id) false)))

http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index a6b160d..f3c70e5 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -282,3 +282,13 @@
 (defn ^LocalState worker-state
   [conf id]
   (LocalState. (worker-heartbeats-root conf id)))
+
+(defn read-storm-version
+  "Returns a string containing the Storm version or 'Unknown'."
+  []
+  (let [storm-home (System/getProperty "storm.home")
+        release-path (format "%s/RELEASE" storm-home)
+        release-file (File. release-path)]
+    (if (and (.exists release-file) (.isFile release-file))
+      (str/trim (slurp release-path))
+      "Unknown")))

http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/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 e354fab..52ee708 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -15,7 +15,8 @@
 ;; limitations under the License.
 (ns backtype.storm.daemon.nimbus
   (:import [java.nio ByteBuffer]
-           [java.util Collections])
+           [java.util Collections]
+           [backtype.storm.generated NimbusSummary])
   (:import [java.io FileNotFoundException])
   (:import [java.net InetAddress])
   (:import [java.nio.channels Channels WritableByteChannel])
@@ -104,6 +105,7 @@
      :id->sched-status (atom {})
      :cred-renewers (AuthUtils/GetCredentialRenewers conf)
      :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)
+     :nimbuses-cache (atom {}) ;;TODO need to figure out how to keep the cache upto date, one more thread
      }))
 
 (defn inbox [nimbus]
@@ -1030,6 +1032,17 @@
   (let [nimbus (nimbus-data conf inimbus)
        principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)]
     (.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
+
+    ;add to nimbuses
+    (.add-nimbus-host! (:storm-cluster-state nimbus)
+      (.toHostPortString (:nimbus-host-port-info nimbus))
+      {
+        :host (.getHost (:nimbus-host-port-info nimbus))
+        :port (.getPort (:nimbus-host-port-info nimbus))
+        :start-time-secs (current-time-secs)
+        :version (read-storm-version)
+        })
+
     (.addToLeaderLockQueue (:leader-elector nimbus))
     (cleanup-corrupt-topologies! nimbus)
     ;register call back for code-distributor
@@ -1287,8 +1300,14 @@
                                                                 (count (:used-ports info))
                                                                 id )
                                             ))
-              nimbus-uptime ((:uptime nimbus))
               bases (topology-bases storm-cluster-state)
+              nimbuses (.nimbuses storm-cluster-state)
+              nimbuses (map #(NimbusSummary. (:host %1) (:port %1) (time-delta (:start-time-secs %1))
+                               (let [leader (.getLeader (:leader-elector nimbus))]
+                                 (and (= (.getHost leader) (:host %1)) (= (.getPort leader) (:port %1))))
+                               (:version %1))
+                         nimbuses
+                         )
               topology-summaries (dofor [[id base] bases :when base]
 	                                  (let [assignment (.assignment-info storm-cluster-state id nil)
                                                 topo-summ (TopologySummary. id
@@ -1312,8 +1331,8 @@
                                                topo-summ
                                           ))]
           (ClusterSummary. supervisor-summaries
-                           nimbus-uptime
-                           topology-summaries)
+                           topology-summaries
+                           nimbuses)
           ))
       
       (^TopologyInfo getTopologyInfo [this ^String storm-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/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 3c7f578..94b0311 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -78,16 +78,6 @@
        (map #(.get_stats ^ExecutorSummary %))
        (filter not-nil?)))
 
-(defn read-storm-version
-  "Returns a string containing the Storm version or 'Unknown'."
-  []
-  (let [storm-home (System/getProperty "storm.home")
-        release-path (format "%s/RELEASE" storm-home)
-        release-file (File. release-path)]
-    (if (and (.exists release-file) (.isFile release-file))
-      (trim (slurp release-path))
-      "Unknown")))
-
 (defn component-type
   "Returns the component type (either :bolt or :spout) for a given
   topology and component id. Returns nil if not found."
@@ -520,7 +510,6 @@
                              (reduce +))]
        {"user" user
         "stormVersion" (read-storm-version)
-        "nimbusUptime" (pretty-uptime-sec (.get_nimbus_uptime_secs summ))
         "supervisors" (count sups)
         "slotsTotal" total-slots
         "slotsUsed"  used-slots
@@ -530,18 +519,19 @@
 
 (defn nimbus-summary
   ([]
-    (let [leader-elector (zk-leader-elector *STORM-CONF*)
-          nimbus-hosts (.getAllNimbuses leader-elector)
-          no-op (.close leader-elector)]
-      (nimbus-summary nimbus-hosts)))
+    (with-nimbus nimbus
+      (nimbus-summary
+        (.get_nimbuses (.getClusterInfo ^Nimbus$Client nimbus)))))
   ([nimbuses]
     {"nimbuses"
      (for [^NimbusInfo n nimbuses]
        {
-        "host" (.getHost n)
-        "port" (.getPort n)
-        "nimbusLogLink" (nimbus-log-link (.getHost n) (.getPort n))
-        "isLeader" (.isLeader n)})}))
+        "host" (.get_host n)
+        "port" (.get_port n)
+        "nimbusLogLink" (nimbus-log-link (.get_host n) (.get_port n))
+        "isLeader" (.is_isLeader n)
+        "version" (.get_version n)
+        "nimbusUpTime" (pretty-uptime-sec (.get_uptimeSecs n))})}))
 
 (defn supervisor-summary
   ([]

http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
index a2623ab..7e32c72 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
@@ -42,18 +42,18 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterSummary");
 
   private static final org.apache.thrift.protocol.TField SUPERVISORS_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisors", org.apache.thrift.protocol.TType.LIST, (short)1);
-  private static final org.apache.thrift.protocol.TField NIMBUS_UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus_uptime_secs", org.apache.thrift.protocol.TType.I32, (short)2);
   private static final org.apache.thrift.protocol.TField TOPOLOGIES_FIELD_DESC = new org.apache.thrift.protocol.TField("topologies", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField NIMBUSES_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbuses", org.apache.thrift.protocol.TType.LIST, (short)4);
 
   private List<SupervisorSummary> supervisors; // required
-  private int nimbus_uptime_secs; // required
   private List<TopologySummary> topologies; // required
+  private List<NimbusSummary> nimbuses; // 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 {
     SUPERVISORS((short)1, "supervisors"),
-    NIMBUS_UPTIME_SECS((short)2, "nimbus_uptime_secs"),
-    TOPOLOGIES((short)3, "topologies");
+    TOPOLOGIES((short)3, "topologies"),
+    NIMBUSES((short)4, "nimbuses");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -70,10 +70,10 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       switch(fieldId) {
         case 1: // SUPERVISORS
           return SUPERVISORS;
-        case 2: // NIMBUS_UPTIME_SECS
-          return NIMBUS_UPTIME_SECS;
         case 3: // TOPOLOGIES
           return TOPOLOGIES;
+        case 4: // NIMBUSES
+          return NIMBUSES;
         default:
           return null;
       }
@@ -114,8 +114,6 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
   }
 
   // isset id assignments
-  private static final int __NIMBUS_UPTIME_SECS_ISSET_ID = 0;
-  private BitSet __isset_bit_vector = new BitSet(1);
 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
@@ -123,11 +121,12 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     tmpMap.put(_Fields.SUPERVISORS, new org.apache.thrift.meta_data.FieldMetaData("supervisors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         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, SupervisorSummary.class))));
-    tmpMap.put(_Fields.NIMBUS_UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("nimbus_uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.TOPOLOGIES, new org.apache.thrift.meta_data.FieldMetaData("topologies", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         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, TopologySummary.class))));
+    tmpMap.put(_Fields.NIMBUSES, new org.apache.thrift.meta_data.FieldMetaData("nimbuses", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        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, NimbusSummary.class))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClusterSummary.class, metaDataMap);
   }
@@ -137,22 +136,19 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
 
   public ClusterSummary(
     List<SupervisorSummary> supervisors,
-    int nimbus_uptime_secs,
-    List<TopologySummary> topologies)
+    List<TopologySummary> topologies,
+    List<NimbusSummary> nimbuses)
   {
     this();
     this.supervisors = supervisors;
-    this.nimbus_uptime_secs = nimbus_uptime_secs;
-    set_nimbus_uptime_secs_isSet(true);
     this.topologies = topologies;
+    this.nimbuses = nimbuses;
   }
 
   /**
    * Performs a deep copy on <i>other</i>.
    */
   public ClusterSummary(ClusterSummary other) {
-    __isset_bit_vector.clear();
-    __isset_bit_vector.or(other.__isset_bit_vector);
     if (other.is_set_supervisors()) {
       List<SupervisorSummary> __this__supervisors = new ArrayList<SupervisorSummary>();
       for (SupervisorSummary other_element : other.supervisors) {
@@ -160,7 +156,6 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       }
       this.supervisors = __this__supervisors;
     }
-    this.nimbus_uptime_secs = other.nimbus_uptime_secs;
     if (other.is_set_topologies()) {
       List<TopologySummary> __this__topologies = new ArrayList<TopologySummary>();
       for (TopologySummary other_element : other.topologies) {
@@ -168,6 +163,13 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       }
       this.topologies = __this__topologies;
     }
+    if (other.is_set_nimbuses()) {
+      List<NimbusSummary> __this__nimbuses = new ArrayList<NimbusSummary>();
+      for (NimbusSummary other_element : other.nimbuses) {
+        __this__nimbuses.add(new NimbusSummary(other_element));
+      }
+      this.nimbuses = __this__nimbuses;
+    }
   }
 
   public ClusterSummary deepCopy() {
@@ -177,9 +179,8 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
   @Override
   public void clear() {
     this.supervisors = null;
-    set_nimbus_uptime_secs_isSet(false);
-    this.nimbus_uptime_secs = 0;
     this.topologies = null;
+    this.nimbuses = null;
   }
 
   public int get_supervisors_size() {
@@ -220,28 +221,6 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     }
   }
 
-  public int get_nimbus_uptime_secs() {
-    return this.nimbus_uptime_secs;
-  }
-
-  public void set_nimbus_uptime_secs(int nimbus_uptime_secs) {
-    this.nimbus_uptime_secs = nimbus_uptime_secs;
-    set_nimbus_uptime_secs_isSet(true);
-  }
-
-  public void unset_nimbus_uptime_secs() {
-    __isset_bit_vector.clear(__NIMBUS_UPTIME_SECS_ISSET_ID);
-  }
-
-  /** Returns true if field nimbus_uptime_secs is set (has been assigned a value) and false otherwise */
-  public boolean is_set_nimbus_uptime_secs() {
-    return __isset_bit_vector.get(__NIMBUS_UPTIME_SECS_ISSET_ID);
-  }
-
-  public void set_nimbus_uptime_secs_isSet(boolean value) {
-    __isset_bit_vector.set(__NIMBUS_UPTIME_SECS_ISSET_ID, value);
-  }
-
   public int get_topologies_size() {
     return (this.topologies == null) ? 0 : this.topologies.size();
   }
@@ -280,6 +259,44 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     }
   }
 
+  public int get_nimbuses_size() {
+    return (this.nimbuses == null) ? 0 : this.nimbuses.size();
+  }
+
+  public java.util.Iterator<NimbusSummary> get_nimbuses_iterator() {
+    return (this.nimbuses == null) ? null : this.nimbuses.iterator();
+  }
+
+  public void add_to_nimbuses(NimbusSummary elem) {
+    if (this.nimbuses == null) {
+      this.nimbuses = new ArrayList<NimbusSummary>();
+    }
+    this.nimbuses.add(elem);
+  }
+
+  public List<NimbusSummary> get_nimbuses() {
+    return this.nimbuses;
+  }
+
+  public void set_nimbuses(List<NimbusSummary> nimbuses) {
+    this.nimbuses = nimbuses;
+  }
+
+  public void unset_nimbuses() {
+    this.nimbuses = null;
+  }
+
+  /** Returns true if field nimbuses is set (has been assigned a value) and false otherwise */
+  public boolean is_set_nimbuses() {
+    return this.nimbuses != null;
+  }
+
+  public void set_nimbuses_isSet(boolean value) {
+    if (!value) {
+      this.nimbuses = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case SUPERVISORS:
@@ -290,19 +307,19 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       }
       break;
 
-    case NIMBUS_UPTIME_SECS:
+    case TOPOLOGIES:
       if (value == null) {
-        unset_nimbus_uptime_secs();
+        unset_topologies();
       } else {
-        set_nimbus_uptime_secs((Integer)value);
+        set_topologies((List<TopologySummary>)value);
       }
       break;
 
-    case TOPOLOGIES:
+    case NIMBUSES:
       if (value == null) {
-        unset_topologies();
+        unset_nimbuses();
       } else {
-        set_topologies((List<TopologySummary>)value);
+        set_nimbuses((List<NimbusSummary>)value);
       }
       break;
 
@@ -314,12 +331,12 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     case SUPERVISORS:
       return get_supervisors();
 
-    case NIMBUS_UPTIME_SECS:
-      return Integer.valueOf(get_nimbus_uptime_secs());
-
     case TOPOLOGIES:
       return get_topologies();
 
+    case NIMBUSES:
+      return get_nimbuses();
+
     }
     throw new IllegalStateException();
   }
@@ -333,10 +350,10 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     switch (field) {
     case SUPERVISORS:
       return is_set_supervisors();
-    case NIMBUS_UPTIME_SECS:
-      return is_set_nimbus_uptime_secs();
     case TOPOLOGIES:
       return is_set_topologies();
+    case NIMBUSES:
+      return is_set_nimbuses();
     }
     throw new IllegalStateException();
   }
@@ -363,15 +380,6 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
         return false;
     }
 
-    boolean this_present_nimbus_uptime_secs = true;
-    boolean that_present_nimbus_uptime_secs = true;
-    if (this_present_nimbus_uptime_secs || that_present_nimbus_uptime_secs) {
-      if (!(this_present_nimbus_uptime_secs && that_present_nimbus_uptime_secs))
-        return false;
-      if (this.nimbus_uptime_secs != that.nimbus_uptime_secs)
-        return false;
-    }
-
     boolean this_present_topologies = true && this.is_set_topologies();
     boolean that_present_topologies = true && that.is_set_topologies();
     if (this_present_topologies || that_present_topologies) {
@@ -381,6 +389,15 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
         return false;
     }
 
+    boolean this_present_nimbuses = true && this.is_set_nimbuses();
+    boolean that_present_nimbuses = true && that.is_set_nimbuses();
+    if (this_present_nimbuses || that_present_nimbuses) {
+      if (!(this_present_nimbuses && that_present_nimbuses))
+        return false;
+      if (!this.nimbuses.equals(that.nimbuses))
+        return false;
+    }
+
     return true;
   }
 
@@ -393,16 +410,16 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     if (present_supervisors)
       builder.append(supervisors);
 
-    boolean present_nimbus_uptime_secs = true;
-    builder.append(present_nimbus_uptime_secs);
-    if (present_nimbus_uptime_secs)
-      builder.append(nimbus_uptime_secs);
-
     boolean present_topologies = true && (is_set_topologies());
     builder.append(present_topologies);
     if (present_topologies)
       builder.append(topologies);
 
+    boolean present_nimbuses = true && (is_set_nimbuses());
+    builder.append(present_nimbuses);
+    if (present_nimbuses)
+      builder.append(nimbuses);
+
     return builder.toHashCode();
   }
 
@@ -424,22 +441,22 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_nimbus_uptime_secs()).compareTo(typedOther.is_set_nimbus_uptime_secs());
+    lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(typedOther.is_set_topologies());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (is_set_nimbus_uptime_secs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus_uptime_secs, typedOther.nimbus_uptime_secs);
+    if (is_set_topologies()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, typedOther.topologies);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(typedOther.is_set_topologies());
+    lastComparison = Boolean.valueOf(is_set_nimbuses()).compareTo(typedOther.is_set_nimbuses());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (is_set_topologies()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, typedOther.topologies);
+    if (is_set_nimbuses()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbuses, typedOther.nimbuses);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -479,14 +496,6 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 2: // NIMBUS_UPTIME_SECS
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.nimbus_uptime_secs = iprot.readI32();
-            set_nimbus_uptime_secs_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
         case 3: // TOPOLOGIES
           if (field.type == org.apache.thrift.protocol.TType.LIST) {
             {
@@ -505,6 +514,24 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 4: // NIMBUSES
+          if (field.type == org.apache.thrift.protocol.TType.LIST) {
+            {
+              org.apache.thrift.protocol.TList _list43 = iprot.readListBegin();
+              this.nimbuses = new ArrayList<NimbusSummary>(_list43.size);
+              for (int _i44 = 0; _i44 < _list43.size; ++_i44)
+              {
+                NimbusSummary _elem45; // required
+                _elem45 = new NimbusSummary();
+                _elem45.read(iprot);
+                this.nimbuses.add(_elem45);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
       }
@@ -522,24 +549,33 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC);
       {
         oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, this.supervisors.size()));
-        for (SupervisorSummary _iter43 : this.supervisors)
+        for (SupervisorSummary _iter46 : this.supervisors)
         {
-          _iter43.write(oprot);
+          _iter46.write(oprot);
         }
         oprot.writeListEnd();
       }
       oprot.writeFieldEnd();
     }
-    oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC);
-    oprot.writeI32(this.nimbus_uptime_secs);
-    oprot.writeFieldEnd();
     if (this.topologies != null) {
       oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC);
       {
         oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, this.topologies.size()));
-        for (TopologySummary _iter44 : this.topologies)
+        for (TopologySummary _iter47 : this.topologies)
+        {
+          _iter47.write(oprot);
+        }
+        oprot.writeListEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.nimbuses != null) {
+      oprot.writeFieldBegin(NIMBUSES_FIELD_DESC);
+      {
+        oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, this.nimbuses.size()));
+        for (NimbusSummary _iter48 : this.nimbuses)
         {
-          _iter44.write(oprot);
+          _iter48.write(oprot);
         }
         oprot.writeListEnd();
       }
@@ -562,10 +598,6 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     }
     first = false;
     if (!first) sb.append(", ");
-    sb.append("nimbus_uptime_secs:");
-    sb.append(this.nimbus_uptime_secs);
-    first = false;
-    if (!first) sb.append(", ");
     sb.append("topologies:");
     if (this.topologies == null) {
       sb.append("null");
@@ -573,6 +605,14 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       sb.append(this.topologies);
     }
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("nimbuses:");
+    if (this.nimbuses == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.nimbuses);
+    }
+    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -583,14 +623,14 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisors' is unset! Struct:" + toString());
     }
 
-    if (!is_set_nimbus_uptime_secs()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbus_uptime_secs' is unset! Struct:" + toString());
-    }
-
     if (!is_set_topologies()) {
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'topologies' is unset! Struct:" + toString());
     }
 
+    if (!is_set_nimbuses()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbuses' is unset! Struct:" + toString());
+    }
+
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -603,8 +643,6 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
 
   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_bit_vector = new BitSet(1);
       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);

http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
new file mode 100644
index 0000000..195048a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
@@ -0,0 +1,723 @@
+/**
+ * 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.7.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, NimbusSummary._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusSummary");
+
+  private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptimeSecs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField IS_LEADER_FIELD_DESC = new org.apache.thrift.protocol.TField("isLeader", org.apache.thrift.protocol.TType.BOOL, (short)4);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)5);
+
+  private String host; // required
+  private int port; // required
+  private int uptimeSecs; // required
+  private boolean isLeader; // required
+  private String version; // 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 {
+    HOST((short)1, "host"),
+    PORT((short)2, "port"),
+    UPTIME_SECS((short)3, "uptimeSecs"),
+    IS_LEADER((short)4, "isLeader"),
+    VERSION((short)5, "version");
+
+    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: // HOST
+          return HOST;
+        case 2: // PORT
+          return PORT;
+        case 3: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 4: // IS_LEADER
+          return IS_LEADER;
+        case 5: // VERSION
+          return VERSION;
+        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 __PORT_ISSET_ID = 0;
+  private static final int __UPTIMESECS_ISSET_ID = 1;
+  private static final int __ISLEADER_ISSET_ID = 2;
+  private BitSet __isset_bit_vector = new BitSet(3);
+
+  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.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptimeSecs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.IS_LEADER, new org.apache.thrift.meta_data.FieldMetaData("isLeader", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NimbusSummary.class, metaDataMap);
+  }
+
+  public NimbusSummary() {
+  }
+
+  public NimbusSummary(
+    String host,
+    int port,
+    int uptimeSecs,
+    boolean isLeader,
+    String version)
+  {
+    this();
+    this.host = host;
+    this.port = port;
+    set_port_isSet(true);
+    this.uptimeSecs = uptimeSecs;
+    set_uptimeSecs_isSet(true);
+    this.isLeader = isLeader;
+    set_isLeader_isSet(true);
+    this.version = version;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NimbusSummary(NimbusSummary other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.port = other.port;
+    this.uptimeSecs = other.uptimeSecs;
+    this.isLeader = other.isLeader;
+    if (other.is_set_version()) {
+      this.version = other.version;
+    }
+  }
+
+  public NimbusSummary deepCopy() {
+    return new NimbusSummary(this);
+  }
+
+  @Override
+  public void clear() {
+    this.host = null;
+    set_port_isSet(false);
+    this.port = 0;
+    set_uptimeSecs_isSet(false);
+    this.uptimeSecs = 0;
+    set_isLeader_isSet(false);
+    this.isLeader = false;
+    this.version = null;
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bit_vector.clear(__PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return __isset_bit_vector.get(__PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bit_vector.set(__PORT_ISSET_ID, value);
+  }
+
+  public int get_uptimeSecs() {
+    return this.uptimeSecs;
+  }
+
+  public void set_uptimeSecs(int uptimeSecs) {
+    this.uptimeSecs = uptimeSecs;
+    set_uptimeSecs_isSet(true);
+  }
+
+  public void unset_uptimeSecs() {
+    __isset_bit_vector.clear(__UPTIMESECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptimeSecs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptimeSecs() {
+    return __isset_bit_vector.get(__UPTIMESECS_ISSET_ID);
+  }
+
+  public void set_uptimeSecs_isSet(boolean value) {
+    __isset_bit_vector.set(__UPTIMESECS_ISSET_ID, value);
+  }
+
+  public boolean is_isLeader() {
+    return this.isLeader;
+  }
+
+  public void set_isLeader(boolean isLeader) {
+    this.isLeader = isLeader;
+    set_isLeader_isSet(true);
+  }
+
+  public void unset_isLeader() {
+    __isset_bit_vector.clear(__ISLEADER_ISSET_ID);
+  }
+
+  /** Returns true if field isLeader is set (has been assigned a value) and false otherwise */
+  public boolean is_set_isLeader() {
+    return __isset_bit_vector.get(__ISLEADER_ISSET_ID);
+  }
+
+  public void set_isLeader_isSet(boolean value) {
+    __isset_bit_vector.set(__ISLEADER_ISSET_ID, value);
+  }
+
+  public String get_version() {
+    return this.version;
+  }
+
+  public void set_version(String version) {
+    this.version = version;
+  }
+
+  public void unset_version() {
+    this.version = null;
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean is_set_version() {
+    return this.version != null;
+  }
+
+  public void set_version_isSet(boolean value) {
+    if (!value) {
+      this.version = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptimeSecs();
+      } else {
+        set_uptimeSecs((Integer)value);
+      }
+      break;
+
+    case IS_LEADER:
+      if (value == null) {
+        unset_isLeader();
+      } else {
+        set_isLeader((Boolean)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unset_version();
+      } else {
+        set_version((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case HOST:
+      return get_host();
+
+    case PORT:
+      return Integer.valueOf(get_port());
+
+    case UPTIME_SECS:
+      return Integer.valueOf(get_uptimeSecs());
+
+    case IS_LEADER:
+      return Boolean.valueOf(is_isLeader());
+
+    case VERSION:
+      return get_version();
+
+    }
+    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 HOST:
+      return is_set_host();
+    case PORT:
+      return is_set_port();
+    case UPTIME_SECS:
+      return is_set_uptimeSecs();
+    case IS_LEADER:
+      return is_set_isLeader();
+    case VERSION:
+      return is_set_version();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NimbusSummary)
+      return this.equals((NimbusSummary)that);
+    return false;
+  }
+
+  public boolean equals(NimbusSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_port = true;
+    boolean that_present_port = true;
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    boolean this_present_uptimeSecs = true;
+    boolean that_present_uptimeSecs = true;
+    if (this_present_uptimeSecs || that_present_uptimeSecs) {
+      if (!(this_present_uptimeSecs && that_present_uptimeSecs))
+        return false;
+      if (this.uptimeSecs != that.uptimeSecs)
+        return false;
+    }
+
+    boolean this_present_isLeader = true;
+    boolean that_present_isLeader = true;
+    if (this_present_isLeader || that_present_isLeader) {
+      if (!(this_present_isLeader && that_present_isLeader))
+        return false;
+      if (this.isLeader != that.isLeader)
+        return false;
+    }
+
+    boolean this_present_version = true && this.is_set_version();
+    boolean that_present_version = true && that.is_set_version();
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (!this.version.equals(that.version))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_host = true && (is_set_host());
+    builder.append(present_host);
+    if (present_host)
+      builder.append(host);
+
+    boolean present_port = true;
+    builder.append(present_port);
+    if (present_port)
+      builder.append(port);
+
+    boolean present_uptimeSecs = true;
+    builder.append(present_uptimeSecs);
+    if (present_uptimeSecs)
+      builder.append(uptimeSecs);
+
+    boolean present_isLeader = true;
+    builder.append(present_isLeader);
+    if (present_isLeader)
+      builder.append(isLeader);
+
+    boolean present_version = true && (is_set_version());
+    builder.append(present_version);
+    if (present_version)
+      builder.append(version);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(NimbusSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    NimbusSummary typedOther = (NimbusSummary)other;
+
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, typedOther.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, typedOther.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptimeSecs()).compareTo(typedOther.is_set_uptimeSecs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptimeSecs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptimeSecs, typedOther.uptimeSecs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_isLeader()).compareTo(typedOther.is_set_isLeader());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_isLeader()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isLeader, typedOther.isLeader);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(typedOther.is_set_version());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_version()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, typedOther.version);
+      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 {
+    org.apache.thrift.protocol.TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // HOST
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.host = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // PORT
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.port = iprot.readI32();
+            set_port_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // UPTIME_SECS
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.uptimeSecs = iprot.readI32();
+            set_uptimeSecs_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // IS_LEADER
+          if (field.type == org.apache.thrift.protocol.TType.BOOL) {
+            this.isLeader = iprot.readBool();
+            set_isLeader_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 5: // VERSION
+          if (field.type == org.apache.thrift.protocol.TType.STRING) {
+            this.version = iprot.readString();
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.host != null) {
+      oprot.writeFieldBegin(HOST_FIELD_DESC);
+      oprot.writeString(this.host);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(PORT_FIELD_DESC);
+    oprot.writeI32(this.port);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.uptimeSecs);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(IS_LEADER_FIELD_DESC);
+    oprot.writeBool(this.isLeader);
+    oprot.writeFieldEnd();
+    if (this.version != null) {
+      oprot.writeFieldBegin(VERSION_FIELD_DESC);
+      oprot.writeString(this.version);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NimbusSummary(");
+    boolean first = true;
+
+    sb.append("host:");
+    if (this.host == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.host);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    sb.append(this.port);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptimeSecs:");
+    sb.append(this.uptimeSecs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("isLeader:");
+    sb.append(this.isLeader);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("version:");
+    if (this.version == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.version);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_host()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptimeSecs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptimeSecs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_isLeader()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'isLeader' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_version()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString());
+    }
+
+  }
+
+  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_bit_vector = new BitSet(1);
+      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);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/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 e4fb751..a5e155c 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -2354,28 +2354,150 @@ class SupervisorSummary:
   def __ne__(self, other):
     return not (self == other)
 
+class NimbusSummary:
+  """
+  Attributes:
+   - host
+   - port
+   - uptimeSecs
+   - isLeader
+   - version
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'host', None, None, ), # 1
+    (2, TType.I32, 'port', None, None, ), # 2
+    (3, TType.I32, 'uptimeSecs', None, None, ), # 3
+    (4, TType.BOOL, 'isLeader', None, None, ), # 4
+    (5, TType.STRING, 'version', None, None, ), # 5
+  )
+
+  def __hash__(self):
+    return 0 + hash(self.host) + hash(self.port) + hash(self.uptimeSecs) + hash(self.isLeader) + hash(self.version)
+
+  def __init__(self, host=None, port=None, uptimeSecs=None, isLeader=None, version=None,):
+    self.host = host
+    self.port = port
+    self.uptimeSecs = uptimeSecs
+    self.isLeader = isLeader
+    self.version = version
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.host = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.port = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.uptimeSecs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.BOOL:
+          self.isLeader = iprot.readBool();
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.version = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('NimbusSummary')
+    if self.host is not None:
+      oprot.writeFieldBegin('host', TType.STRING, 1)
+      oprot.writeString(self.host.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.port is not None:
+      oprot.writeFieldBegin('port', TType.I32, 2)
+      oprot.writeI32(self.port)
+      oprot.writeFieldEnd()
+    if self.uptimeSecs is not None:
+      oprot.writeFieldBegin('uptimeSecs', TType.I32, 3)
+      oprot.writeI32(self.uptimeSecs)
+      oprot.writeFieldEnd()
+    if self.isLeader is not None:
+      oprot.writeFieldBegin('isLeader', TType.BOOL, 4)
+      oprot.writeBool(self.isLeader)
+      oprot.writeFieldEnd()
+    if self.version is not None:
+      oprot.writeFieldBegin('version', TType.STRING, 5)
+      oprot.writeString(self.version.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.host is None:
+      raise TProtocol.TProtocolException(message='Required field host is unset!')
+    if self.port is None:
+      raise TProtocol.TProtocolException(message='Required field port is unset!')
+    if self.uptimeSecs is None:
+      raise TProtocol.TProtocolException(message='Required field uptimeSecs is unset!')
+    if self.isLeader is None:
+      raise TProtocol.TProtocolException(message='Required field isLeader is unset!')
+    if self.version is None:
+      raise TProtocol.TProtocolException(message='Required field version is unset!')
+    return
+
+
+  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 ClusterSummary:
   """
   Attributes:
    - supervisors
-   - nimbus_uptime_secs
    - topologies
+   - nimbuses
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.LIST, 'supervisors', (TType.STRUCT,(SupervisorSummary, SupervisorSummary.thrift_spec)), None, ), # 1
-    (2, TType.I32, 'nimbus_uptime_secs', None, None, ), # 2
+    None, # 2
     (3, TType.LIST, 'topologies', (TType.STRUCT,(TopologySummary, TopologySummary.thrift_spec)), None, ), # 3
+    (4, TType.LIST, 'nimbuses', (TType.STRUCT,(NimbusSummary, NimbusSummary.thrift_spec)), None, ), # 4
   )
 
   def __hash__(self):
-    return 0 + hash(self.supervisors) + hash(self.nimbus_uptime_secs) + hash(self.topologies)
+    return 0 + hash(self.supervisors) + hash(self.topologies) + hash(self.nimbuses)
 
-  def __init__(self, supervisors=None, nimbus_uptime_secs=None, topologies=None,):
+  def __init__(self, supervisors=None, topologies=None, nimbuses=None,):
     self.supervisors = supervisors
-    self.nimbus_uptime_secs = nimbus_uptime_secs
     self.topologies = topologies
+    self.nimbuses = nimbuses
 
   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:
@@ -2397,11 +2519,6 @@ class ClusterSummary:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.I32:
-          self.nimbus_uptime_secs = iprot.readI32();
-        else:
-          iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.topologies = []
@@ -2413,6 +2530,17 @@ class ClusterSummary:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.nimbuses = []
+          (_etype81, _size78) = iprot.readListBegin()
+          for _i82 in xrange(_size78):
+            _elem83 = NimbusSummary()
+            _elem83.read(iprot)
+            self.nimbuses.append(_elem83)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2426,19 +2554,22 @@ class ClusterSummary:
     if self.supervisors is not None:
       oprot.writeFieldBegin('supervisors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.supervisors))
-      for iter78 in self.supervisors:
-        iter78.write(oprot)
+      for iter84 in self.supervisors:
+        iter84.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
-    if self.nimbus_uptime_secs is not None:
-      oprot.writeFieldBegin('nimbus_uptime_secs', TType.I32, 2)
-      oprot.writeI32(self.nimbus_uptime_secs)
-      oprot.writeFieldEnd()
     if self.topologies is not None:
       oprot.writeFieldBegin('topologies', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.topologies))
-      for iter79 in self.topologies:
-        iter79.write(oprot)
+      for iter85 in self.topologies:
+        iter85.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.nimbuses is not None:
+      oprot.writeFieldBegin('nimbuses', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRUCT, len(self.nimbuses))
+      for iter86 in self.nimbuses:
+        iter86.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -2447,10 +2578,10 @@ class ClusterSummary:
   def validate(self):
     if self.supervisors is None:
       raise TProtocol.TProtocolException(message='Required field supervisors is unset!')
-    if self.nimbus_uptime_secs is None:
-      raise TProtocol.TProtocolException(message='Required field nimbus_uptime_secs is unset!')
     if self.topologies is None:
       raise TProtocol.TProtocolException(message='Required field topologies is unset!')
+    if self.nimbuses is None:
+      raise TProtocol.TProtocolException(message='Required field nimbuses is unset!')
     return
 
 
@@ -2609,90 +2740,90 @@ class BoltStats:
       if fid == 1:
         if ftype == TType.MAP:
           self.acked = {}
-          (_ktype81, _vtype82, _size80 ) = iprot.readMapBegin() 
-          for _i84 in xrange(_size80):
-            _key85 = iprot.readString().decode('utf-8')
-            _val86 = {}
-            (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin() 
-            for _i91 in xrange(_size87):
-              _key92 = GlobalStreamId()
-              _key92.read(iprot)
-              _val93 = iprot.readI64();
-              _val86[_key92] = _val93
+          (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin() 
+          for _i91 in xrange(_size87):
+            _key92 = iprot.readString().decode('utf-8')
+            _val93 = {}
+            (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin() 
+            for _i98 in xrange(_size94):
+              _key99 = GlobalStreamId()
+              _key99.read(iprot)
+              _val100 = iprot.readI64();
+              _val93[_key99] = _val100
             iprot.readMapEnd()
-            self.acked[_key85] = _val86
+            self.acked[_key92] = _val93
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.failed = {}
-          (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin() 
-          for _i98 in xrange(_size94):
-            _key99 = iprot.readString().decode('utf-8')
-            _val100 = {}
-            (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin() 
-            for _i105 in xrange(_size101):
-              _key106 = GlobalStreamId()
-              _key106.read(iprot)
-              _val107 = iprot.readI64();
-              _val100[_key106] = _val107
+          (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin() 
+          for _i105 in xrange(_size101):
+            _key106 = iprot.readString().decode('utf-8')
+            _val107 = {}
+            (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin() 
+            for _i112 in xrange(_size108):
+              _key113 = GlobalStreamId()
+              _key113.read(iprot)
+              _val114 = iprot.readI64();
+              _val107[_key113] = _val114
             iprot.readMapEnd()
-            self.failed[_key99] = _val100
+            self.failed[_key106] = _val107
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.process_ms_avg = {}
-          (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin() 
-          for _i112 in xrange(_size108):
-            _key113 = iprot.readString().decode('utf-8')
-            _val114 = {}
-            (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin() 
-            for _i119 in xrange(_size115):
-              _key120 = GlobalStreamId()
-              _key120.read(iprot)
-              _val121 = iprot.readDouble();
-              _val114[_key120] = _val121
+          (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin() 
+          for _i119 in xrange(_size115):
+            _key120 = iprot.readString().decode('utf-8')
+            _val121 = {}
+            (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin() 
+            for _i126 in xrange(_size122):
+              _key127 = GlobalStreamId()
+              _key127.read(iprot)
+              _val128 = iprot.readDouble();
+              _val121[_key127] = _val128
             iprot.readMapEnd()
-            self.process_ms_avg[_key113] = _val114
+            self.process_ms_avg[_key120] = _val121
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.executed = {}
-          (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin() 
-          for _i126 in xrange(_size122):
-            _key127 = iprot.readString().decode('utf-8')
-            _val128 = {}
-            (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin() 
-            for _i133 in xrange(_size129):
-              _key134 = GlobalStreamId()
-              _key134.read(iprot)
-              _val135 = iprot.readI64();
-              _val128[_key134] = _val135
+          (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin() 
+          for _i133 in xrange(_size129):
+            _key134 = iprot.readString().decode('utf-8')
+            _val135 = {}
+            (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin() 
+            for _i140 in xrange(_size136):
+              _key141 = GlobalStreamId()
+              _key141.read(iprot)
+              _val142 = iprot.readI64();
+              _val135[_key141] = _val142
             iprot.readMapEnd()
-            self.executed[_key127] = _val128
+            self.executed[_key134] = _val135
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.MAP:
           self.execute_ms_avg = {}
-          (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin() 
-          for _i140 in xrange(_size136):
-            _key141 = iprot.readString().decode('utf-8')
-            _val142 = {}
-            (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin() 
-            for _i147 in xrange(_size143):
-              _key148 = GlobalStreamId()
-              _key148.read(iprot)
-              _val149 = iprot.readDouble();
-              _val142[_key148] = _val149
+          (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin() 
+          for _i147 in xrange(_size143):
+            _key148 = iprot.readString().decode('utf-8')
+            _val149 = {}
+            (_ktype151, _vtype152, _size150 ) = iprot.readMapBegin() 
+            for _i154 in xrange(_size150):
+              _key155 = GlobalStreamId()
+              _key155.read(iprot)
+              _val156 = iprot.readDouble();
+              _val149[_key155] = _val156
             iprot.readMapEnd()
-            self.execute_ms_avg[_key141] = _val142
+            self.execute_ms_avg[_key148] = _val149
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -2709,60 +2840,60 @@ class BoltStats:
     if self.acked is not None:
       oprot.writeFieldBegin('acked', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked))
-      for kiter150,viter151 in self.acked.items():
-        oprot.writeString(kiter150.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter151))
-        for kiter152,viter153 in viter151.items():
-          kiter152.write(oprot)
-          oprot.writeI64(viter153)
+      for kiter157,viter158 in self.acked.items():
+        oprot.writeString(kiter157.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter158))
+        for kiter159,viter160 in viter158.items():
+          kiter159.write(oprot)
+          oprot.writeI64(viter160)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.failed is not None:
       oprot.writeFieldBegin('failed', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed))
-      for kiter154,viter155 in self.failed.items():
-        oprot.writeString(kiter154.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter155))
-        for kiter156,viter157 in viter155.items():
-          kiter156.write(oprot)
-          oprot.writeI64(viter157)
+      for kiter161,viter162 in self.failed.items():
+        oprot.writeString(kiter161.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter162))
+        for kiter163,viter164 in viter162.items():
+          kiter163.write(oprot)
+          oprot.writeI64(viter164)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.process_ms_avg is not None:
       oprot.writeFieldBegin('process_ms_avg', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.process_ms_avg))
-      for kiter158,viter159 in self.process_ms_avg.items():
-        oprot.writeString(kiter158.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter159))
-        for kiter160,viter161 in viter159.items():
-          kiter160.write(oprot)
-          oprot.writeDouble(viter161)
+      for kiter165,viter166 in self.process_ms_avg.items():
+        oprot.writeString(kiter165.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter166))
+        for kiter167,viter168 in viter166.items():
+          kiter167.write(oprot)
+          oprot.writeDouble(viter168)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executed is not None:
       oprot.writeFieldBegin('executed', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.executed))
-      for kiter162,viter163 in self.executed.items():
-        oprot.writeString(kiter162.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter163))
-        for kiter164,viter165 in viter163.items():
-          kiter164.write(oprot)
-          oprot.writeI64(viter165)
+      for kiter169,viter170 in self.executed.items():
+        oprot.writeString(kiter169.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter170))
+        for kiter171,viter172 in viter170.items():
+          kiter171.write(oprot)
+          oprot.writeI64(viter172)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.execute_ms_avg is not None:
       oprot.writeFieldBegin('execute_ms_avg', TType.MAP, 5)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.execute_ms_avg))
-      for kiter166,viter167 in self.execute_ms_avg.items():
-        oprot.writeString(kiter166.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter167))
-        for kiter168,viter169 in viter167.items():
-          kiter168.write(oprot)
-          oprot.writeDouble(viter169)
+      for kiter173,viter174 in self.execute_ms_avg.items():
+        oprot.writeString(kiter173.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter174))
+        for kiter175,viter176 in viter174.items():
+          kiter175.write(oprot)
+          oprot.writeDouble(viter176)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -2829,51 +2960,51 @@ class SpoutStats:
       if fid == 1:
         if ftype == TType.MAP:
           self.acked = {}
-          (_ktype171, _vtype172, _size170 ) = iprot.readMapBegin() 
-          for _i174 in xrange(_size170):
-            _key175 = iprot.readString().decode('utf-8')
-            _val176 = {}
-            (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin() 
-            for _i181 in xrange(_size177):
-              _key182 = iprot.readString().decode('utf-8')
-              _val183 = iprot.readI64();
-              _val176[_key182] = _val183
+          (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin() 
+          for _i181 in xrange(_size177):
+            _key182 = iprot.readString().decode('utf-8')
+            _val183 = {}
+            (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin() 
+            for _i188 in xrange(_size184):
+              _key189 = iprot.readString().decode('utf-8')
+              _val190 = iprot.readI64();
+              _val183[_key189] = _val190
             iprot.readMapEnd()
-            self.acked[_key175] = _val176
+            self.acked[_key182] = _val183
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.failed = {}
-          (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin() 
-          for _i188 in xrange(_size184):
-            _key189 = iprot.readString().decode('utf-8')
-            _val190 = {}
-            (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin() 
-            for _i195 in xrange(_size191):
-              _key196 = iprot.readString().decode('utf-8')
-              _val197 = iprot.readI64();
-              _val190[_key196] = _val197
+          (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin() 
+          for _i195 in xrange(_size191):
+            _key196 = iprot.readString().decode('utf-8')
+            _val197 = {}
+            (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin() 
+            for _i202 in xrange(_size198):
+              _key203 = iprot.readString().decode('utf-8')
+              _val204 = iprot.readI64();
+              _val197[_key203] = _val204
             iprot.readMapEnd()
-            self.failed[_key189] = _val190
+            self.failed[_key196] = _val197
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.complete_ms_avg = {}
-          (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin() 
-          for _i202 in xrange(_size198):
-            _key203 = iprot.readString().decode('utf-8')
-            _val204 = {}
-            (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin() 
-            for _i209 in xrange(_size205):
-              _key210 = iprot.readString().decode('utf-8')
-              _val211 = iprot.readDouble();
-              _val204[_key210] = _val211
+          (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin() 
+          for _i209 in xrange(_size205):
+            _key210 = iprot.readString().decode('utf-8')
+            _val211 = {}
+            (_ktype213, _vtype214, _size212 ) = iprot.readMapBegin() 
+            for _i216 in xrange(_size212):
+              _key217 = iprot.readString().decode('utf-8')
+              _val218 = iprot.readDouble();
+              _val211[_key217] = _val218
             iprot.readMapEnd()
-            self.complete_ms_avg[_key203] = _val204
+            self.complete_ms_avg[_key210] = _val211
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -2890,36 +3021,36 @@ class SpoutStats:
     if self.acked is not None:
       oprot.writeFieldBegin('acked', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked))
-      for kiter212,viter213 in self.acked.items():
-        oprot.writeString(kiter212.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter213))
-        for kiter214,viter215 in viter213.items():
-          oprot.writeString(kiter214.encode('utf-8'))
-          oprot.writeI64(viter215)
+      for kiter219,viter220 in self.acked.items():
+        oprot.writeString(kiter219.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter220))
+        for kiter221,viter222 in viter220.items():
+          oprot.writeString(kiter221.encode('utf-8'))
+          oprot.writeI64(viter222)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.failed is not None:
       oprot.writeFieldBegin('failed', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed))
-      for kiter216,viter217 in self.failed.items():
-        oprot.writeString(kiter216.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter217))
-        for kiter218,viter219 in viter217.items():
-          oprot.writeString(kiter218.encode('utf-8'))
-          oprot.writeI64(viter219)
+      for kiter223,viter224 in self.failed.items():
+        oprot.writeString(kiter223.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter224))
+        for kiter225,viter226 in viter224.items():
+          oprot.writeString(kiter225.encode('utf-8'))
+          oprot.writeI64(viter226)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.complete_ms_avg is not None:
       oprot.writeFieldBegin('complete_ms_avg', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.complete_ms_avg))
-      for kiter220,viter221 in self.complete_ms_avg.items():
-        oprot.writeString(kiter220.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter221))
-        for kiter222,viter223 in viter221.items():
-          oprot.writeString(kiter222.encode('utf-8'))
-          oprot.writeDouble(viter223)
+      for kiter227,viter228 in self.complete_ms_avg.items():
+        oprot.writeString(kiter227.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter228))
+        for kiter229,viter230 in viter228.items():
+          oprot.writeString(kiter229.encode('utf-8'))
+          oprot.writeDouble(viter230)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -3059,34 +3190,34 @@ class ExecutorStats:
       if fid == 1:
         if ftype == TType.MAP:
           self.emitted = {}
-          (_ktype225, _vtype226, _size224 ) = iprot.readMapBegin() 
-          for _i228 in xrange(_size224):
-            _key229 = iprot.readString().decode('utf-8')
-            _val230 = {}
-            (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin() 
-            for _i235 in xrange(_size231):
-              _key236 = iprot.readString().decode('utf-8')
-              _val237 = iprot.readI64();
-              _val230[_key236] = _val237
+          (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin() 
+          for _i235 in xrange(_size231):
+            _key236 = iprot.readString().decode('utf-8')
+            _val237 = {}
+            (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin() 
+            for _i242 in xrange(_size238):
+              _key243 = iprot.readString().decode('utf-8')
+              _val244 = iprot.readI64();
+              _val237[_key243] = _val244
             iprot.readMapEnd()
-            self.emitted[_key229] = _val230
+            self.emitted[_key236] = _val237
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.transferred = {}
-          (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin() 
-          for _i242 in xrange(_size238):
-            _key243 = iprot.readString().decode('utf-8')
-            _val244 = {}
-            (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin() 
-            for _i249 in xrange(_size245):
-              _key250 = iprot.readString().decode('utf-8')
-              _val251 = iprot.readI64();
-              _val244[_key250] = _val251
+          (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin() 
+          for _i249 in xrange(_size245):
+            _key250 = iprot.readString().decode('utf-8')
+            _val251 = {}
+            (_ktype253, _vtype254, _size252 ) = iprot.readMapBegin() 
+            for _i256 in xrange(_size252):
+              _key257 = iprot.readString().decode('utf-8')
+              _val258 = iprot.readI64();
+              _val251[_key257] = _val258
             iprot.readMapEnd()
-            self.transferred[_key243] = _val244
+            self.transferred[_key250] = _val251
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3109,24 +3240,24 @@ class ExecutorStats:
     if self.emitted is not None:
       oprot.writeFieldBegin('emitted', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.emitted))
-      for kiter252,viter253 in self.emitted.items():
-        oprot.writeString(kiter252.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter253))
-        for kiter254,viter255 in viter253.items():
-          oprot.writeString(kiter254.encode('utf-8'))
-          oprot.writeI64(viter255)
+      for kiter259,viter260 in self.emitted.items():
+        oprot.writeString(kiter259.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter260))
+        for kiter261,viter262 in viter260.items():
+          oprot.writeString(kiter261.encode('utf-8'))
+          oprot.writeI64(viter262)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.transferred is not None:
       oprot.writeFieldBegin('transferred', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.transferred))
-      for kiter256,viter257 in self.transferred.items():
-        oprot.writeString(kiter256.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter257))
-        for kiter258,viter259 in viter257.items():
-          oprot.writeString(kiter258.encode('utf-8'))
-          oprot.writeI64(viter259)
+      for kiter263,viter264 in self.transferred.items():
+        oprot.writeString(kiter263.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter264))
+        for kiter265,viter266 in viter264.items():
+          oprot.writeString(kiter265.encode('utf-8'))
+          oprot.writeI64(viter266)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -3947,11 +4078,11 @@ class TopologyInfo:
       elif fid == 4:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype263, _size260) = iprot.readListBegin()
-          for _i264 in xrange(_size260):
-            _elem265 = ExecutorSummary()
-            _elem265.read(iprot)
-            self.executors.append(_elem265)
+          (_etype270, _size267) = iprot.readListBegin()
+          for _i271 in xrange(_size267):
+            _elem272 = ExecutorSummary()
+            _elem272.read(iprot)
+            self.executors.append(_elem272)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -3963,17 +4094,17 @@ class TopologyInfo:
       elif fid == 6:
         if ftype == TType.MAP:
           self.errors = {}
-          (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin() 
-          for _i270 in xrange(_size266):
-            _key271 = iprot.readString().decode('utf-8')
-            _val272 = []
-            (_etype276, _size273) = iprot.readListBegin()
-            for _i277 in xrange(_size273):
-              _elem278 = ErrorInfo()
-              _elem278.read(iprot)
-              _val272.append(_elem278)
+          (_ktype274, _vtype275, _size273 ) = iprot.readMapBegin() 
+          for _i277 in xrange(_size273):
+            _key278 = iprot.readString().decode('utf-8')
+            _val279 = []
+            (_etype283, _size280) = iprot.readListBegin()
+            for _i284 in xrange(_size280):
+              _elem285 = ErrorInfo()
+              _elem285.read(iprot)
+              _val279.append(_elem285)
             iprot.readListEnd()
-            self.errors[_key271] = _val272
+            self.errors[_key278] = _val279
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4017,8 +4148,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:
@@ -4028,11 +4159,11 @@ 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()
@@ -4186,11 +4317,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
+          (_ktype291, _vtype292, _size290 ) = iprot.readMapBegin() 
+          for _i294 in xrange(_size290):
+            _key295 = iprot.readString().decode('utf-8')
+            _val296 = iprot.readI32();
+            self.num_executors[_key295] = _val296
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4215,9 +4346,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 kiter297,viter298 in self.num_executors.items():
+        oprot.writeString(kiter297.encode('utf-8'))
+        oprot.writeI32(viter298)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4267,11 +4398,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
+          (_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
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4288,9 +4419,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 kiter306,viter307 in self.creds.items():
+        oprot.writeString(kiter306.encode('utf-8'))
+        oprot.writeString(viter307.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()


[44/50] [abbrv] storm git commit: STORM-726: Adding nimbus.host config for backward compatibility.

Posted by bo...@apache.org.
STORM-726: Adding nimbus.host config for backward compatibility.

Conflicts:
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/src/jvm/backtype/storm/Config.java
	storm-core/src/jvm/backtype/storm/utils/NimbusClient.java

Conflicts:
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/src/jvm/backtype/storm/Config.java
	storm-core/src/jvm/backtype/storm/utils/NimbusClient.java


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

Branch: refs/heads/master
Commit: 0395fd6878f574724686328c1039454ce6acd23d
Parents: 95fb680
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Mar 26 11:22:10 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 09:49:44 2015 -0700

----------------------------------------------------------------------
 conf/defaults.yaml                              |  2 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  9 ++++----
 storm-core/src/jvm/backtype/storm/Config.java   |  6 +++--
 .../jvm/backtype/storm/utils/NimbusClient.java  | 24 ++++++++++----------
 4 files changed, 21 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0395fd68/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index dd69eb6..cad5021 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -51,7 +51,7 @@ storm.meta.serialization.delegate: "backtype.storm.serialization.GzipThriftSeria
 storm.codedistributor.class: "backtype.storm.codedistributor.LocalFileSystemCodeDistributor"
 
 ### nimbus.* configs are for the master
-nimbus.seeds : ["localhost:6627"]
+nimbus.seeds : ["localhost"]
 nimbus.thrift.port: 6627
 nimbus.thrift.threads: 64
 nimbus.thrift.max_buffer_size: 1048576

http://git-wip-us.apache.org/repos/asf/storm/blob/0395fd68/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 40d030f..a872fd6 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -587,12 +587,12 @@
       (nimbus-summary
         (.get_nimbuses (.getClusterInfo ^Nimbus$Client nimbus)))))
   ([nimbuses]
-    (let [nimbus-seeds (set (*STORM-CONF* NIMBUS-SEEDS))
+    (let [nimbus-seeds (set (map #(str %1 ":" (*STORM-CONF* NIMBUS-THRIFT-PORT)) (set (*STORM-CONF* NIMBUS-SEEDS))))
           alive-nimbuses (set (map #(str (.get_host %1) ":" (.get_port %1)) nimbuses))
-          dead-nimbuses (clojure.set/difference nimbus-seeds alive-nimbuses)
-          dead-nimbuses-summary (map #(convert-to-nimbus-summary %1) dead-nimbuses)]
+          offline-nimbuses (clojure.set/difference nimbus-seeds alive-nimbuses)
+          offline-nimbuses-summary (map #(convert-to-nimbus-summary %1) offline-nimbuses)]
       {"nimbuses"
-       (concat dead-nimbuses-summary
+       (concat offline-nimbuses-summary
        (for [^NimbusSummary n nimbuses]
          {
           "host" (.get_host n)
@@ -1080,7 +1080,6 @@
       (catch Exception ex
         (json-response (exception->json ex) ((:query-params request) "callback") :status 500)))))
 
-
 (def app
   (handler/site (-> main-routes
                     (wrap-json-params)

http://git-wip-us.apache.org/repos/asf/storm/blob/0395fd68/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 36749ca..cef87f3 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -42,6 +42,7 @@ import java.util.Map;
  * Spouts.</p>
  */
 public class Config extends HashMap<String, Object> {
+
     //DO NOT CHANGE UNLESS WE ADD IN STATE NOT STORED IN THE PARENT CLASS
     private static final long serialVersionUID = -1550278723792864455L;
 
@@ -316,14 +317,15 @@ public class Config extends HashMap<String, Object> {
     public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
-     * The host that the master server is running on, only here for backward compatibility.
+     * The host that the master server is running on, added only for backward compatibility,
+     * the usage deprecated in favor of nimbus.seeds config.
      */
     @Deprecated
     public static final String NIMBUS_HOST = "nimbus.host";
     public static final Object NIMBUS_HOST_SCHEMA = String.class;
 
     /**
-     * List of seed nimbus hosts:port to use for leader nimbus discovery.
+     * List of seed nimbus hosts to use for leader nimbus discovery.
      */
     public static final String NIMBUS_SEEDS = "nimbus.seeds";
     public static final Object NIMBUS_SEEDS_SCHEMA = ConfigValidation.StringsValidator;

http://git-wip-us.apache.org/repos/asf/storm/blob/0395fd68/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index 323d2a8..a715de2 100644
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -56,23 +56,23 @@ public class NimbusClient extends ThriftClient {
             asUser = (String) conf.get(Config.STORM_DO_AS_USER);
         }
 
-        List<String> seeds = (List<String>) conf.get(Config.NIMBUS_SEEDS);
-
-        if(seeds == null  || seeds.isEmpty()) {
-            LOG.warn("config {} has no value. Failing over to deprecated config {}. ", Config.NIMBUS_SEEDS, Config.NIMBUS_HOST);
-            seeds = Lists.newArrayList(conf.get(Config.NIMBUS_HOST) + ":" + conf.get(Config.NIMBUS_THRIFT_PORT));
+        List<String> seeds = null;
+        if(conf.containsKey(Config.NIMBUS_HOST)) {
+            LOG.warn("Using deprecated config {} for backward compatibility. Please update your storm.yaml so it only has config {}",
+                    Config.NIMBUS_HOST, Config.NIMBUS_SEEDS);
+            seeds = Lists.newArrayList(conf.get(Config.NIMBUS_HOST).toString());
+        } else {
+            seeds = (List<String>) conf.get(Config.NIMBUS_SEEDS);
         }
 
-        for(String seed : seeds) {
-            String[] split = seed.split(DELIMITER);
-            String host = split[0];
-            int port = Integer.parseInt(split[1]);
+        for (String host : seeds) {
+            int port = Integer.parseInt(conf.get(Config.NIMBUS_THRIFT_PORT).toString());
             ClusterSummary clusterInfo = null;
             try {
                 NimbusClient client = new NimbusClient(conf, host, port);
                 clusterInfo = client.getClient().getClusterInfo();
             } catch (Exception e) {
-                LOG.warn("Ignoring exception while trying to get leader nimbus info from " + seed
+                LOG.warn("Ignoring exception while trying to get leader nimbus info from " + host
                         + ". will retry with a different seed host.", e);
                 continue;
             }
@@ -83,7 +83,7 @@ public class NimbusClient extends ThriftClient {
                         try {
                             return new NimbusClient(conf, nimbusSummary.get_host(), nimbusSummary.get_port(), null, asUser);
                         } catch (TTransportException e) {
-                            String leaderNimbus = nimbusSummary.get_host() + DELIMITER + nimbusSummary.get_port();
+                            String leaderNimbus = nimbusSummary.get_host() + ":" + nimbusSummary.get_port();
                             throw new RuntimeException("Failed to create a nimbus client for the leader " + leaderNimbus, e);
                         }
                     }
@@ -93,7 +93,7 @@ public class NimbusClient extends ThriftClient {
             }
         }
         throw new RuntimeException("Could not find leader nimbus from seed hosts " + seeds + ". " +
-                "Did you specify a valid list of nimbus host:port for config " + Config.NIMBUS_SEEDS);
+                "Did you specify a valid list of nimbus hosts for config " + Config.NIMBUS_SEEDS);
     }
 
     public NimbusClient(Map conf, String host, int port) throws TTransportException {


[30/50] [abbrv] storm git commit: removing read-storm-version function, fixing a typo in ui.

Posted by bo...@apache.org.
removing read-storm-version function, fixing a typo in ui.


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

Branch: refs/heads/master
Commit: a8aacca6444a30638162ac27f8c7213a4a96aed1
Parents: 23ed537
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 16:44:52 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 16:44:52 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/config.clj              | 10 ----------
 .../src/ui/public/templates/index-page-template.html      |  4 ++--
 2 files changed, 2 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a8aacca6/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index f3c70e5..a6b160d 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -282,13 +282,3 @@
 (defn ^LocalState worker-state
   [conf id]
   (LocalState. (worker-heartbeats-root conf id)))
-
-(defn read-storm-version
-  "Returns a string containing the Storm version or 'Unknown'."
-  []
-  (let [storm-home (System/getProperty "storm.home")
-        release-path (format "%s/RELEASE" storm-home)
-        release-file (File. release-path)]
-    (if (and (.exists release-file) (.isFile release-file))
-      (str/trim (slurp release-path))
-      "Unknown")))

http://git-wip-us.apache.org/repos/asf/storm/blob/a8aacca6/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index eb4581a..84da576 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -83,10 +83,10 @@
                 <span class="tip above" title="True if this host is leader, false otherwise.">IsLeader</span>
             </th>
             <th>
-                <span class="tip left" title="Storm version that this nimbus host is running.">Version</span>
+                <span class="tip left" title="Storm version this nimbus host is running.">Version</span>
             </th>
             <th>
-                <span class="tip left" title="Number of seconds this nimbus host has been running the nimbus process.">UpTime Seconds</span>
+                <span class="tip left" title="Time since this nimbus host has been running.">UpTime Seconds</span>
             </th>
         </tr>
         </thead>


[04/50] [abbrv] storm git commit: STORM-166: reformatting some sections.

Posted by bo...@apache.org.
STORM-166: reformatting some sections.


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

Branch: refs/heads/master
Commit: e05d6663a3dbc056e9f21194f83fe0e27509f83e
Parents: 07b69b7
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 13:01:27 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 13:01:27 2014 -0800

----------------------------------------------------------------------
 docs/documentation/nimbus-ha-design.md | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e05d6663/docs/documentation/nimbus-ha-design.md
----------------------------------------------------------------------
diff --git a/docs/documentation/nimbus-ha-design.md b/docs/documentation/nimbus-ha-design.md
index 51b15f8..cb6332f 100644
--- a/docs/documentation/nimbus-ha-design.md
+++ b/docs/documentation/nimbus-ha-design.md
@@ -13,9 +13,8 @@ of the backups will take over.
 the list of potential leaders automatically. 
 * No topology resubmissions required in case of nimbus fail overs.
 * No active topology should ever be lost. 
-##Components:
-Following are different components to achieve the above goals.
-###Leader Election:
+
+##Leader Election:
 The nimbus server will use the following interface:
 
 ```java
@@ -81,7 +80,7 @@ This sequence diagram describes how leader election and failover would work with
 
 ![Nimbus Fail Over](images/nimbus_ha_leader_election_and_failover.png)
 
-###Nimbus state store:
+##Nimbus state store:
 
 Currently the nimbus stores 2 kind of data
 * Meta information like supervisor info, assignment info which is stored in zookeeper
@@ -93,9 +92,9 @@ and it is hard to test for correctness.However many storm users do not want to t
 storage system like HDFS and still need high availability.Eventually, we want to move to the bittorrent protocol for code 
 distribution given the size of the jars and to achieve better scaling when the total number of supervisors is very high. 
 The current file system based model for code distribution works fine with systems that have file system like structure
-but it fails to support a non file system based approach like bit torrent. To support bit torrent we can go with the 
-following interface instead of the storage interface described above. The interface described below can still be used with
-HDFS,S3 and local file system, so this is a more extensible interface. 
+but it fails to support a non file system based approach like bit torrent. To support bit torrent and all the file
+system based replicated storage systems we propose the following interface:
+
 ```java
 /**
  * Interface responsible to distribute code in the cluster.


[16/50] [abbrv] storm git commit: Merge remote-tracking branch 'upstream/master' into STORM-166

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into STORM-166

Conflicts:
	storm-core/src/jvm/backtype/storm/Config.java


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

Branch: refs/heads/master
Commit: 58667be951623848406f699f82cafa79a40cb496
Parents: 5658bbb e71e2a3
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Jan 8 11:59:22 2015 -0500
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Jan 8 11:59:22 2015 -0500

----------------------------------------------------------------------
 CHANGELOG.md                                    |  14 +
 DEVELOPER.md                                    |   5 +-
 README.markdown                                 |  18 +-
 SECURITY.md                                     |  25 +-
 bin/storm                                       |  59 ++--
 conf/defaults.yaml                              |   1 +
 docs/about/integrates.md                        |   2 +-
 docs/about/multi-language.md                    |   6 +-
 docs/about/simple-api.md                        |   2 +-
 .../Acking-framework-implementation.md          |   8 +-
 docs/documentation/Clojure-DSL.md               |   6 +-
 docs/documentation/Common-patterns.md           |   2 +-
 docs/documentation/Configuration.md             |   4 +-
 .../Creating-a-new-Storm-project.md             |   6 +-
 .../DSLs-and-multilang-adapters.md              |   3 +-
 ...Defining-a-non-jvm-language-dsl-for-storm.md |   2 +-
 docs/documentation/Distributed-RPC.md           |   2 +-
 docs/documentation/FAQ.md                       |   4 +-
 docs/documentation/Kestrel-and-Storm.md         |   2 +-
 docs/documentation/Lifecycle-of-a-topology.md   |  70 ++---
 docs/documentation/Maven.md                     |  50 +---
 docs/documentation/Multilang-protocol.md        |  30 +-
 .../Serialization-(prior-to-0.6.0).md           |   2 +-
 .../documentation/Setting-up-a-Storm-cluster.md |   4 +-
 .../Setting-up-development-environment.md       |   2 +-
 docs/documentation/Structure-of-the-codebase.md |  88 +++---
 docs/documentation/Transactional-topologies.md  |  12 +-
 docs/documentation/Trident-API-Overview.md      |   4 +-
 docs/documentation/Trident-spouts.md            |   8 +-
 docs/documentation/Trident-state.md             |  12 +-
 docs/documentation/Tutorial.md                  |   4 +-
 docs/downloads.html                             |   2 +-
 external/storm-hbase/README.md                  |  49 +++-
 external/storm-hbase/pom.xml                    |  17 +-
 .../storm/hbase/bolt/AbstractHBaseBolt.java     |   9 +-
 .../apache/storm/hbase/security/AutoHBase.java  | 243 ++++++++++++++++
 .../storm/hbase/security/HBaseSecurityUtil.java |  32 ++-
 .../storm/hbase/trident/state/HBaseState.java   |   9 +-
 external/storm-hdfs/README.md                   |  45 +++
 .../storm/hdfs/bolt/format/SequenceFormat.java  |   5 +-
 .../storm/hdfs/common/security/AutoHDFS.java    | 281 +++++++++++++++++++
 .../hdfs/common/security/HdfsSecurityUtil.java  |  30 +-
 .../src/jvm/storm/kafka/KafkaUtils.java         |  12 +-
 .../src/jvm/storm/kafka/PartitionManager.java   |   6 +-
 .../kafka/TopicOffsetOutOfRangeException.java   |  25 ++
 .../jvm/storm/kafka/UpdateOffsetException.java  |  22 --
 .../FieldNameBasedTupleToKafkaMapper.java       |   2 +-
 .../kafka/trident/TridentKafkaEmitter.java      |  43 ++-
 .../src/test/storm/kafka/KafkaUtilsTest.java    |   2 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |  36 ++-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   7 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  22 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |   1 +
 storm-core/src/jvm/backtype/storm/Config.java   |  38 +--
 .../storm/security/auth/hadoop/AutoHDFS.java    | 262 -----------------
 .../jvm/backtype/storm/utils/ShellProcess.java  |  11 +-
 .../src/native/worker-launcher/configure.ac     |   2 +-
 .../worker-launcher/impl/worker-launcher.c      |   7 +-
 .../test/clj/backtype/storm/logviewer_test.clj  |  47 +++-
 .../backtype/storm/security/auth/auth_test.clj  |   8 +-
 .../test/clj/backtype/storm/supervisor_test.clj |   6 +-
 61 files changed, 1108 insertions(+), 630 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/storm/blob/58667be9/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index c96dd88,9d2f313..3fcf7eb
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@@ -296,9 -294,12 +296,13 @@@
                                 (log-error t "Error when processing event")
                                 (exit-process! 20 "Error when processing an event")
                                 ))
+    :event-timer (mk-timer :kill-fn (fn [t]
+                                          (log-error t "Error when processing event")
+                                          (exit-process! 20 "Error when processing an event")
+                                          ))
     :assignment-versions (atom {})
     :sync-retry (atom 0)
 +   :bt-tracker (mk-bt-tracker conf)
     })
  
  (defn sync-processes [supervisor]

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

http://git-wip-us.apache.org/repos/asf/storm/blob/58667be9/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/Config.java
index 4678177,3a241ba..df330a0
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@@ -1234,54 -1246,6 +1240,36 @@@ public class Config extends HashMap<Str
      public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
      public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class;
  
 +    /**
-      * HDFS information, used to get the delegation token on behalf of the topology
-      * submitter user and renew the tokens. see {@link backtype.storm.security.auth.hadoop.AutoHDFS}
-      * kerberos principal name with realm should be provided.
-      */
-     public static final Object TOPOLOGY_HDFS_PRINCIPAL = "topology.hdfs.user";
-     public static final Object TOPOLOGY_HDFS_PRINCIPAL_SCHEMA = String.class;
- 
-     /**
-      * The HDFS URI to be used by AutoHDFS.java to grab the delegation token on topology
-      * submitter user's behalf by the nimbus. If this is not provided the default URI provided
-      * in the hdfs configuration files will be used.
-      */
-     public static final Object TOPOLOGY_HDFS_URI = "topology.hdfs.uri";
-     public static final Object TOPOLOGY_HDFS_URI_SCHEMA = String.class;
- 
-     /**
 +     * Which implementation of {@link backtype.storm.codedistributor.ICodeDistributor} should be used by storm for code
 +     * distribution.
 +     */
 +    public static final String STORM_CODE_DISTRIBUTOR_CLASS = "storm.codedistributor.class";
 +    public static final Object STORM_CODE_DISTRIBUTOR_CLASS_SCHEMA = String.class;
 +
 +    /**
 +     * Minimum number of nimbus hosts where the code must be replicated before leader nimbus
 +     * is allowed to perform topology activation tasks like setting up heartbeats/assignments
 +     * and marking the topology as active. default is 0.
 +     */
 +    public static final String NIMBUS_MIN_REPLICATION_COUNT = "nimbus.min.replication.count";
 +    public static final Object NIMBUS_MIN_REPLICATION_COUNT_SCHEMA = Number.class;
 +
 +    /**
 +     * Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
 +     * Once this time is elapsed nimbus will go ahead and perform topology activation tasks even
 +     * if required nimbus.min.replication.count is not achieved. The default is 0 seconds, a value of
 +     * -1 indicates to wait for ever.
 +     */
 +    public static final String NIMBUS_MAX_REPLICATION_WAIT_TIME_SEC = "nimbus.max.replication.wait.time.sec";
 +    public static final Object NIMBUS_MAX_REPLICATION_WAIT_TIME_SEC_SCHEMA = Number.class;
 +
 +    /**
-      * How often nimbus should wake the cleanup thread to clean the inbox.
-      * @see NIMBUS_INBOX_JAR_EXPIRATION_SECS
++     * How often nimbus's background thread to sync code for missing topologies should run.
 +     */
 +    public static final String NIMBUS_CODE_SYNC_FREQ_SECS = "nimbus.code.sync.freq.secs";
 +    public static final Object NIMBUS_CODE_SYNC_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 +
- 
      public static void setClasspath(Map conf, String cp) {
          conf.put(Config.TOPOLOGY_CLASSPATH, cp);
      }

http://git-wip-us.apache.org/repos/asf/storm/blob/58667be9/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/58667be9/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------


[23/50] [abbrv] storm git commit: Fixing a broken test.

Posted by bo...@apache.org.
Fixing a broken test.


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

Branch: refs/heads/master
Commit: aa24375258626ed6cb3db5257ebc6b7708558bb0
Parents: 4502bff
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 12:01:52 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 12:01:52 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/daemon/nimbus.clj | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/aa243752/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 52ee708..2059c0d 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1327,7 +1327,9 @@
                                                             (extract-status-str base))]
                                                (when-let [owner (:owner base)] (.set_owner topo-summ owner))
                                                (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
-                                               (.set_replication_count topo-summ (.getReplicationCount (:code-distributor nimbus) id))
+                                               (.set_replication_count topo-summ (if (:code-distributor nimbus)
+                                                                                   (.getReplicationCount (:code-distributor nimbus) id)
+                                                                                   1))
                                                topo-summ
                                           ))]
           (ClusterSummary. supervisor-summaries


[39/50] [abbrv] storm git commit: Merge remote-tracking branch 'apache/nimbus-ha-branch' into ha-merge

Posted by bo...@apache.org.
Merge remote-tracking branch 'apache/nimbus-ha-branch' into ha-merge

Conflicts:
	STORM-UI-REST-API.md
	conf/defaults.yaml
	storm-core/src/clj/backtype/storm/daemon/nimbus.clj
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/src/jvm/backtype/storm/Config.java
	storm-core/src/jvm/backtype/storm/generated/TopologySummary.java


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

Branch: refs/heads/master
Commit: d1afefde51e34dc993591ad79d3fe217bef86f87
Parents: c54cea1 765e4c2
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Tue Aug 11 22:31:02 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Tue Aug 11 22:31:02 2015 -0700

----------------------------------------------------------------------
 STORM-UI-REST-API.md                            |  40 +-
 conf/defaults.yaml                              |   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 +++++
 .../ha/codedistributor/HDFSCodeDistributor.java | 101 +++
 pom.xml                                         |  16 +
 storm-core/pom.xml                              |  16 +
 storm-core/src/clj/backtype/storm/cluster.clj   |  57 +-
 .../backtype/storm/command/shell_submission.clj |   9 +-
 storm-core/src/clj/backtype/storm/config.clj    |  15 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 243 ++++--
 .../clj/backtype/storm/daemon/supervisor.clj    |  53 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |  23 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  72 +-
 storm-core/src/clj/backtype/storm/zookeeper.clj |  94 ++-
 storm-core/src/jvm/backtype/storm/Config.java   |  42 +-
 .../storm/codedistributor/ICodeDistributor.java |  56 ++
 .../LocalFileSystemCodeDistributor.java         | 106 +++
 .../storm/generated/ClusterSummary.java         | 292 ++++---
 .../backtype/storm/generated/NimbusSummary.java | 796 +++++++++++++++++++
 .../backtype/storm/generated/TopologyInfo.java  | 221 +++--
 .../storm/generated/TopologySummary.java        | 107 ++-
 .../backtype/storm/nimbus/ILeaderElector.java   |  60 ++
 .../jvm/backtype/storm/nimbus/NimbusInfo.java   |  93 +++
 .../jvm/backtype/storm/utils/NimbusClient.java  |  63 +-
 .../src/jvm/backtype/storm/utils/Utils.java     |   9 +
 storm-core/src/py/storm/ttypes.py               | 613 ++++++++------
 storm-core/src/storm.thrift                     |  12 +-
 storm-core/src/ui/public/index.html             |  21 +
 .../public/templates/index-page-template.html   |  58 +-
 .../templates/topology-page-template.html       |   6 +
 .../test/clj/backtype/storm/cluster_test.clj    |  23 +-
 .../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 -
 38 files changed, 3124 insertions(+), 658 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --cc STORM-UI-REST-API.md
index 2836105,baaca84..35ba6ed
--- a/STORM-UI-REST-API.md
+++ b/STORM-UI-REST-API.md
@@@ -231,7 -263,13 +263,8 @@@ Response fields
  |bolts.errorLapsedSecs| Integer |Number of seconds elapsed since that last error happened in a bolt|
  |bolts.errorWorkerLogLink| String | Link to the worker log that reported the exception |
  |bolts.emitted| Long |Number of tuples emitted|
 -|antiForgeryToken| String | CSRF token|
+ |replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
  
 -Caution: users need to unescape the antiForgeryToken value before using this token to make POST calls(simple-json escapes forward slashes)
 -[ISSUE-8](https://code.google.com/p/json-simple/issues/detail?id=8)
 -
 -
  Examples:
  
  ```no-highlight
@@@ -375,7 -413,9 +408,8 @@@ Sample response
          "storm.zookeeper.retry.intervalceiling.millis": 30000,
          "supervisor.enable": true,
          "storm.messaging.netty.server_worker_threads": 1
-     }
+     },
 -    "antiForgeryToken": "lAFTN\/5iSedRLwJeUNqkJ8hgYubRl2OxjXGoDf9A4Bt1nZY3rvJW0\/P4zqu9yAk\/LvDhlmn7gigw\/z8C",
+     "replicationCount": 1
  }
  ```
  

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/conf/defaults.yaml
----------------------------------------------------------------------
diff --cc conf/defaults.yaml
index c3fa372,49584f2..dd69eb6
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@@ -47,10 -47,12 +47,11 @@@ storm.auth.simple-white-list.users: [
  storm.auth.simple-acl.users: []
  storm.auth.simple-acl.users.commands: []
  storm.auth.simple-acl.admins: []
 -storm.meta.serialization.delegate: "backtype.storm.serialization.DefaultSerializationDelegate"
 +storm.meta.serialization.delegate: "backtype.storm.serialization.GzipThriftSerializationDelegate"
+ storm.codedistributor.class: "backtype.storm.codedistributor.LocalFileSystemCodeDistributor"
 -storm.meta.serialization.delegate: "backtype.storm.serialization.ThriftSerializationDelegate"
  
  ### nimbus.* configs are for the master
- nimbus.host: "localhost"
+ nimbus.seeds : ["localhost:6627"]
  nimbus.thrift.port: 6627
  nimbus.thrift.threads: 64
  nimbus.thrift.max_buffer_size: 1048576

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/cluster.clj
index 63e385f,333feec..f75648a
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@@ -284,11 -311,9 +311,12 @@@
                        (condp = subtree
                           ASSIGNMENTS-ROOT (if (empty? args)
                                               (issue-callback! assignments-callback)
 -                                             (issue-map-callback! assignment-info-callback (first args)))
 +                                             (do
 +                                               (issue-map-callback! assignment-info-callback (first args))
 +                                               (issue-map-callback! assignment-version-callback (first args))
 +                                               (issue-map-callback! assignment-info-with-version-callback (first args))))
                           SUPERVISORS-ROOT (issue-callback! supervisors-callback)
+                          CODE-DISTRIBUTOR-ROOT (issue-callback! code-distributor-callback)
                           STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
                           CREDENTIALS-ROOT (issue-map-callback! credentials-callback (first args))
                           ;; this should never happen

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

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index c88e36b,8a2c0fb..35154d3
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@@ -327,9 -341,32 +345,32 @@@
     (FileUtils/cleanDirectory (File. stormroot))
     (setup-jar conf tmp-jar-location stormroot)
     (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
 -   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/javaSerialize storm-conf))
 +   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/toCompressedJsonConf storm-conf))
+    (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
     ))
  
+ (defn- wait-for-desired-code-replication [nimbus conf storm-id]
+   (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
+         max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
+         total-wait-time (atom 0)
+         current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
+   (if (:code-distributor nimbus)
+     (while (and (> min-replication-count @current-replication-count)
+              (or (= -1 max-replication-wait-time)
+                (< @total-wait-time max-replication-wait-time)))
+         (sleep-secs 1)
+         (log-debug "waiting for desired replication to be achieved.
+           min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
+           "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
+         (swap! total-wait-time inc)
+         (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
+   (if (< min-replication-count @current-replication-count)
+     (log-message "desired replication count "  min-replication-count " achieved,
+       current-replication-count" @current-replication-count)
+     (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time
+       so moving on with replication count = " @current-replication-count)
+     )))
+ 
  (defn- read-storm-topology [conf storm-id]
    (let [stormroot (master-stormdist-root conf storm-id)]
      (Utils/deserialize
@@@ -1258,12 -1323,18 +1330,20 @@@
                                                                  (:uptime-secs info)
                                                                  (count ports)
                                                                  (count (:used-ports info))
 -                                                                id )
 +                                                                id) ]
 +                                            (when-let [version (:version info)] (.set_version sup-sum version))
 +                                            sup-sum
                                              ))
-               nimbus-uptime ((:uptime nimbus))
                bases (topology-bases storm-cluster-state)
+               nimbuses (.nimbuses storm-cluster-state)
+ 
+               ;;update the isLeader field for each nimbus summary
+               _ (let [leader (.getLeader (:leader-elector nimbus))
+                       leader-host (.getHost leader)
+                       leader-port (.getPort leader)]
+                   (doseq [nimbus-summary nimbuses]
+                     (.set_isLeader nimbus-summary (and (= leader-host (.get_host nimbus-summary)) (= leader-port (.get_port nimbus-summary))))))
+ 
                topology-summaries (dofor [[id base] bases :when base]
  	                                  (let [assignment (.assignment-info storm-cluster-state id nil)
                                                  topo-summ (TopologySummary. id

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index be8f682,4fc219e..5f819bd
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@@ -37,7 -37,11 +37,8 @@@
  
  (defmulti download-storm-code cluster-mode)
  (defmulti launch-worker (fn [supervisor & _] (cluster-mode (:conf supervisor))))
+ (defmulti mk-code-distributor cluster-mode)
  
 -;; used as part of a map from port to this
 -(defrecord LocalAssignment [storm-id executors])
 -
  (defprotocol SupervisorDaemon
    (get-id [this])
    (get-conf [this])

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index b0e266c,02c3d90..950b88d
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -25,8 -25,10 +25,9 @@@
    (: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 [ring.middleware.anti-forgery])
    (: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
              ErrorInfo ClusterSummary SupervisorSummary TopologySummary
@@@ -293,18 -287,14 +286,21 @@@
                (bolt-comp-summs id))]
      (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret)))
  
 -(defn worker-log-link [host port topology-id]
 +(defn worker-log-link [host port topology-id secure?]
    (let [fname (logs-filename topology-id port)]
 -    (url-format (str "http://%s:%s/log?file=%s")
 -          host (*STORM-CONF* LOGVIEWER-PORT) fname)))
 +    (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 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)
@@@ -705,8 -711,8 +718,8 @@@
          "acked" (get-in stats [:acked k])
          "failed" (get-in stats [:failed k])})))
  
 -(defn topology-page [id window include-sys? user]
 +(defn topology-page [id window include-sys? user secure?]
-   (with-nimbus nimbus
+   (thrift/with-configured-nimbus-connection nimbus
      (let [window (if window window ":all-time")
            window-hint (window-hint window)
            summ (->> (doto
@@@ -738,10 -745,12 +752,11 @@@
          "windowHint" window-hint
          "msgTimeout" msg-timeout
          "topologyStats" (topology-stats id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
 -        "spouts" (spout-comp id spout-comp-summs (.get_errors summ) window include-sys?)
 -        "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys?)
 +        "spouts" (spout-comp id spout-comp-summs (.get_errors summ) window include-sys? secure?)
 +        "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys? secure?)
          "configuration" topology-conf
-         "visualizationTable" (stream-boxes visualizer-data)}))))
+         "visualizationTable" (stream-boxes visualizer-data)
 -        "antiForgeryToken" *anti-forgery-token*
+         "replicationCount" replication-count}))))
  
  (defn spout-output-stats
    [stream-summary window]
@@@ -885,11 -894,11 +900,11 @@@
       "inputStats" (bolt-input-stats stream-summary window)
       "outputStats" (bolt-output-stats stream-summary window)
       "executorStats" (bolt-executor-stats
 -                       (.get_id topology-info) executors window include-sys?)}))
 +                       (.get_id topology-info) executors window include-sys? secure?)}))
  
  (defn component-page
 -  [topology-id component window include-sys? user]
 +  [topology-id component window include-sys? user secure?]
-   (with-nimbus nimbus
+   (thrift/with-configured-nimbus-connection nimbus
      (let [window (if window window ":all-time")
            summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
            topology (.getTopology ^Nimbus$Client nimbus topology-id)
@@@ -935,7 -944,7 +950,7 @@@
       {:status status
        :headers (merge {"Cache-Control" "no-cache, no-store"
                         "Access-Control-Allow-Origin" "*"
--                       "Access-Control-Allow-Headers" "Content-Type, Access-Control-Allow-Headers, Access-Controler-Allow-Origin, X-Requested-By, X-Csrf-Token, Authorization, X-Requested-With"}
++                       "Access-Control-Allow-Headers" "Content-Type, Access-Control-Allow-Headers, Access-Controler-Allow-Origin, X-Requested-By, Authorization, X-Requested-With"}
                        (if (not-nil? callback) {"Content-Type" "application/javascript;charset=utf-8"}
                            {"Content-Type" "application/json;charset=utf-8"}))
        :body (if (not-nil? callback)
@@@ -965,15 -977,15 +983,15 @@@
    (GET "/api/v1/topology/:id/visualization" [:as {:keys [cookies servlet-request]} id & m]
          (assert-authorized-user servlet-request "getTopology" (topology-config id))
          (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m))) (:callback m)))
 -  (GET "/api/v1/topology/:id/component/:component" [:as {:keys [cookies servlet-request]} id component & m]
 +  (GET "/api/v1/topology/:id/component/:component" [:as {:keys [cookies servlet-request scheme]} id component & m]
         (let [user (.getUserName http-creds-handler servlet-request)]
           (assert-authorized-user servlet-request "getTopology" (topology-config id))
 -         (json-response (component-page id component (:window m) (check-include-sys? (:sys m)) user) (:callback m))))
 -  (GET "/api/v1/token" [ & m]
 -       (json-response (format "{\"antiForgeryToken\": \"%s\"}" *anti-forgery-token*) (:callback m) :serialize-fn identity))
 +         (json-response
 +          (component-page id component (:window m) (check-include-sys? (:sys m)) user (= scheme :https))
 +          (:callback m))))
    (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id & m]
+     (thrift/with-configured-nimbus-connection nimbus
      (assert-authorized-user servlet-request "activate" (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/d1afefde/storm-core/src/clj/backtype/storm/zookeeper.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/jvm/backtype/storm/Config.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/Config.java
index 4628bd4,bd145d5..3cba37c
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@@ -1454,12 -1293,35 +1454,42 @@@ public class Config extends HashMap<Str
      public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class;
  
      /**
 +     * Configure timeout milliseconds used for disruptor queue wait strategy. Can be used to tradeoff latency
 +     * vs. CPU usage
 +     */
 +    public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis";
 +    public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator;
 +
++    /**
+      * Which implementation of {@link backtype.storm.codedistributor.ICodeDistributor} should be used by storm for code
+      * distribution.
+      */
+     public static final String STORM_CODE_DISTRIBUTOR_CLASS = "storm.codedistributor.class";
+     public static final Object STORM_CODE_DISTRIBUTOR_CLASS_SCHEMA = String.class;
+ 
+     /**
+      * Minimum number of nimbus hosts where the code must be replicated before leader nimbus
+      * is allowed to perform topology activation tasks like setting up heartbeats/assignments
+      * and marking the topology as active. default is 0.
+      */
+     public static final String TOPOLOGY_MIN_REPLICATION_COUNT = "topology.min.replication.count";
+     public static final Object TOPOLOGY_MIN_REPLICATION_COUNT_SCHEMA = Number.class;
+ 
+     /**
+      * Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
+      * Once this time is elapsed nimbus will go ahead and perform topology activation tasks even
+      * if required nimbus.min.replication.count is not achieved. The default is 0 seconds, a value of
+      * -1 indicates to wait for ever.
+      */
 -    public static final String TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC = "nimbus.max.replication.wait.time.sec";
++    public static final String TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC = "topology.max.replication.wait.time.sec";
+     public static final Object TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC_SCHEMA = Number.class;
+ 
+     /**
+      * How often nimbus's background thread to sync code for missing topologies should run.
+      */
+     public static final String NIMBUS_CODE_SYNC_FREQ_SECS = "nimbus.code.sync.freq.secs";
+     public static final Object NIMBUS_CODE_SYNC_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator;
+ 
      public static void setClasspath(Map conf, String cp) {
          conf.put(Config.TOPOLOGY_CLASSPATH, cp);
      }

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --cc storm-core/src/storm.thrift
index a4b0b2a,839f6da..a585924
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@@ -153,13 -154,20 +154,21 @@@ struct SupervisorSummary 
    3: required i32 num_workers;
    4: required i32 num_used_workers;
    5: required string supervisor_id;
 +  6: optional string version = "VERSION_NOT_PROVIDED";
  }
  
+ struct NimbusSummary {
+   1: required string host;
+   2: required i32 port;
+   3: required i32 uptime_secs;
+   4: required bool isLeader;
+   5: required string version;
+ }
+ 
  struct ClusterSummary {
    1: required list<SupervisorSummary> supervisors;
-   2: required i32 nimbus_uptime_secs;
    3: required list<TopologySummary> topologies;
+   4: required list<NimbusSummary> nimbuses;
  }
  
  struct ErrorInfo {

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/ui/public/index.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/nimbus_test.clj
index 00fb6d6,057dd30..cbd88c4
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@@ -1180,32 -1277,3 +1277,33 @@@
          (is (thrown-cause? InvalidTopologyException
            (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)))
++        (sleep-secs 1)
 +        (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)
 +        ))))

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/d1afefde/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------


[45/50] [abbrv] storm git commit: Nimbus-HAL: Bug fix, install watch on code-distrbutor everytime sync-code is called.

Posted by bo...@apache.org.
Nimbus-HAL: Bug fix, install watch on code-distrbutor everytime sync-code is called.


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

Branch: refs/heads/master
Commit: 21ba9c1c9f73c7a84ac8d221b6ca6a7ca7639d14
Parents: 0395fd6
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed May 27 15:30:10 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 09:55:00 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/daemon/nimbus.clj | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/21ba9c1c/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 9aa098a..4f516ce 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -367,8 +367,8 @@
   (if (< min-replication-count @current-replication-count)
     (log-message "desired replication count "  min-replication-count " achieved,
       current-replication-count" @current-replication-count)
-    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time
-      so moving on with replication count = " @current-replication-count)
+    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time "
+      max-replication-wait-time " so moving on with replication count = " @current-replication-count)
     )))
 
 (defn- read-storm-topology [conf storm-id]
@@ -1485,7 +1485,7 @@
 (defmethod sync-code :distributed [conf nimbus]
   (let [storm-cluster-state (:storm-cluster-state nimbus)
         code-ids (set (code-ids (:conf nimbus)))
-        active-topologies (set (.code-distributor storm-cluster-state nil))
+        active-topologies (set (.code-distributor storm-cluster-state (fn [] (sync-code conf nimbus))))
         missing-topologies (set/difference active-topologies code-ids)]
     (if (not (empty? missing-topologies))
       (do
@@ -1499,8 +1499,8 @@
                 (try
                   (download-code conf nimbus missing (.getHost nimbus-host-port) (.getPort nimbus-host-port))
                   (catch Exception e (log-error e "Exception while trying to syn-code for missing topology" missing)))))))))
-    (.addToLeaderLockQueue (:leader-elector nimbus))
-    (log-message "local disk is completely in sync with zk code-distributor.")))
+    ;;TODO Ideally This should only be called if all missing topology code was successfully downloaded.
+    (.addToLeaderLockQueue (:leader-elector nimbus))))
 
 (defmethod sync-code :local [conf nimbus]
   nil)


[05/50] [abbrv] storm git commit: STORM-166 adding cluster_test for code-distributor.

Posted by bo...@apache.org.
STORM-166 adding cluster_test for code-distributor.


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

Branch: refs/heads/master
Commit: 9cd52c88ec67c1dd4a01370a835a0861dcee2c25
Parents: e05d666
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 13:53:34 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 13:53:34 2014 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj       |  3 +++
 storm-core/test/clj/backtype/storm/cluster_test.clj | 14 +++++++++++++-
 2 files changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9cd52c88/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 6078852..2c58510 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -142,7 +142,9 @@
   (assignment-info [this storm-id callback])
   (assignment-info-with-version [this storm-id callback])
   (assignment-version [this storm-id callback])
+  ;returns topologyIds under /stormroot/code-distributor
   (code-distributor [this callback])
+  ;returns lits of nimbusinfos under /stormroot/code-distributor/storm-id
   (code-distributor-info [this storm-id])
   (active-storms [this])
   (storm-base [this storm-id callback])
@@ -162,6 +164,7 @@
   (update-storm! [this storm-id new-elems])
   (remove-storm-base! [this storm-id])
   (set-assignment! [this storm-id info])
+  ;adds nimbusinfo under /stormroot/code-distributor/storm-id
   (setup-code-distributor! [this storm-id info])
   (remove-storm! [this storm-id])
   (report-error [this storm-id task-id node port error])

http://git-wip-us.apache.org/repos/asf/storm/blob/9cd52c88/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 7ed1028..b7630b1 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -14,7 +14,8 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.cluster-test
-  (:import [java.util Arrays])
+  (:import [java.util Arrays]
+           [backtype.storm.nimbus NimbusInfo])
   (:import [backtype.storm.daemon.common Assignment StormBase SupervisorInfo])
   (:import [org.apache.zookeeper ZooDefs ZooDefs$Ids])
   (:import [org.mockito Mockito])
@@ -170,6 +171,8 @@
     (let [state (mk-storm-state zk-port)
           assignment1 (Assignment. "/aaa" {} {1 [2 2002 1]} {})
           assignment2 (Assignment. "/aaa" {} {1 [2 2002]} {})
+          nimbusInfo1 (NimbusInfo. "nimbus1" 6667 false)
+          nimbusInfo2 (NimbusInfo. "nimbus2" 6667 false)
           base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "")
           base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "")]
       (is (= [] (.assignments state nil)))
@@ -201,6 +204,15 @@
       (.set-credentials! state "storm1" {"b" "b"} {})
       (is (= {"b" "b"} (.credentials state "storm1" nil)))
 
+      (is (= [] (.code-distributor state nil)))
+      (.setup-code-distributor! state "storm1" nimbusInfo1)
+      (is (= ["storm1"] (.code-distributor state nil)))
+      (is (= [nimbusInfo1] (.code-distributor-info state "storm1")))
+      (.setup-code-distributor! state "storm1" nimbusInfo2)
+      (is (= #{nimbusInfo1 nimbusInfo2} (set (.code-distributor-info state "storm1"))))
+      (.remove-storm! state "storm1")
+      (is (= [] (.code-distributor state nil)))
+
       ;; TODO add tests for task info and task heartbeat setting and getting
       (.disconnect state)
       )))


[50/50] [abbrv] storm git commit: Added STORM-166 to Changlog

Posted by bo...@apache.org.
Added STORM-166 to Changlog


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

Branch: refs/heads/master
Commit: d4db410548aab93eec6ba745c10c64d1b0850b9a
Parents: 6b3cb75
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Aug 24 08:51:04 2015 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Aug 24 08:51:04 2015 -0500

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


http://git-wip-us.apache.org/repos/asf/storm/blob/d4db4105/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 90cfde1..1563869 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-166: Nimbus HA
  * STORM-976: Config storm.logback.conf.dir is specific to previous logging framework
  * STORM-995: Fix excessive logging
  * STORM-837: HdfsState ignores commits


[08/50] [abbrv] storm git commit: STORM-166: Leveraging NimbusInfo.parse

Posted by bo...@apache.org.
STORM-166: Leveraging NimbusInfo.parse


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

Branch: refs/heads/master
Commit: a92a1e9c7887383286295d90d53899dbdbdde000
Parents: 2954eae
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 13:56:25 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 13:56:25 2014 -0800

----------------------------------------------------------------------
 .../LocalFileSystemCodeDistributor.java              | 15 ++++++++-------
 1 file changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a92a1e9c/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java b/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
index 96422e2..02d5e2d 100644
--- a/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
+++ b/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
@@ -1,5 +1,6 @@
 package backtype.storm.codedistributor;
 
+import backtype.storm.nimbus.NimbusInfo;
 import backtype.storm.utils.ZookeeperAuthInfo;
 import com.google.common.collect.Lists;
 import org.apache.commons.io.FileUtils;
@@ -62,26 +63,26 @@ public class LocalFileSystemCodeDistributor implements ICodeDistributor {
         List<String> hostInfos = zkClient.getChildren().forPath("/code-distributor/" + topologyid);
         File destDir = metafile.getParentFile();
         List<File> downloadedFiles = Lists.newArrayList();
-        for (String absoluteFilePath : FileUtils.readLines(metafile)) {
+        for (String absolutePathOnRemote : FileUtils.readLines(metafile)) {
 
-            File localFile = new File(destDir, new File(absoluteFilePath).getName());
+            File localFile = new File(destDir, new File(absolutePathOnRemote).getName());
 
             boolean isSuccess = false;
             for (String hostAndPort : hostInfos) {
-                String host = hostAndPort.split(":")[0];
-                int port = Integer.parseInt(hostAndPort.split(":")[1]);
+                NimbusInfo nimbusInfo = NimbusInfo.parse(hostAndPort);
                 try {
-                    downloadFromHost(conf, absoluteFilePath, localFile.getAbsolutePath(), host, port);
+                    LOG.info("Attempting to download meta file {} from remote {}", absolutePathOnRemote, nimbusInfo.toHostPortString());
+                    downloadFromHost(conf, absolutePathOnRemote, localFile.getAbsolutePath(), nimbusInfo.getHost(), nimbusInfo.getPort());
                     downloadedFiles.add(localFile);
                     isSuccess = true;
                     break;
                 } catch (Exception e) {
-                    LOG.error("download failed from {}:{}, will try another endpoint ", host, port, e);
+                    LOG.error("download failed from {}:{}, will try another endpoint ", nimbusInfo.getHost(), nimbusInfo.getPort(), e);
                 }
             }
 
             if(!isSuccess) {
-                throw new RuntimeException("File " + absoluteFilePath +" could not be downloaded from any endpoint");
+                throw new RuntimeException("File " + absolutePathOnRemote +" could not be downloaded from any endpoint");
             }
         }
 


[02/50] [abbrv] storm git commit: STORM-166: fixing a bug where the leader lock was not queued due to incorrect latch state being used

Posted by bo...@apache.org.
STORM-166: fixing a bug where the leader lock was not queued due to incorrect latch state being used


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

Branch: refs/heads/master
Commit: dd991e53a9c0e474360e6cb70b6c1d2217a2aa07
Parents: d0bb383
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 12:09:16 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 12:09:16 2014 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/zookeeper.clj | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/dd991e53/storm-core/src/clj/backtype/storm/zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj
index a7d43ac..a058098 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -233,7 +233,7 @@
         STORMS-ROOT (str (conf STORM-ZOOKEEPER-ROOT) "/storms")]
     (reify LeaderLatchListener
       (^void isLeader[this]
-        (log-message (str hostname "gained leadership, checking if it has all the topology code locally."))
+        (log-message (str hostname " gained leadership, checking if it has all the topology code locally."))
         (let [active-topology-ids (set (get-children zk STORMS-ROOT false))
               local-topology-ids (set (.list (File. (master-stormdist-root conf))))
               diff-topology (first (set-delta active-topology-ids local-topology-ids))]
@@ -241,9 +241,9 @@
                           "] local-topology-ids [" (clojure.string/join "," local-topology-ids)
                           "] diff-topology [" (clojure.string/join "," diff-topology) "]")
         (if (empty? diff-topology)
-          (log-message " Accepting leadership, all active topology found localy.")
+          (log-message "Accepting leadership, all active topology found localy.")
           (do
-            (log-message " code for all active topologies not available locally, giving up leadership.")
+            (log-message "code for all active topologies not available locally, giving up leadership.")
             (.close leader-latch)))))
       (^void notLeader[this]
         (log-message (str hostname " lost leadership."))))))
@@ -263,9 +263,8 @@
         (log-message "no-op for zookeeper implementation"))
 
       (^void addToLeaderLockQueue [this]
-        (let [state (.getState @leader-latch)]
         ;if this latch is already closed, we need to create new instance.
-        (if (.equals LeaderLatch$State/CLOSED state)
+        (if (.equals LeaderLatch$State/CLOSED (.getState @leader-latch))
           (do
             (reset! leader-latch (LeaderLatch. zk leader-lock-path id))
             (reset! leader-latch-listener (leader-latch-listener-impl conf zk @leader-latch))
@@ -273,12 +272,12 @@
             ))
 
         ;Only if the latch is not already started we invoke start.
-        (if (.equals LeaderLatch$State/LATENT state)
+        (if (.equals LeaderLatch$State/LATENT (.getState @leader-latch))
           (do
             (.addListener @leader-latch @leader-latch-listener)
             (.start @leader-latch)
             (log-message "Queued up for leader lock."))
-          (log-message "Node already in queue for leader lock."))))
+          (log-message "Node already in queue for leader lock.")))
 
       (^void removeFromLeaderLockQueue [this]
         ;Only started latches can be closed.


[47/50] [abbrv] storm git commit: Addressing TODOs. Calling sync before calling getChildren on code-distributor path as zookeeper does not gurantee Simultaneously Consistent Cross-Client Views unless sync is called.

Posted by bo...@apache.org.
Addressing TODOs. Calling sync before calling getChildren on code-distributor path as zookeeper does not gurantee Simultaneously Consistent Cross-Client Views unless sync is called.


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

Branch: refs/heads/master
Commit: 93dbcafaa2e5e9fd4cdc9445d1104fb61ae286f2
Parents: ef3cee6
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Aug 12 13:53:39 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 13:53:39 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj      | 17 ++++++++++++++---
 .../src/clj/backtype/storm/daemon/nimbus.clj       |  7 +++----
 2 files changed, 17 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/93dbcafa/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 04f5b89..3884096 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -44,7 +44,8 @@
   (close [this])
   (register [this callback])
   (unregister [this id])
-  (add-listener [this listener]))
+  (add-listener [this listener])
+  (sync-path [this path]))
 
 (defn mk-topo-only-acls
   [topo-conf]
@@ -145,6 +146,10 @@
       (add-listener
         [this listener]
         (zk/add-listener zk listener))
+
+      (sync-path
+        [this path]
+        (zk/sync-path zk path))
       )))
 
 (defprotocol StormClusterState
@@ -365,7 +370,9 @@
         [this callback]
         (when callback
           (reset! code-distributor-callback callback))
-        (get-children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback)))
+        (do
+          (sync-path cluster-state CODE-DISTRIBUTOR-SUBTREE)
+          (get-children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback))))
 
       (nimbuses
         [this]
@@ -389,7 +396,11 @@
 
       (code-distributor-info
         [this storm-id]
-        (map (fn [nimbus-info] (NimbusInfo/parse nimbus-info)) (get-children cluster-state (code-distributor-path storm-id) false)))
+        (map (fn [nimbus-info] (NimbusInfo/parse nimbus-info))
+          (let [path (code-distributor-path storm-id)]
+            (do
+              (sync-path cluster-state path)
+              (get-children cluster-state path false)))))
 
       (active-storms
         [this]

http://git-wip-us.apache.org/repos/asf/storm/blob/93dbcafa/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 fa3d8ec..3ee4cdc 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1492,8 +1492,6 @@
         (.removeFromLeaderLockQueue (:leader-elector nimbus))
         (doseq [missing missing-topologies]
           (log-message "missing topology " missing " has state on zookeeper but doesn't have a local dir on this host.")
-          ;; complete heck to get around zookeeper eventual consistency issue. zk/sync is not helping us so adding a sleep.
-          (sleep-secs 5)
           (let [nimbuses-with-missing (.code-distributor-info storm-cluster-state missing)]
             (log-message "trying to download missing topology code from " (clojure.string/join "," nimbuses-with-missing))
             (doseq [nimbus-host-port nimbuses-with-missing]
@@ -1501,8 +1499,9 @@
                 (try
                   (download-code conf nimbus missing (.getHost nimbus-host-port) (.getPort nimbus-host-port))
                   (catch Exception e (log-error e "Exception while trying to syn-code for missing topology" missing)))))))))
-    ;;TODO Ideally This should only be called if all missing topology code was successfully downloaded.
-    (.addToLeaderLockQueue (:leader-elector nimbus))))
+
+    (if (empty? (set/difference active-topologies (set (code-ids (:conf nimbus)))))
+      (.addToLeaderLockQueue (:leader-elector nimbus)))))
 
 (defmethod sync-code :local [conf nimbus]
   nil)


[11/50] [abbrv] storm git commit: STORM-166: deleting the bittorrent code distributor as the ttorent library does not support Distrubted hash table for trackerless torrents.

Posted by bo...@apache.org.
STORM-166: deleting the bittorrent code distributor as the ttorent library does not support Distrubted hash table for trackerless torrents.


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

Branch: refs/heads/master
Commit: dc24e440fa98f62e232e7929fd075387e2e56e4e
Parents: 27d6b4c
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 14:37:38 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 14:37:38 2014 -0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |   6 -
 storm-core/src/jvm/backtype/storm/Config.java   |  26 ---
 .../BitTorrentCodeDistributor.java              | 191 -------------------
 3 files changed, 223 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/dc24e440/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index e189966..9fd9c32 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -50,11 +50,6 @@ storm.auth.simple-acl.admins: []
 storm.meta.serialization.delegate: "backtype.storm.serialization.DefaultSerializationDelegate"
 storm.codedistributor.class: "backtype.storm.codedistributor.LocalFileSystemCodeDistributor"
 
-### bittorrent configuration
-bittorrent.port: 6969
-bittorrent.max.upload.rate: 0.0
-bittorrent.max.download.rate: 0.0
-
 ### nimbus.* configs are for the master
 nimbus.thrift.port: 6627
 nimbus.thrift.threads: 64
@@ -129,7 +124,6 @@ supervisor.monitor.frequency.secs: 3
 #how frequently the supervisor heartbeats to the cluster state (for nimbus)
 supervisor.heartbeat.frequency.secs: 5
 supervisor.enable: true
-supervisor.bittorrent.seed.duration: 0
 supervisor.supervisors: []
 supervisor.supervisors.commands: []
 

http://git-wip-us.apache.org/repos/asf/storm/blob/dc24e440/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 9746565..4678177 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1258,32 +1258,6 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_CODE_DISTRIBUTOR_CLASS_SCHEMA = String.class;
 
     /**
-     * Which port the BitTorrent tracker should bind to.
-     */
-    public static final String BITTORRENT_PORT = "bittorrent.port";
-    public static final Object BITTORRENT_PORT_SCHEMA = Number.class;
-
-    /**
-     * Max upload rate for topology torrents in kB/sec. 0.0 == unlimited.
-     */
-    public static final String BITTORRENT_MAX_UPLOAD_RATE = "bittorrent.max.upload.rate";
-    public static final Object BITTORRENT_MAX_UPLOAD_RATE_SCHEMA = Number.class;
-
-    /**
-     * Max download rate for topology torrents in kB/sec. 0.0 == unlimited.
-     */
-    public static final String BITTORRENT_MAX_DOWNLOAD_RATE = "bittorrent.max.download.rate";
-    public static final Object BITTORRENT_MAX_DOWNLOAD_RATE_SCHEMA = Number.class;
-
-    /**
-     * Time in seconds that a supervisor should seed after completing a topology torrent download.
-     * A value of 0 will disable seeding (download only). A value of -1 indicates that the supervisor
-     * should seed indefinitely (until the topology is killed).
-     */
-    public static final String SUPERVISOR_BITTORRENT_SEED_DURATION = "supervisor.bittorrent.seed.duration";
-    public static final Object SUPERVISOR_BITTORRENT_SEED_DURATION_SCHEMA = Number.class;
-
-    /**
      * Minimum number of nimbus hosts where the code must be replicated before leader nimbus
      * is allowed to perform topology activation tasks like setting up heartbeats/assignments
      * and marking the topology as active. default is 0.

http://git-wip-us.apache.org/repos/asf/storm/blob/dc24e440/storm-core/src/jvm/backtype/storm/codedistributor/BitTorrentCodeDistributor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/codedistributor/BitTorrentCodeDistributor.java b/storm-core/src/jvm/backtype/storm/codedistributor/BitTorrentCodeDistributor.java
deleted file mode 100644
index 190cc5f..0000000
--- a/storm-core/src/jvm/backtype/storm/codedistributor/BitTorrentCodeDistributor.java
+++ /dev/null
@@ -1,191 +0,0 @@
-package backtype.storm.codedistributor;
-
-import backtype.storm.Config;
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Shorts;
-import com.turn.ttorrent.client.Client;
-import com.turn.ttorrent.client.SharedTorrent;
-import com.turn.ttorrent.common.Torrent;
-import com.turn.ttorrent.tracker.TrackedTorrent;
-import com.turn.ttorrent.tracker.Tracker;
-import org.apache.commons.io.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.util.*;
-
-public class BitTorrentCodeDistributor implements ICodeDistributor {
-    private static final Logger LOG = LoggerFactory.getLogger(BitTorrentCodeDistributor.class);
-    private Tracker tracker;
-    private String hostName;
-    private InetSocketAddress address;
-    private Integer port;
-    protected HashMap<String, Client> clients = new HashMap<String, Client>();
-    protected Double maxDownload;
-    protected Double maxUpload;
-    private Integer seedDuration;
-
-    @Override
-    public void prepare(Map conf) throws Exception {
-        this.hostName = InetAddress.getLocalHost().getCanonicalHostName();
-        this.port = (Integer) conf.get(Config.BITTORRENT_PORT);
-        this.maxDownload = (Double) conf.get(Config.BITTORRENT_MAX_DOWNLOAD_RATE);
-        this.maxUpload = (Double) conf.get(Config.BITTORRENT_MAX_UPLOAD_RATE);
-        this.seedDuration = (Integer) conf.get(Config.SUPERVISOR_BITTORRENT_SEED_DURATION);
-
-        LOG.info("Download rates [U/D]: {}/{} kB/sec", format(this.maxUpload), format(this.maxDownload));
-
-        LOG.info("Starting bt tracker bound to hostname '{}'", hostName);
-        //using "0.0.0.0" to ensure we bind to all IPV4 network interfaces.
-        this.address = new InetSocketAddress("0.0.0.0", port);
-
-        this.tracker = new Tracker(address);
-        LOG.info("Announce URL: {}", this.tracker.getAnnounceUrl());
-        this.tracker.start();
-    }
-
-    @Override
-    public File upload(String dirPath, String topologyId) throws Exception {
-        File destDir = new File(dirPath);
-        LOG.info("Generating torrent for directory: {}", destDir.getAbsolutePath());
-
-        URI uri = URI.create("http://" + this.hostName + ":" + this.port + "/announce");
-        LOG.info("Creating torrent with announce URL: {}", uri);
-
-        //TODO: why does listing the directory not work?
-        ArrayList<File> files = new ArrayList<File>();
-        files.add(new File(destDir, "stormjar.jar"));
-        files.add(new File(destDir, "stormconf.ser"));
-        files.add(new File(destDir, "stormcode.ser"));
-
-        Torrent torrent = Torrent.create(destDir, files, uri, "storm-nimbus");
-        File torrentFile = new File(destDir, "storm-code-distributor.meta");
-        torrent.save(new FileOutputStream(torrentFile));
-        LOG.info("Saved torrent: {}", torrentFile.getAbsolutePath());
-        this.tracker.announce(new TrackedTorrent(torrent));
-
-        Client client = new Client(getInetAddress(), new SharedTorrent(torrent, destDir.getParentFile(), true));
-        this.clients.put(topologyId, client);
-        rebalanceRates();
-        client.share();
-        LOG.info("Seeding torrent...");
-
-        /**
-         *
-         * TODO: Every time on prepare we need to call tracker.announce for all torrents that
-         * exists in the file system, other wise the tracker will reject any peer request
-         * with unknown torrents. You need to bootstrap trackers.
-         */
-        return torrentFile;
-    }
-
-    @Override
-    public List<File> download(String topologyId, File torrentFile) throws Exception {
-        LOG.info("Initiating BitTorrent download.");
-
-        File destDir = torrentFile.getParentFile();
-        LOG.info("Downloading with torrent file: {}", torrentFile.getAbsolutePath());
-        LOG.info("Saving files to directory: {}", destDir.getAbsolutePath());
-        SharedTorrent st = SharedTorrent.fromFile(torrentFile, destDir);
-        Client client = new Client(getInetAddress(), st);
-        this.clients.put(topologyId, client);
-        rebalanceRates();
-        client.share(this.seedDuration);
-
-        //TODO: Should have a timeout after which we just fail the supervisor.
-        if (this.seedDuration == 0) {
-            client.waitForCompletion();
-        } else {
-            LOG.info("Waiting for seeding to begin...");
-            while (client.getState() != Client.ClientState.SEEDING && client.getState() != Client.ClientState.ERROR) {
-                try {
-                    Thread.sleep(10);
-                } catch (InterruptedException e) {
-                }
-            }
-        }
-        LOG.info("BitTorrent download complete.");
-
-        /**
-         * This should not be needed. currently the bittorrent library uses the torrent name (which is topologyId)
-         * as the folder name and downloads all the files under that folder. so we need to either download
-         * the torrent files under /storm-local/supervisor/stormdist or nimbus/stormdist/ to ensure stormdist becomes
-         * the parent of all torrent files and the actual code will be downloaded under stormdist/topologyId/.
-         * Ideally we should be able to specify that the downloaded files must be downloaded under
-         * given folder only and no extra folder needs to be created.
-         */
-
-        File srcDir = new File(destDir, topologyId);
-        for (File file : srcDir.listFiles()) {
-            FileUtils.copyFileToDirectory(file, destDir);
-            file.delete();
-        }
-        srcDir.delete();
-
-        return Lists.newArrayList(destDir.listFiles());
-    }
-
-    private InetAddress getInetAddress() throws UnknownHostException {
-        for (InetAddress addr : InetAddress.getAllByName(this.hostName)) {
-            if (!addr.isAnyLocalAddress() && !addr.isLoopbackAddress() && !addr.isMulticastAddress()) {
-                return addr;
-            }
-        }
-
-        throw new RuntimeException("No valid InetAddress could be obtained, something really wrong with network configuration.");
-    }
-
-    @Override
-    public short getReplicationCount(String topologyId) {
-        Collection<TrackedTorrent> trackedTorrents = tracker.getTrackedTorrents();
-        for (final TrackedTorrent trackedTorrent : trackedTorrents) {
-            if (trackedTorrent.getName().equals(topologyId)) {
-                return Shorts.checkedCast(trackedTorrent.seeders());
-            }
-        }
-
-        LOG.warn("No torrent found in tracker for topologyId = " + topologyId);
-        return 0;
-    }
-
-    @Override
-    public void cleanup(String topologyId) {
-        LOG.info("Stop seeding/tracking for topology {}", topologyId);
-        Client client = this.clients.remove(topologyId);
-        if (client != null) {
-            Torrent torrent = client.getTorrent();
-            client.stop();
-            this.tracker.remove(torrent);
-        }
-        rebalanceRates();
-    }
-
-    @Override
-    public void close(Map conf) {
-        this.tracker.stop();
-    }
-
-    private synchronized void rebalanceRates() {
-        int clientCount = this.clients.size();
-        if (clientCount > 0) {
-            double maxDl = this.maxDownload <= 0.0 ? this.maxDownload : this.maxDownload / clientCount;
-            double maxUl = this.maxUpload <= 0.0 ? this.maxUpload : this.maxUpload / clientCount;
-            LOG.info("Rebalancing bandwidth allocation based on {} topology torrents.", clientCount);
-            LOG.info("Per-torrent allocation [D/U]: {}/{} kB/sec.", format(maxDl), format(maxUl));
-            for (Client client : this.clients.values()) {
-                client.setMaxDownloadRate(maxDl);
-                client.setMaxUploadRate(maxUl);
-            }
-        }
-    }
-
-    private static String format(double val) {
-        return val <= 0.0 ? "UNLIMITED" : String.format("%.2f", val);
-    }
-}


[42/50] [abbrv] storm git commit: modifed ui to reflect dead nimbus hosts based on nimbus.seeds.

Posted by bo...@apache.org.
modifed ui to reflect dead nimbus hosts based on nimbus.seeds.

Conflicts:
	storm-core/src/jvm/backtype/storm/utils/NimbusClient.java


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

Branch: refs/heads/master
Commit: 16293e46f9b32bb495067217a721fe720a488975
Parents: 3f66ffd
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Apr 9 11:57:19 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 09:29:35 2015 -0700

----------------------------------------------------------------------
 STORM-UI-REST-API.md                            |  3 +-
 storm-core/src/clj/backtype/storm/cluster.clj   |  2 ++
 storm-core/src/clj/backtype/storm/ui/core.clj   | 34 ++++++++++++++------
 .../jvm/backtype/storm/utils/NimbusClient.java  | 30 ++++++++++-------
 .../public/templates/index-page-template.html   |  5 +--
 5 files changed, 51 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/16293e46/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md
index 35ba6ed..a8f2bd1 100644
--- a/STORM-UI-REST-API.md
+++ b/STORM-UI-REST-API.md
@@ -148,6 +148,7 @@ Response fields:
 |---	|---	|---
 |host| String | Nimbus' host name|
 |port| int| Nimbus' port number|
+|status| String| Possible values are Leader, Not a Leader, Dead|
 |nimbusUpTime| String| Shows since how long the nimbus has been running|
 |nimbusLogLink| String| Logviewer url to view the nimbus.log|
 |version| String| Version of storm this nimbus host is running|
@@ -161,7 +162,7 @@ Sample response:
             "host":"192.168.202.1",
             "port":6627,
             "nimbusLogLink":"http:\/\/192.168.202.1:8000\/log?file=nimbus.log",
-            "isLeader":true,
+            "status":Leader,
             "version":"0.10.0-SNAPSHOT",
             "nimbusUpTime":"3m 33s"
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/16293e46/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 15af1b0..65cdb47 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -366,6 +366,8 @@
 
       (add-nimbus-host!
         [this nimbus-id nimbus-summary]
+        ;explicit delete for ephmeral node to ensure this session creates the entry.
+        (delete-node cluster-state (nimbus-path nimbus-id))
         (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
 
       (code-distributor-info

http://git-wip-us.apache.org/repos/asf/storm/blob/16293e46/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 950b88d..40d030f 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -570,21 +570,37 @@
         "executorsTotal" total-executors
         "tasksTotal" total-tasks })))
 
+(defn convert-to-nimbus-summary[nimbus-seed]
+  (let [[host port] (.split nimbus-seed ":")]
+    {
+      "host" host
+      "port" port
+      "nimbusLogLink" (nimbus-log-link host port)
+      "status" "Offline"
+      "version" "Not applicable"
+      "nimbusUpTime" "Not applicable"}
+    ))
+
 (defn nimbus-summary
   ([]
     (thrift/with-configured-nimbus-connection nimbus
       (nimbus-summary
         (.get_nimbuses (.getClusterInfo ^Nimbus$Client nimbus)))))
   ([nimbuses]
-    {"nimbuses"
-     (for [^NimbusSummary n nimbuses]
-       {
-        "host" (.get_host n)
-        "port" (.get_port n)
-        "nimbusLogLink" (nimbus-log-link (.get_host n) (.get_port n))
-        "isLeader" (.is_isLeader n)
-        "version" (.get_version n)
-        "nimbusUpTime" (pretty-uptime-sec (.get_uptime_secs n))})}))
+    (let [nimbus-seeds (set (*STORM-CONF* NIMBUS-SEEDS))
+          alive-nimbuses (set (map #(str (.get_host %1) ":" (.get_port %1)) nimbuses))
+          dead-nimbuses (clojure.set/difference nimbus-seeds alive-nimbuses)
+          dead-nimbuses-summary (map #(convert-to-nimbus-summary %1) dead-nimbuses)]
+      {"nimbuses"
+       (concat dead-nimbuses-summary
+       (for [^NimbusSummary n nimbuses]
+         {
+          "host" (.get_host n)
+          "port" (.get_port n)
+          "nimbusLogLink" (nimbus-log-link (.get_host n) (.get_port n))
+          "status" (if (.is_isLeader n) "Leader" "Not a Leader")
+          "version" (.get_version n)
+          "nimbusUpTime" (pretty-uptime-sec (.get_uptime_secs n))}))})))
 
 (defn supervisor-summary
   ([]

http://git-wip-us.apache.org/repos/asf/storm/blob/16293e46/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index 60fa3aa..323d2a8 100644
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -38,6 +38,7 @@ import java.util.List;
 import java.util.Map;
 
 public class NimbusClient extends ThriftClient {
+    public static final String DELIMITER = ":";
     private Nimbus.Client _client;
     private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class);
 
@@ -63,25 +64,32 @@ public class NimbusClient extends ThriftClient {
         }
 
         for(String seed : seeds) {
-            String[] split = seed.split(":");
+            String[] split = seed.split(DELIMITER);
             String host = split[0];
             int port = Integer.parseInt(split[1]);
+            ClusterSummary clusterInfo = null;
             try {
                 NimbusClient client = new NimbusClient(conf, host, port);
-                ClusterSummary clusterInfo = client.getClient().getClusterInfo();
-                List<NimbusSummary> nimbuses = clusterInfo.get_nimbuses();
-                if (nimbuses != null) {
-                    for (NimbusSummary nimbusSummary : nimbuses) {
-                        if (nimbusSummary.is_isLeader()) {
+                clusterInfo = client.getClient().getClusterInfo();
+            } catch (Exception e) {
+                LOG.warn("Ignoring exception while trying to get leader nimbus info from " + seed
+                        + ". will retry with a different seed host.", e);
+                continue;
+            }
+            List<NimbusSummary> nimbuses = clusterInfo.get_nimbuses();
+            if (nimbuses != null) {
+                for (NimbusSummary nimbusSummary : nimbuses) {
+                    if (nimbusSummary.is_isLeader()) {
+                        try {
                             return new NimbusClient(conf, nimbusSummary.get_host(), nimbusSummary.get_port(), null, asUser);
+                        } catch (TTransportException e) {
+                            String leaderNimbus = nimbusSummary.get_host() + DELIMITER + nimbusSummary.get_port();
+                            throw new RuntimeException("Failed to create a nimbus client for the leader " + leaderNimbus, e);
                         }
                     }
-                    throw new RuntimeException("Found nimbuses " + nimbuses + " none of which is elected as leader, please try " +
-                            "again after some time.");
                 }
-            } catch (Exception e) {
-                LOG.warn("Ignoring exception while trying to get leader nimbus info from " + seed
-                        + ". will retry with a different seed host.", e);
+                throw new RuntimeException("Found nimbuses " + nimbuses + " none of which is elected as leader, please try " +
+                        "again after some time.");
             }
         }
         throw new RuntimeException("Could not find leader nimbus from seed hosts " + seeds + ". " +

http://git-wip-us.apache.org/repos/asf/storm/blob/16293e46/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index 37c1aac..1e585ff 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -80,7 +80,8 @@
                 <span data-toggle="tooltip" data-placement="above" title="Nimbus Thrift port number.">Port</span>
             </th>
             <th>
-                <span data-toggle="tooltip" data-placement="above" title="True if this host is leader, false otherwise.">IsLeader</span>
+                <span data-toggle="tooltip" data-placement="above" title="Leader if this host is leader, Not a Leader for all other live hosts,
+                note that these hosts may or may not be in leader lock queue, and Dead for hosts that are part of nimbus.seeds list but are not alive.">Status</span>
             </th>
             <th>
                 <span data-toggle="tooltip" data-placement="above" title="Storm version this nimbus host is running.">Version</span>
@@ -95,7 +96,7 @@
         <tr>
             <td>{{host}}</td>
             <td><a href="{{nimbusLogLink}}">{{port}}</a></td>
-            <td>{{isLeader}}</td>
+            <td>{{status}}</td>
             <td>{{version}}</td>
             <td>{{nimbusUpTime}}</td>
         </tr>


[32/50] [abbrv] storm git commit: Merge remote-tracking branch 'upstream/master' into STORM-166

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
index bc09e68,bd8f90e..24d96fa
--- a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
@@@ -38,7 -50,9 +50,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class TopologySummary implements org.apache.thrift.TBase<TopologySummary, TopologySummary._Fields>, java.io.Serializable, Cloneable {
+ @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-3-2")
+ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary, TopologySummary._Fields>, java.io.Serializable, Cloneable, Comparable<TopologySummary> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologySummary");
  
    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
@@@ -50,8 -64,13 +64,14 @@@
    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)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 {
+     schemes.put(StandardScheme.class, new TopologySummaryStandardSchemeFactory());
+     schemes.put(TupleScheme.class, new TopologySummaryTupleSchemeFactory());
+   }
+ 
    private String id; // required
    private String name; // required
    private int num_tasks; // required
@@@ -59,9 -78,8 +79,9 @@@
    private int num_workers; // required
    private int uptime_secs; // required
    private String status; // required
-   private String sched_status; // required
-   private String owner; // required
-   private int replication_count; // required
+   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 {
@@@ -153,9 -168,8 +173,9 @@@
    private static final int __NUM_EXECUTORS_ISSET_ID = 1;
    private static final int __NUM_WORKERS_ISSET_ID = 2;
    private static final int __UPTIME_SECS_ISSET_ID = 3;
 +  private static final int __REPLICATION_COUNT_ISSET_ID = 4;
-   private BitSet __isset_bit_vector = new BitSet(5);
- 
+   private byte __isset_bitfield = 0;
 -  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER};
++  private static final _Fields optionals[] = {_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);
@@@ -463,28 -471,6 +482,28 @@@
      }
    }
  
 +  public int get_replication_count() {
 +    return this.replication_count;
 +  }
 +
 +  public void set_replication_count(int replication_count) {
 +    this.replication_count = replication_count;
 +    set_replication_count_isSet(true);
 +  }
 +
 +  public void unset_replication_count() {
-     __isset_bit_vector.clear(__REPLICATION_COUNT_ISSET_ID);
++    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
 +  }
 +
 +  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
 +  public boolean is_set_replication_count() {
-     return __isset_bit_vector.get(__REPLICATION_COUNT_ISSET_ID);
++    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
 +  }
 +
 +  public void set_replication_count_isSet(boolean value) {
-     __isset_bit_vector.set(__REPLICATION_COUNT_ISSET_ID, value);
++    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
 +  }
 +
    public void setFieldValue(_Fields field, Object value) {
      switch (field) {
      case ID:
@@@ -745,61 -709,57 +764,62 @@@
  
    @Override
    public int hashCode() {
-     HashCodeBuilder builder = new HashCodeBuilder();
+     List<Object> list = new ArrayList<Object>();
  
      boolean present_id = true && (is_set_id());
-     builder.append(present_id);
+     list.add(present_id);
      if (present_id)
-       builder.append(id);
+       list.add(id);
  
      boolean present_name = true && (is_set_name());
-     builder.append(present_name);
+     list.add(present_name);
      if (present_name)
-       builder.append(name);
+       list.add(name);
  
      boolean present_num_tasks = true;
-     builder.append(present_num_tasks);
+     list.add(present_num_tasks);
      if (present_num_tasks)
-       builder.append(num_tasks);
+       list.add(num_tasks);
  
      boolean present_num_executors = true;
-     builder.append(present_num_executors);
+     list.add(present_num_executors);
      if (present_num_executors)
-       builder.append(num_executors);
+       list.add(num_executors);
  
      boolean present_num_workers = true;
-     builder.append(present_num_workers);
+     list.add(present_num_workers);
      if (present_num_workers)
-       builder.append(num_workers);
+       list.add(num_workers);
  
      boolean present_uptime_secs = true;
-     builder.append(present_uptime_secs);
+     list.add(present_uptime_secs);
      if (present_uptime_secs)
-       builder.append(uptime_secs);
+       list.add(uptime_secs);
  
      boolean present_status = true && (is_set_status());
-     builder.append(present_status);
+     list.add(present_status);
      if (present_status)
-       builder.append(status);
+       list.add(status);
  
      boolean present_sched_status = true && (is_set_sched_status());
-     builder.append(present_sched_status);
+     list.add(present_sched_status);
      if (present_sched_status)
-       builder.append(sched_status);
+       list.add(sched_status);
  
      boolean present_owner = true && (is_set_owner());
-     builder.append(present_owner);
+     list.add(present_owner);
      if (present_owner)
-       builder.append(owner);
+       list.add(owner);
  
 +    boolean present_replication_count = true && (is_set_replication_count());
-     builder.append(present_replication_count);
++    list.add(present_replication_count);
 +    if (present_replication_count)
-       builder.append(replication_count);
++      list.add(replication_count);
 +
-     return builder.toHashCode();
+     return list.hashCode();
    }
  
+   @Override
    public int compareTo(TopologySummary other) {
      if (!getClass().equals(other.getClass())) {
        return getClass().getName().compareTo(other.getClass().getName());
@@@ -898,16 -857,6 +917,16 @@@
          return lastComparison;
        }
      }
-     lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(typedOther.is_set_replication_count());
++    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(other.is_set_replication_count());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (is_set_replication_count()) {
-       lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, typedOther.replication_count);
++      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, other.replication_count);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
      return 0;
    }
  
@@@ -1187,5 -991,218 +1067,241 @@@
      }
    }
  
+   private static class TopologySummaryStandardSchemeFactory implements SchemeFactory {
+     public TopologySummaryStandardScheme getScheme() {
+       return new TopologySummaryStandardScheme();
+     }
+   }
+ 
+   private static class TopologySummaryStandardScheme extends StandardScheme<TopologySummary> {
+ 
+     public void read(org.apache.thrift.protocol.TProtocol iprot, TopologySummary struct) throws org.apache.thrift.TException {
+       org.apache.thrift.protocol.TField schemeField;
+       iprot.readStructBegin();
+       while (true)
+       {
+         schemeField = iprot.readFieldBegin();
+         if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+           break;
+         }
+         switch (schemeField.id) {
+           case 1: // ID
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.id = iprot.readString();
+               struct.set_id_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 2: // NAME
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.name = iprot.readString();
+               struct.set_name_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 3: // NUM_TASKS
+             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+               struct.num_tasks = iprot.readI32();
+               struct.set_num_tasks_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 4: // NUM_EXECUTORS
+             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+               struct.num_executors = iprot.readI32();
+               struct.set_num_executors_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 5: // NUM_WORKERS
+             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+               struct.num_workers = iprot.readI32();
+               struct.set_num_workers_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 6: // UPTIME_SECS
+             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+               struct.uptime_secs = iprot.readI32();
+               struct.set_uptime_secs_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 7: // STATUS
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.status = iprot.readString();
+               struct.set_status_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 513: // SCHED_STATUS
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.sched_status = iprot.readString();
+               struct.set_sched_status_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 514: // OWNER
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.owner = iprot.readString();
+               struct.set_owner_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
++          case 515: // REPLICATION_COUNT
++            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
++              struct.replication_count = iprot.readI32();
++              struct.set_replication_count_isSet(true);
++            } else { 
++              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
++            }
++            break;
+           default:
+             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+         }
+         iprot.readFieldEnd();
+       }
+       iprot.readStructEnd();
+       struct.validate();
+     }
+ 
+     public void write(org.apache.thrift.protocol.TProtocol oprot, TopologySummary struct) throws org.apache.thrift.TException {
+       struct.validate();
+ 
+       oprot.writeStructBegin(STRUCT_DESC);
+       if (struct.id != null) {
+         oprot.writeFieldBegin(ID_FIELD_DESC);
+         oprot.writeString(struct.id);
+         oprot.writeFieldEnd();
+       }
+       if (struct.name != null) {
+         oprot.writeFieldBegin(NAME_FIELD_DESC);
+         oprot.writeString(struct.name);
+         oprot.writeFieldEnd();
+       }
+       oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+       oprot.writeI32(struct.num_tasks);
+       oprot.writeFieldEnd();
+       oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+       oprot.writeI32(struct.num_executors);
+       oprot.writeFieldEnd();
+       oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+       oprot.writeI32(struct.num_workers);
+       oprot.writeFieldEnd();
+       oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+       oprot.writeI32(struct.uptime_secs);
+       oprot.writeFieldEnd();
+       if (struct.status != null) {
+         oprot.writeFieldBegin(STATUS_FIELD_DESC);
+         oprot.writeString(struct.status);
+         oprot.writeFieldEnd();
+       }
+       if (struct.sched_status != null) {
+         if (struct.is_set_sched_status()) {
+           oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
+           oprot.writeString(struct.sched_status);
+           oprot.writeFieldEnd();
+         }
+       }
+       if (struct.owner != null) {
+         if (struct.is_set_owner()) {
+           oprot.writeFieldBegin(OWNER_FIELD_DESC);
+           oprot.writeString(struct.owner);
+           oprot.writeFieldEnd();
+         }
+       }
++      if (struct.is_set_replication_count()) {
++        oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC);
++        oprot.writeI32(struct.replication_count);
++        oprot.writeFieldEnd();
++      }
+       oprot.writeFieldStop();
+       oprot.writeStructEnd();
+     }
+ 
+   }
+ 
+   private static class TopologySummaryTupleSchemeFactory implements SchemeFactory {
+     public TopologySummaryTupleScheme getScheme() {
+       return new TopologySummaryTupleScheme();
+     }
+   }
+ 
+   private static class TopologySummaryTupleScheme extends TupleScheme<TopologySummary> {
+ 
+     @Override
+     public void write(org.apache.thrift.protocol.TProtocol prot, TopologySummary struct) throws org.apache.thrift.TException {
+       TTupleProtocol oprot = (TTupleProtocol) prot;
+       oprot.writeString(struct.id);
+       oprot.writeString(struct.name);
+       oprot.writeI32(struct.num_tasks);
+       oprot.writeI32(struct.num_executors);
+       oprot.writeI32(struct.num_workers);
+       oprot.writeI32(struct.uptime_secs);
+       oprot.writeString(struct.status);
+       BitSet optionals = new BitSet();
+       if (struct.is_set_sched_status()) {
+         optionals.set(0);
+       }
+       if (struct.is_set_owner()) {
+         optionals.set(1);
+       }
 -      oprot.writeBitSet(optionals, 2);
++      if (struct.is_set_replication_count()) {
++        optionals.set(2);
++      }
++      oprot.writeBitSet(optionals, 3);
+       if (struct.is_set_sched_status()) {
+         oprot.writeString(struct.sched_status);
+       }
+       if (struct.is_set_owner()) {
+         oprot.writeString(struct.owner);
+       }
++      if (struct.is_set_replication_count()) {
++        oprot.writeI32(struct.replication_count);
++      }
+     }
+ 
+     @Override
+     public void read(org.apache.thrift.protocol.TProtocol prot, TopologySummary struct) throws org.apache.thrift.TException {
+       TTupleProtocol iprot = (TTupleProtocol) prot;
+       struct.id = iprot.readString();
+       struct.set_id_isSet(true);
+       struct.name = iprot.readString();
+       struct.set_name_isSet(true);
+       struct.num_tasks = iprot.readI32();
+       struct.set_num_tasks_isSet(true);
+       struct.num_executors = iprot.readI32();
+       struct.set_num_executors_isSet(true);
+       struct.num_workers = iprot.readI32();
+       struct.set_num_workers_isSet(true);
+       struct.uptime_secs = iprot.readI32();
+       struct.set_uptime_secs_isSet(true);
+       struct.status = iprot.readString();
+       struct.set_status_isSet(true);
 -      BitSet incoming = iprot.readBitSet(2);
++      BitSet incoming = iprot.readBitSet(3);
+       if (incoming.get(0)) {
+         struct.sched_status = iprot.readString();
+         struct.set_sched_status_isSet(true);
+       }
+       if (incoming.get(1)) {
+         struct.owner = iprot.readString();
+         struct.set_owner_isSet(true);
+       }
++      if (incoming.get(2)) {
++        struct.replication_count = iprot.readI32();
++        struct.set_replication_count_isSet(true);
++      }
+     }
+   }
+ 
  }
  

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --cc storm-core/src/py/storm/ttypes.py
index d351652,8c3f5e6..4fc20ce
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@@ -2091,13 -2167,9 +2168,10 @@@ class TopologySummary
      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 __hash__(self):
-     return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status) + hash(self.sched_status) + hash(self.owner) + hash(self.replication_count)
- 
 -  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None,):
 +  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None, replication_count=None,):
      self.id = id
      self.name = name
      self.num_tasks = num_tasks
@@@ -2239,6 -2301,19 +2313,20 @@@
      return
  
  
+   def __hash__(self):
+     value = 17
+     value = (value * 31) ^ hash(self.id)
+     value = (value * 31) ^ hash(self.name)
+     value = (value * 31) ^ hash(self.num_tasks)
+     value = (value * 31) ^ hash(self.num_executors)
+     value = (value * 31) ^ hash(self.num_workers)
+     value = (value * 31) ^ hash(self.uptime_secs)
+     value = (value * 31) ^ hash(self.status)
+     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):
      L = ['%s=%r' % (key, value)
        for key, value in self.__dict__.iteritems()]
@@@ -2371,127 -2452,6 +2465,133 @@@ class SupervisorSummary
    def __ne__(self, other):
      return not (self == other)
  
 +class NimbusSummary:
 +  """
 +  Attributes:
 +   - host
 +   - port
-    - uptimeSecs
++   - uptime_secs
 +   - isLeader
 +   - version
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.STRING, 'host', None, None, ), # 1
 +    (2, TType.I32, 'port', None, None, ), # 2
-     (3, TType.I32, 'uptimeSecs', None, None, ), # 3
++    (3, TType.I32, 'uptime_secs', None, None, ), # 3
 +    (4, TType.BOOL, 'isLeader', None, None, ), # 4
 +    (5, TType.STRING, 'version', None, None, ), # 5
 +  )
 +
-   def __hash__(self):
-     return 0 + hash(self.host) + hash(self.port) + hash(self.uptimeSecs) + hash(self.isLeader) + hash(self.version)
- 
-   def __init__(self, host=None, port=None, uptimeSecs=None, isLeader=None, version=None,):
++  def __init__(self, host=None, port=None, uptime_secs=None, isLeader=None, version=None,):
 +    self.host = host
 +    self.port = port
-     self.uptimeSecs = uptimeSecs
++    self.uptime_secs = uptime_secs
 +    self.isLeader = isLeader
 +    self.version = version
 +
 +  def read(self, iprot):
 +    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
 +      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
 +      return
 +    iprot.readStructBegin()
 +    while True:
 +      (fname, ftype, fid) = iprot.readFieldBegin()
 +      if ftype == TType.STOP:
 +        break
 +      if fid == 1:
 +        if ftype == TType.STRING:
 +          self.host = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.I32:
 +          self.port = iprot.readI32();
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 3:
 +        if ftype == TType.I32:
-           self.uptimeSecs = iprot.readI32();
++          self.uptime_secs = iprot.readI32();
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 4:
 +        if ftype == TType.BOOL:
 +          self.isLeader = iprot.readBool();
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 5:
 +        if ftype == TType.STRING:
 +          self.version = iprot.readString().decode('utf-8')
 +        else:
 +          iprot.skip(ftype)
 +      else:
 +        iprot.skip(ftype)
 +      iprot.readFieldEnd()
 +    iprot.readStructEnd()
 +
 +  def write(self, oprot):
 +    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
 +      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
 +      return
 +    oprot.writeStructBegin('NimbusSummary')
 +    if self.host is not None:
 +      oprot.writeFieldBegin('host', TType.STRING, 1)
 +      oprot.writeString(self.host.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    if self.port is not None:
 +      oprot.writeFieldBegin('port', TType.I32, 2)
 +      oprot.writeI32(self.port)
 +      oprot.writeFieldEnd()
-     if self.uptimeSecs is not None:
-       oprot.writeFieldBegin('uptimeSecs', TType.I32, 3)
-       oprot.writeI32(self.uptimeSecs)
++    if self.uptime_secs is not None:
++      oprot.writeFieldBegin('uptime_secs', TType.I32, 3)
++      oprot.writeI32(self.uptime_secs)
 +      oprot.writeFieldEnd()
 +    if self.isLeader is not None:
 +      oprot.writeFieldBegin('isLeader', TType.BOOL, 4)
 +      oprot.writeBool(self.isLeader)
 +      oprot.writeFieldEnd()
 +    if self.version is not None:
 +      oprot.writeFieldBegin('version', TType.STRING, 5)
 +      oprot.writeString(self.version.encode('utf-8'))
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    if self.host is None:
 +      raise TProtocol.TProtocolException(message='Required field host is unset!')
 +    if self.port is None:
 +      raise TProtocol.TProtocolException(message='Required field port is unset!')
-     if self.uptimeSecs is None:
-       raise TProtocol.TProtocolException(message='Required field uptimeSecs is unset!')
++    if self.uptime_secs is None:
++      raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!')
 +    if self.isLeader is None:
 +      raise TProtocol.TProtocolException(message='Required field isLeader is unset!')
 +    if self.version is None:
 +      raise TProtocol.TProtocolException(message='Required field version is unset!')
 +    return
 +
 +
++  def __hash__(self):
++    value = 17
++    value = (value * 31) ^ hash(self.host)
++    value = (value * 31) ^ hash(self.port)
++    value = (value * 31) ^ hash(self.uptime_secs)
++    value = (value * 31) ^ hash(self.isLeader)
++    value = (value * 31) ^ hash(self.version)
++    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 ClusterSummary:
    """
    Attributes:
@@@ -2503,18 -2463,14 +2603,15 @@@
    thrift_spec = (
      None, # 0
      (1, TType.LIST, 'supervisors', (TType.STRUCT,(SupervisorSummary, SupervisorSummary.thrift_spec)), None, ), # 1
 -    (2, TType.I32, 'nimbus_uptime_secs', None, None, ), # 2
 +    None, # 2
      (3, TType.LIST, 'topologies', (TType.STRUCT,(TopologySummary, TopologySummary.thrift_spec)), None, ), # 3
 +    (4, TType.LIST, 'nimbuses', (TType.STRUCT,(NimbusSummary, NimbusSummary.thrift_spec)), None, ), # 4
    )
  
-   def __hash__(self):
-     return 0 + hash(self.supervisors) + hash(self.topologies) + hash(self.nimbuses)
- 
 -  def __init__(self, supervisors=None, nimbus_uptime_secs=None, topologies=None,):
 +  def __init__(self, supervisors=None, topologies=None, nimbuses=None,):
      self.supervisors = supervisors
 -    self.nimbus_uptime_secs = nimbus_uptime_secs
      self.topologies = topologies
 +    self.nimbuses = nimbuses
  
    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:
@@@ -2602,6 -2549,13 +2699,13 @@@
      return
  
  
+   def __hash__(self):
+     value = 17
+     value = (value * 31) ^ hash(self.supervisors)
 -    value = (value * 31) ^ hash(self.nimbus_uptime_secs)
+     value = (value * 31) ^ hash(self.topologies)
++    value = (value * 31) ^ hash(self.nimbuses)
+     return value
+ 
    def __repr__(self):
      L = ['%s=%r' % (key, value)
        for key, value in self.__dict__.iteritems()]
@@@ -2757,90 -2713,90 +2863,90 @@@ class BoltStats
        if fid == 1:
          if ftype == TType.MAP:
            self.acked = {}
-           (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin() 
 -          (_ktype81, _vtype82, _size80 ) = iprot.readMapBegin()
 -          for _i84 in xrange(_size80):
 -            _key85 = iprot.readString().decode('utf-8')
 -            _val86 = {}
 -            (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin()
 -            for _i91 in xrange(_size87):
 -              _key92 = GlobalStreamId()
 -              _key92.read(iprot)
 -              _val93 = iprot.readI64();
 -              _val86[_key92] = _val93
++          (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin()
 +          for _i91 in xrange(_size87):
 +            _key92 = iprot.readString().decode('utf-8')
 +            _val93 = {}
-             (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin() 
++            (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin()
 +            for _i98 in xrange(_size94):
 +              _key99 = GlobalStreamId()
 +              _key99.read(iprot)
 +              _val100 = iprot.readI64();
 +              _val93[_key99] = _val100
              iprot.readMapEnd()
 -            self.acked[_key85] = _val86
 +            self.acked[_key92] = _val93
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 2:
          if ftype == TType.MAP:
            self.failed = {}
-           (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin() 
 -          (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin()
 -          for _i98 in xrange(_size94):
 -            _key99 = iprot.readString().decode('utf-8')
 -            _val100 = {}
 -            (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin()
 -            for _i105 in xrange(_size101):
 -              _key106 = GlobalStreamId()
 -              _key106.read(iprot)
 -              _val107 = iprot.readI64();
 -              _val100[_key106] = _val107
++          (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin()
 +          for _i105 in xrange(_size101):
 +            _key106 = iprot.readString().decode('utf-8')
 +            _val107 = {}
-             (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin() 
++            (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin()
 +            for _i112 in xrange(_size108):
 +              _key113 = GlobalStreamId()
 +              _key113.read(iprot)
 +              _val114 = iprot.readI64();
 +              _val107[_key113] = _val114
              iprot.readMapEnd()
 -            self.failed[_key99] = _val100
 +            self.failed[_key106] = _val107
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 3:
          if ftype == TType.MAP:
            self.process_ms_avg = {}
-           (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin() 
 -          (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin()
 -          for _i112 in xrange(_size108):
 -            _key113 = iprot.readString().decode('utf-8')
 -            _val114 = {}
 -            (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin()
 -            for _i119 in xrange(_size115):
 -              _key120 = GlobalStreamId()
 -              _key120.read(iprot)
 -              _val121 = iprot.readDouble();
 -              _val114[_key120] = _val121
++          (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin()
 +          for _i119 in xrange(_size115):
 +            _key120 = iprot.readString().decode('utf-8')
 +            _val121 = {}
-             (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin() 
++            (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin()
 +            for _i126 in xrange(_size122):
 +              _key127 = GlobalStreamId()
 +              _key127.read(iprot)
 +              _val128 = iprot.readDouble();
 +              _val121[_key127] = _val128
              iprot.readMapEnd()
 -            self.process_ms_avg[_key113] = _val114
 +            self.process_ms_avg[_key120] = _val121
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 4:
          if ftype == TType.MAP:
            self.executed = {}
-           (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin() 
 -          (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin()
 -          for _i126 in xrange(_size122):
 -            _key127 = iprot.readString().decode('utf-8')
 -            _val128 = {}
 -            (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin()
 -            for _i133 in xrange(_size129):
 -              _key134 = GlobalStreamId()
 -              _key134.read(iprot)
 -              _val135 = iprot.readI64();
 -              _val128[_key134] = _val135
++          (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin()
 +          for _i133 in xrange(_size129):
 +            _key134 = iprot.readString().decode('utf-8')
 +            _val135 = {}
-             (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin() 
++            (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin()
 +            for _i140 in xrange(_size136):
 +              _key141 = GlobalStreamId()
 +              _key141.read(iprot)
 +              _val142 = iprot.readI64();
 +              _val135[_key141] = _val142
              iprot.readMapEnd()
 -            self.executed[_key127] = _val128
 +            self.executed[_key134] = _val135
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 5:
          if ftype == TType.MAP:
            self.execute_ms_avg = {}
-           (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin() 
 -          (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin()
 -          for _i140 in xrange(_size136):
 -            _key141 = iprot.readString().decode('utf-8')
 -            _val142 = {}
 -            (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin()
 -            for _i147 in xrange(_size143):
 -              _key148 = GlobalStreamId()
 -              _key148.read(iprot)
 -              _val149 = iprot.readDouble();
 -              _val142[_key148] = _val149
++          (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin()
 +          for _i147 in xrange(_size143):
 +            _key148 = iprot.readString().decode('utf-8')
 +            _val149 = {}
-             (_ktype151, _vtype152, _size150 ) = iprot.readMapBegin() 
++            (_ktype151, _vtype152, _size150 ) = iprot.readMapBegin()
 +            for _i154 in xrange(_size150):
 +              _key155 = GlobalStreamId()
 +              _key155.read(iprot)
 +              _val156 = iprot.readDouble();
 +              _val149[_key155] = _val156
              iprot.readMapEnd()
 -            self.execute_ms_avg[_key141] = _val142
 +            self.execute_ms_avg[_key148] = _val149
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -2977,51 -2939,51 +3089,51 @@@ class SpoutStats
        if fid == 1:
          if ftype == TType.MAP:
            self.acked = {}
-           (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin() 
 -          (_ktype171, _vtype172, _size170 ) = iprot.readMapBegin()
 -          for _i174 in xrange(_size170):
 -            _key175 = iprot.readString().decode('utf-8')
 -            _val176 = {}
 -            (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin()
 -            for _i181 in xrange(_size177):
 -              _key182 = iprot.readString().decode('utf-8')
 -              _val183 = iprot.readI64();
 -              _val176[_key182] = _val183
++          (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin()
 +          for _i181 in xrange(_size177):
 +            _key182 = iprot.readString().decode('utf-8')
 +            _val183 = {}
-             (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin() 
++            (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin()
 +            for _i188 in xrange(_size184):
 +              _key189 = iprot.readString().decode('utf-8')
 +              _val190 = iprot.readI64();
 +              _val183[_key189] = _val190
              iprot.readMapEnd()
 -            self.acked[_key175] = _val176
 +            self.acked[_key182] = _val183
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 2:
          if ftype == TType.MAP:
            self.failed = {}
-           (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin() 
 -          (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin()
 -          for _i188 in xrange(_size184):
 -            _key189 = iprot.readString().decode('utf-8')
 -            _val190 = {}
 -            (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin()
 -            for _i195 in xrange(_size191):
 -              _key196 = iprot.readString().decode('utf-8')
 -              _val197 = iprot.readI64();
 -              _val190[_key196] = _val197
++          (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin()
 +          for _i195 in xrange(_size191):
 +            _key196 = iprot.readString().decode('utf-8')
 +            _val197 = {}
-             (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin() 
++            (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin()
 +            for _i202 in xrange(_size198):
 +              _key203 = iprot.readString().decode('utf-8')
 +              _val204 = iprot.readI64();
 +              _val197[_key203] = _val204
              iprot.readMapEnd()
 -            self.failed[_key189] = _val190
 +            self.failed[_key196] = _val197
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 3:
          if ftype == TType.MAP:
            self.complete_ms_avg = {}
-           (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin() 
 -          (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin()
 -          for _i202 in xrange(_size198):
 -            _key203 = iprot.readString().decode('utf-8')
 -            _val204 = {}
 -            (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin()
 -            for _i209 in xrange(_size205):
 -              _key210 = iprot.readString().decode('utf-8')
 -              _val211 = iprot.readDouble();
 -              _val204[_key210] = _val211
++          (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin()
 +          for _i209 in xrange(_size205):
 +            _key210 = iprot.readString().decode('utf-8')
 +            _val211 = {}
-             (_ktype213, _vtype214, _size212 ) = iprot.readMapBegin() 
++            (_ktype213, _vtype214, _size212 ) = iprot.readMapBegin()
 +            for _i216 in xrange(_size212):
 +              _key217 = iprot.readString().decode('utf-8')
 +              _val218 = iprot.readDouble();
 +              _val211[_key217] = _val218
              iprot.readMapEnd()
 -            self.complete_ms_avg[_key203] = _val204
 +            self.complete_ms_avg[_key210] = _val211
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -3207,34 -3176,34 +3326,34 @@@ class ExecutorStats
        if fid == 1:
          if ftype == TType.MAP:
            self.emitted = {}
-           (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin() 
 -          (_ktype225, _vtype226, _size224 ) = iprot.readMapBegin()
 -          for _i228 in xrange(_size224):
 -            _key229 = iprot.readString().decode('utf-8')
 -            _val230 = {}
 -            (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin()
 -            for _i235 in xrange(_size231):
 -              _key236 = iprot.readString().decode('utf-8')
 -              _val237 = iprot.readI64();
 -              _val230[_key236] = _val237
++          (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin()
 +          for _i235 in xrange(_size231):
 +            _key236 = iprot.readString().decode('utf-8')
 +            _val237 = {}
-             (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin() 
++            (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin()
 +            for _i242 in xrange(_size238):
 +              _key243 = iprot.readString().decode('utf-8')
 +              _val244 = iprot.readI64();
 +              _val237[_key243] = _val244
              iprot.readMapEnd()
 -            self.emitted[_key229] = _val230
 +            self.emitted[_key236] = _val237
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
        elif fid == 2:
          if ftype == TType.MAP:
            self.transferred = {}
-           (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin() 
 -          (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin()
 -          for _i242 in xrange(_size238):
 -            _key243 = iprot.readString().decode('utf-8')
 -            _val244 = {}
 -            (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin()
 -            for _i249 in xrange(_size245):
 -              _key250 = iprot.readString().decode('utf-8')
 -              _val251 = iprot.readI64();
 -              _val244[_key250] = _val251
++          (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin()
 +          for _i249 in xrange(_size245):
 +            _key250 = iprot.readString().decode('utf-8')
 +            _val251 = {}
-             (_ktype253, _vtype254, _size252 ) = iprot.readMapBegin() 
++            (_ktype253, _vtype254, _size252 ) = iprot.readMapBegin()
 +            for _i256 in xrange(_size252):
 +              _key257 = iprot.readString().decode('utf-8')
 +              _val258 = iprot.readI64();
 +              _val251[_key257] = _val258
              iprot.readMapEnd()
 -            self.transferred[_key243] = _val244
 +            self.transferred[_key250] = _val251
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -4051,13 -4036,9 +4187,10 @@@ class TopologyInfo
      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 __hash__(self):
-     return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors) + hash(self.sched_status) + hash(self.owner) + hash(self.replication_count)
- 
 -  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, sched_status=None, owner=None, replication_count=None,):
      self.id = id
      self.name = name
      self.uptime_secs = uptime_secs
@@@ -4111,17 -4091,17 +4244,17 @@@
        elif fid == 6:
          if ftype == TType.MAP:
            self.errors = {}
-           (_ktype274, _vtype275, _size273 ) = iprot.readMapBegin() 
 -          (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin()
 -          for _i270 in xrange(_size266):
 -            _key271 = iprot.readString().decode('utf-8')
 -            _val272 = []
 -            (_etype276, _size273) = iprot.readListBegin()
 -            for _i277 in xrange(_size273):
 -              _elem278 = ErrorInfo()
 -              _elem278.read(iprot)
 -              _val272.append(_elem278)
++          (_ktype274, _vtype275, _size273 ) = iprot.readMapBegin()
 +          for _i277 in xrange(_size273):
 +            _key278 = iprot.readString().decode('utf-8')
 +            _val279 = []
 +            (_etype283, _size280) = iprot.readListBegin()
 +            for _i284 in xrange(_size280):
 +              _elem285 = ErrorInfo()
 +              _elem285.read(iprot)
 +              _val279.append(_elem285)
              iprot.readListEnd()
 -            self.errors[_key271] = _val272
 +            self.errors[_key278] = _val279
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -4215,6 -4186,18 +4348,19 @@@
      return
  
  
+   def __hash__(self):
+     value = 17
+     value = (value * 31) ^ hash(self.id)
+     value = (value * 31) ^ hash(self.name)
+     value = (value * 31) ^ hash(self.uptime_secs)
+     value = (value * 31) ^ hash(self.executors)
+     value = (value * 31) ^ hash(self.status)
+     value = (value * 31) ^ hash(self.errors)
+     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):
      L = ['%s=%r' % (key, value)
        for key, value in self.__dict__.iteritems()]
@@@ -4334,11 -4316,11 +4479,11 @@@ class RebalanceOptions
        elif fid == 3:
          if ftype == TType.MAP:
            self.num_executors = {}
-           (_ktype291, _vtype292, _size290 ) = iprot.readMapBegin() 
 -          (_ktype284, _vtype285, _size283 ) = iprot.readMapBegin()
 -          for _i287 in xrange(_size283):
 -            _key288 = iprot.readString().decode('utf-8')
 -            _val289 = iprot.readI32();
 -            self.num_executors[_key288] = _val289
++          (_ktype291, _vtype292, _size290 ) = iprot.readMapBegin()
 +          for _i294 in xrange(_size290):
 +            _key295 = iprot.readString().decode('utf-8')
 +            _val296 = iprot.readI32();
 +            self.num_executors[_key295] = _val296
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -4415,11 -4401,11 +4564,11 @@@ class Credentials
        if fid == 1:
          if ftype == TType.MAP:
            self.creds = {}
-           (_ktype300, _vtype301, _size299 ) = iprot.readMapBegin() 
 -          (_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
++          (_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
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/ui/public/index.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/test/clj/backtype/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/nimbus_test.clj
index c937aaa,99d1e32..48037c9
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@@ -17,18 -17,23 +17,24 @@@
    (:use [clojure test])
    (:require [backtype.storm [util :as util]])
    (:require [backtype.storm.daemon [nimbus :as nimbus]])
-   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
+   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount
+             TestAggregatesCounter TestPlannerSpout TestPlannerBolt])
    (:import [backtype.storm.scheduler INimbus])
-   (:use [backtype.storm bootstrap testing zookeeper])
 +  (:import [backtype.storm.nimbus ILeaderElector NimbusInfo])
+   (:import [backtype.storm.generated Credentials NotAliveException SubmitOptions
+             TopologyInitialStatus AlreadyAliveException KillOptions RebalanceOptions
+             InvalidTopologyException AuthorizationException])
+   (:import [java.util HashMap])
+   (:import [java.io File])
+   (:import [backtype.storm.utils Time])
+   (:import [org.apache.commons.io FileUtils])
 -  (:use [backtype.storm testing MockAutoCred util config log timer])
++  (:use [backtype.storm testing MockAutoCred util config log timer zookeeper])
    (:use [backtype.storm.daemon common])
-   (:import [backtype.storm.generated Credentials])
-   (:use [backtype.storm bootstrap testing MockAutoCred])
    (:require [conjure.core])
-   (:use [conjure core])
-   )
- 
- (bootstrap)
+   (:require [backtype.storm
+              [thrift :as thrift]
+              [cluster :as cluster]])
+   (:use [conjure core]))
  
  (defn storm-component->task-info [cluster storm-name]
    (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------


[14/50] [abbrv] storm git commit: STORM-166: removing dependency on bit torrent from pom.xml.

Posted by bo...@apache.org.
STORM-166: removing dependency on bit torrent from pom.xml.


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

Branch: refs/heads/master
Commit: 7681686f324c76a4f8df4887c6abdfe7074be23a
Parents: a17167e
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Jan 8 11:51:53 2015 -0500
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Jan 8 11:51:53 2015 -0500

----------------------------------------------------------------------
 pom.xml            | 17 -----------------
 storm-core/pom.xml | 15 ---------------
 2 files changed, 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7681686f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e075ac0..b35a8d2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -209,7 +209,6 @@
         <reply.version>0.3.0</reply.version>
         <conjure.version>2.1.3</conjure.version>
         <zookeeper.version>3.4.6</zookeeper.version>
-        <bittorrent.version>1.4</bittorrent.version>
         <conjure.version>2.1.3</conjure.version>
         <clojure-data-codec.version>0.1.0</clojure-data-codec.version>
         <clojure-contrib.version>1.2.0</clojure-contrib.version>
@@ -384,22 +383,6 @@
             </dependency>
 
             <dependency>
-                <groupId>com.turn</groupId>
-                <artifactId>ttorrent</artifactId>
-                <version>${bittorrent.version}</version>
-                <exclusions>
-                    <exclusion>
-                        <groupId>log4j</groupId>
-                        <artifactId>log4j</artifactId>
-                    </exclusion>
-                    <exclusion>
-                        <groupId>org.slf4j</groupId>
-                        <artifactId>slf4j-log4j12</artifactId>
-                    </exclusion>
-                </exclusions>
-            </dependency>
-
-            <dependency>
                 <groupId>com.googlecode.json-simple</groupId>
                 <artifactId>json-simple</artifactId>
                 <version>${json-simple.version}</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/7681686f/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 11f1982..bd7a935 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -179,21 +179,6 @@
         </dependency>
 
         <dependency>
-            <groupId>com.turn</groupId>
-            <artifactId>ttorrent</artifactId>
-            <exclusions>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
             <groupId>com.googlecode.json-simple</groupId>
             <artifactId>json-simple</artifactId>
         </dependency>


[37/50] [abbrv] storm git commit: Merge remote-tracking branch 'upstream/master' into STORM-166

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into STORM-166

Conflicts:
	storm-core/src/ui/public/index.html
	storm-core/src/ui/public/templates/index-page-template.html
	storm-core/src/ui/public/templates/topology-page-template.html


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

Branch: refs/heads/master
Commit: a11fcc303aeb728af4c9a7a1c232b2f1213330c4
Parents: eb3a837 dcee1e2
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Mar 20 11:07:07 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Mar 20 11:07:07 2015 -0700

----------------------------------------------------------------------
 BYLAWS.md                                       |   95 -
 CHANGELOG.md                                    |    1 +
 LICENSE                                         |  117 +-
 docs/_includes/header.html                      |   13 +-
 docs/css/main.scss                              |    3 +-
 docs/documentation/BYLAWS.md                    |   98 +
 docs/documentation/Powered-By.md                |    2 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |   47 +-
 .../src/clj/backtype/storm/ui/helpers.clj       |   21 -
 storm-core/src/ui/public/component.html         |  167 +-
 .../src/ui/public/css/bootstrap-1.4.0.css       | 2469 ------------------
 .../src/ui/public/css/bootstrap-3.3.1.min.css   |    5 +
 .../src/ui/public/css/dataTables.bootstrap.css  |  319 +++
 .../public/css/jquery.dataTables.1.10.4.min.css |    1 +
 storm-core/src/ui/public/css/style.css          |   94 +-
 .../src/ui/public/images/back_disabled.png      |  Bin 0 -> 1361 bytes
 .../src/ui/public/images/back_enabled.png       |  Bin 0 -> 1379 bytes
 .../src/ui/public/images/back_enabled_hover.png |  Bin 0 -> 1375 bytes
 .../src/ui/public/images/forward_disabled.png   |  Bin 0 -> 1363 bytes
 .../src/ui/public/images/forward_enabled.png    |  Bin 0 -> 1380 bytes
 .../ui/public/images/forward_enabled_hover.png  |  Bin 0 -> 1379 bytes
 storm-core/src/ui/public/images/sort_asc.png    |  Bin 0 -> 1118 bytes
 .../src/ui/public/images/sort_asc_disabled.png  |  Bin 0 -> 2916 bytes
 storm-core/src/ui/public/images/sort_both.png   |  Bin 0 -> 1136 bytes
 storm-core/src/ui/public/images/sort_desc.png   |  Bin 0 -> 1127 bytes
 .../src/ui/public/images/sort_desc_disabled.png |  Bin 0 -> 1045 bytes
 storm-core/src/ui/public/index.html             |  120 +-
 .../src/ui/public/js/bootstrap-3.3.1.min.js     |    7 +
 storm-core/src/ui/public/js/bootstrap-twipsy.js |  310 ---
 .../ui/public/js/dataTables.bootstrap.min.js    |    8 +
 .../src/ui/public/js/jquery-1.11.1.min.js       |    4 +
 storm-core/src/ui/public/js/jquery-1.6.2.min.js |   18 -
 .../public/js/jquery.dataTables.1.10.4.min.js   |  157 ++
 .../src/ui/public/js/jquery.tablesorter.min.js  |    5 -
 storm-core/src/ui/public/js/script.js           |   86 +-
 .../templates/component-page-template.html      |  128 +-
 .../public/templates/index-page-template.html   |   72 +-
 .../templates/topology-page-template.html       |   92 +-
 .../src/ui/public/templates/user-template.html  |    2 +-
 storm-core/src/ui/public/topology.html          |  132 +-
 40 files changed, 1217 insertions(+), 3376 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a11fcc30/storm-core/src/ui/public/index.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/index.html
index 4cd8e5e,0c5ea10..70218a6
--- a/storm-core/src/ui/public/index.html
+++ b/storm-core/src/ui/public/index.html
@@@ -1,3 -1,3 +1,4 @@@
++
  <html><head>
      <!--
       Licensed to the Apache Software Foundation (ASF) under one or more
@@@ -28,24 -32,43 +33,49 @@@
  <script src="/js/script.js" type="text/javascript"></script>
  </head>
  <body>
- <div id="ui-user">
- </div>
- <h1><a href="/">Storm UI</a></h1>
- <h2>Cluster Summary</h2>
- <div id="cluster-summary">
- </div>
- <h2>Nimbus Summary</h2>
- <div id="nimbus-summary">
- </div>
- <h2>Topology summary</h2>
- <div id="topology-summary">
+ <div class="container-fluid">
+   <div class="row">
+     <div class="col-md-11">
+       <h1><a href="/">Storm UI</a></h1>
+     </div>
+     <div id="ui-user" class="col-md-1"></div>
+   </div>
+   <div class="row">
+     <div class="col-md-12">
+       <h2>Cluster Summary</h2>
+       <div id="cluster-summary"></div>
+     </div>
+   </div>
+   <div class="row">
++     <div class="col-md-12">
++        <h2>Nimbus Summary</h2>
++        <div id="nimbus-summary"></div>
++     </div>
++  </div>
++  <div class="row">
+     <div class="col-md-12">
+       <h2>Topology Summary</h2>
+       <div id="topology-summary"></div>
+     </div>
+   </div>
+   <div class="row">
+     <div class="col-md-12">
+       <h2>Supervisor Summary</h2>
+       <div id="supervisor-summary"></div>
+     </div>
+   </div>
+   <div class="row">
+     <div class="col-md-12">
+       <h2>Nimbus Configuration</h2>
+       <div id="nimbus-configuration"></div>
+     </div>
+   </div>
+   <div class="row">
+     <div class="col-md-12">
+       <div id="json-response-error"></div>
+     </div>
+   </div>
  </div>
- <h2>Supervisor summary</h2>
- <div id="supervisor-summary">
- </div>
- <h2>Nimbus Configuration</h2>
- <div id="nimbus-configuration"></div>
- <div id="json-response-error"></div>
  </body>
  <script>
  
@@@ -76,16 -106,9 +114,22 @@@ $(document).ready(function() 
  
          $.get("/templates/index-page-template.html", function(template) {
              clusterSummary.append(Mustache.render($(template).filter("#cluster-summary-template").html(),response));
+             $('#cluster-summary [data-toggle="tooltip"]').tooltip();
          });
      });
 +    $.getJSON("/api/v1/nimbus/summary",function(response,status,jqXHR) {
 +      $.get("/templates/index-page-template.html", function(template) {
 +          nimbusSummary.append(Mustache.render($(template).filter("#nimbus-summary-template").html(),response));
-           if(response["nimbuses"].length > 0) {
-               $("#nimbus-summary-table").tablesorter({ sortList: [[0,0]], headers: {3: { sorter: "stormtimestr"}}});
-           }
++          //host, port, isLeader, version, uptime
++          dtAutoPage("#nimbus-summary-table", {
++            columnDefs: [
++              {type: "num", targets: [1]},
++              {type: "time-str", targets: [4]}
++            ]
++          });
++          $('#nimbus-summary [data-toggle="tooltip"]').tooltip();
 +      });
 +    });
      $.getJSON("/api/v1/topology/summary",function(response,status,jqXHR) {
        $.get("/templates/index-page-template.html", function(template) {
            topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));

http://git-wip-us.apache.org/repos/asf/storm/blob/a11fcc30/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/templates/index-page-template.html
index 84da576,298dfff..07f1afa
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@@ -24,32 -24,37 +24,32 @@@
          </span>
        </th>
        <th>
-         <span class="tip above" title="The number of nodes in the cluster currently.">
 -        <span data-toggle="tooltip" data-placement="right" title="The duration the current Nimbus instance has been running. (Note that the storm cluster may have been deployed and available for a much longer period than the current Nimbus process has been running.)">
 -          Nimbus uptime
 -        </span>
 -      </th>
 -      <th>
+         <span data-toggle="tooltip" data-placement="above" title="The number of nodes in the cluster currently.">
            Supervisors
--      </span>
++        </span>
        </th>
        <th>
-         <span class="tip above" title="Slots are Workers (processes).">
+         <span data-toggle="tooltip" data-placement="above" title="Slots are Workers (processes).">
            Used slots
--      </span>
++        </span>
        </th>
        <th>
-         <span class="tip above" title="Slots are Workers (processes).">
+         <span data-toggle="tooltip" data-placement="above" title="Slots are Workers (processes).">
            Free slots
--      </span>
++        </span>
        </th>
        <th>
-         <span class="tip above" title="Slots are Workers (processes).">
+         <span data-toggle="tooltip" data-placement="above" title="Slots are Workers (processes).">
            Total slots
--      </span>
++        </span>
        </th>
        <th>
-         <span class="tip above" title="Executors are threads in a Worker process.">
+         <span data-toggle="tooltip" data-placement="above" title="Executors are threads in a Worker process.">
            Executors
--      </span>
++        </span>
        </th>
        <th>
-         <span class="tip left" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">
+         <span data-toggle="tooltip" data-placement="left" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">
            Tasks
          </span>
        </th>
@@@ -68,44 -74,8 +68,44 @@@
    </tbody>
  </table>
  </script>
 +
 +<script id="nimbus-summary-template" type="text/html">
 +    <table class="zebra-striped" id="nimbus-summary-table">
 +        <thead>
 +        <tr>
 +            <th>
-                 <span class="tip right" title="nimbus host name.">Host</span>
++                <span data-toggle="tooltip" data-placement="right" title="nimbus host name.">Host</span>
 +            </th>
 +            <th>
-                 <span class="tip above" title="Nimbus Thrift port number.">Port</span>
++                <span data-toggle="tooltip" data-placement="above" title="Nimbus Thrift port number.">Port</span>
 +            </th>
 +            <th>
-                 <span class="tip above" title="True if this host is leader, false otherwise.">IsLeader</span>
++                <span data-toggle="tooltip" data-placement="above" title="True if this host is leader, false otherwise.">IsLeader</span>
 +            </th>
 +            <th>
-                 <span class="tip left" title="Storm version this nimbus host is running.">Version</span>
++                <span data-toggle="tooltip" data-placement="above" title="Storm version this nimbus host is running.">Version</span>
 +            </th>
 +            <th>
-                 <span class="tip left" title="Time since this nimbus host has been running.">UpTime Seconds</span>
++                <span data-toggle="tooltip" data-placement="left" title="Time since this nimbus host has been running.">UpTime Seconds</span>
 +            </th>
 +        </tr>
 +        </thead>
 +        <tbody>
 +        {{#nimbuses}}
 +        <tr>
 +            <td>{{host}}</td>
 +            <td><a href="{{nimbusLogLink}}">{{port}}</a></td>
 +            <td>{{isLeader}}</td>
 +            <td>{{version}}</td>
 +            <td>{{nimbusUpTime}}</td>
 +        </tr>
 +        {{/nimbuses}}
 +        </tbody>
 +    </table>
 +</script>
 +
  <script id="topology-summary-template" type="text/html">
-   <table class="zebra-striped" id="topology-summary-table">
+   <table class="table table-striped compact" id="topology-summary-table">
      <thead>
        <tr>
          <th>
@@@ -149,12 -119,7 +149,12 @@@
            </span>
          </th>
          <th>
-           <span class="tip above" title="Number of nimbus hosts on which this topology's code is replicated. ">
++          <span data-toggle="tooltip" data-placement="above" title="Number of nimbus hosts on which this topology's code is replicated. ">
 +            Replication count
 +          </span>
 +        </th>
 +        <th>
-           <span class="tip left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
+           <span data-toggle="tooltip" data-placement="left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
              Scheduler Info
            </span>
          </th>

http://git-wip-us.apache.org/repos/asf/storm/blob/a11fcc30/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/templates/topology-page-template.html
index be65772,da7929c..c379720
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@@ -59,12 -59,7 +59,12 @@@
            </span>
          </th>
          <th>
-           <span class="tip above" title="Number of nimbus hosts on which this topology's code is replicated. ">
++          <span cdata-toggle="tooltip" data-placement="above" title="Number of nimbus hosts on which this topology's code is replicated. ">
 +            Replication count
 +          </span>
 +        </th>
 +        <th>
-           <span class="tip left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
+           <span data-toggle="tooltip" data-placement="left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
              Scheduler Info
            </span>
          </th>


[35/50] [abbrv] storm git commit: Merge remote-tracking branch 'upstream/master' into STORM-166

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into STORM-166

Conflicts:
	conf/defaults.yaml
	storm-core/src/clj/backtype/storm/cluster.clj
	storm-core/src/clj/backtype/storm/daemon/nimbus.clj
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/test/clj/backtype/storm/cluster_test.clj
	storm-core/test/clj/backtype/storm/security/auth/auth_test.clj


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

Branch: refs/heads/master
Commit: 575871822b116608f47a0a7a4cd3b4b17df9a672
Parents: f7205d2 bb8d48d
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Mar 19 15:04:43 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Mar 19 15:24:50 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.md                                    |   21 +
 DEVELOPER.md                                    |    7 +
 README.markdown                                 |    2 +
 SECURITY.md                                     |   59 +-
 STORM-UI-REST-API.md                            |   12 +
 bin/storm                                       |  584 +--------
 bin/storm.py                                    |  543 ++++++++
 conf/defaults.yaml                              |    3 +
 conf/storm-env.sh                               |   24 +
 dev-tools/test-ns.py                            |   17 +
 docs/documentation/Clojure-DSL.md               |    4 +-
 docs/documentation/Command-line-client.md       |    2 +-
 docs/documentation/Common-patterns.md           |    6 +-
 docs/documentation/Concepts.md                  |   48 +-
 docs/documentation/Configuration.md             |    4 +-
 docs/documentation/Distributed-RPC.md           |    2 +-
 .../Guaranteeing-message-processing.md          |    6 +-
 docs/documentation/Hooks.md                     |    6 +-
 docs/documentation/Local-mode.md                |    4 +-
 docs/documentation/Powered-By.md                |  106 +-
 ...unning-topologies-on-a-production-cluster.md |    6 +-
 .../Serialization-(prior-to-0.6.0).md           |    4 +-
 docs/documentation/Serialization.md             |    2 +-
 docs/documentation/Structure-of-the-codebase.md |    8 +-
 docs/documentation/Transactional-topologies.md  |   18 +-
 docs/documentation/Tutorial.md                  |    8 +-
 ...nding-the-parallelism-of-a-Storm-topology.md |   16 +-
 external/README.md                              |   18 +
 external/storm-jdbc/README.md                   |   84 +-
 .../apache/storm/jdbc/bolt/JdbcInsertBolt.java  |   36 +-
 .../org/apache/storm/jdbc/common/Column.java    |    3 +-
 .../apache/storm/jdbc/common/JdbcClient.java    |   52 +-
 .../jdbc/mapper/SimpleJdbcLookupMapper.java     |    2 +-
 .../storm/jdbc/trident/state/JdbcState.java     |   18 +-
 .../storm/jdbc/common/JdbcClientTest.java       |   39 +-
 .../jdbc/topology/UserPersistanceTopology.java  |    2 +-
 .../ExponentialBackoffMsgRetryManager.java      |    2 +-
 .../src/jvm/storm/kafka/bolt/KafkaBolt.java     |    7 +-
 .../test/storm/kafka/bolt/KafkaBoltTest.java    |    2 +-
 .../storm/redis/trident/state/RedisState.java   |    2 +-
 storm-core/pom.xml                              |    3 +-
 storm-core/src/clj/backtype/storm/cluster.clj   |   60 +-
 storm-core/src/clj/backtype/storm/config.clj    |    7 +-
 storm-core/src/clj/backtype/storm/converter.clj |  201 +++
 .../src/clj/backtype/storm/daemon/common.clj    |   10 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |    4 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |    5 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  150 ++-
 .../clj/backtype/storm/daemon/supervisor.clj    |   16 +-
 .../src/clj/backtype/storm/daemon/task.clj      |   11 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |    3 +-
 storm-core/src/clj/backtype/storm/stats.clj     |   78 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |   32 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |   42 +-
 storm-core/src/genthrift.sh                     |   13 +-
 storm-core/src/java_license_header.txt          |   17 +
 storm-core/src/jvm/backtype/storm/Config.java   |   40 +-
 .../jvm/backtype/storm/ConfigValidation.java    |    8 +-
 .../src/jvm/backtype/storm/StormSubmitter.java  |   88 +-
 .../storm/coordination/BatchBoltExecutor.java   |    4 +-
 .../backtype/storm/generated/Assignment.java    |  983 ++++++++++++++
 .../storm/generated/ClusterWorkerHeartbeat.java |  673 ++++++++++
 .../backtype/storm/generated/ExecutorStats.java |  105 +-
 .../jvm/backtype/storm/generated/NodeInfo.java  |  556 ++++++++
 .../jvm/backtype/storm/generated/StormBase.java | 1211 ++++++++++++++++++
 .../storm/generated/SupervisorInfo.java         | 1182 +++++++++++++++++
 .../storm/generated/TopologyActionOptions.java  |  387 ++++++
 .../storm/generated/TopologyStatus.java         |   68 +
 .../backtype/storm/messaging/netty/Client.java  |   10 +-
 .../backtype/storm/messaging/netty/Context.java |   33 +-
 .../auth/DefaultHttpCredentialsPlugin.java      |   19 +-
 .../storm/security/auth/ITransportPlugin.java   |    4 +-
 .../storm/security/auth/ReqContext.java         |   28 +-
 .../security/auth/SaslTransportPlugin.java      |    3 +-
 .../security/auth/SimpleTransportPlugin.java    |    5 +-
 .../storm/security/auth/TBackoffConnect.java    |    4 +-
 .../storm/security/auth/ThriftClient.java       |   12 +-
 .../authorizer/ImpersonationAuthorizer.java     |  154 +++
 .../auth/authorizer/SimpleACLAuthorizer.java    |   55 +-
 .../auth/digest/DigestSaslTransportPlugin.java  |    6 +-
 .../auth/digest/ServerCallbackHandler.java      |   21 +-
 .../kerberos/KerberosSaslTransportPlugin.java   |    9 +-
 .../auth/kerberos/ServerCallbackHandler.java    |   38 +-
 .../DefaultSerializationDelegate.java           |   11 +-
 .../GzipBridgeSerializationDelegate.java        |    7 +-
 .../GzipSerializationDelegate.java              |   10 +-
 .../serialization/SerializationDelegate.java    |    2 +-
 .../ThriftSerializationDelegate.java            |   52 +
 .../backtype/storm/topology/BoltDeclarer.java   |    4 +
 .../backtype/storm/topology/InputDeclarer.java  |  128 ++
 .../storm/topology/TopologyBuilder.java         |    6 +-
 .../jvm/backtype/storm/utils/DRPCClient.java    |    2 +-
 .../jvm/backtype/storm/utils/LocalState.java    |    4 +-
 .../jvm/backtype/storm/utils/NimbusClient.java  |   16 +-
 .../src/jvm/backtype/storm/utils/Utils.java     |   82 +-
 .../backtype/storm/utils/VersionedStore.java    |    9 +-
 storm-core/src/jvm/storm/trident/Stream.java    |    2 +-
 .../src/jvm/storm/trident/TridentTopology.java  |    6 +-
 storm-core/src/py/storm/Nimbus-remote           |    0
 storm-core/src/py/storm/ttypes.py               |  827 +++++++++++-
 storm-core/src/py_license_header.txt            |   18 +
 storm-core/src/storm.thrift                     |   51 +
 .../templates/component-page-template.html      |    2 +-
 .../templates/topology-page-template.html       |    4 +-
 .../test/clj/backtype/storm/cluster_test.clj    |   35 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |    6 +-
 .../auth/DefaultHttpCredentialsPlugin_test.clj  |   16 +-
 .../backtype/storm/security/auth/auth_test.clj  |  146 ++-
 .../GzipBridgeSerializationDelegateTest.java    |    6 +-
 .../ThriftBridgeSerializationDelegateTest.java  |   60 +
 110 files changed, 8564 insertions(+), 1095 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/STORM-UI-REST-API.md
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/conf/defaults.yaml
----------------------------------------------------------------------
diff --cc conf/defaults.yaml
index 305b31c,78e89bb..49584f2
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@@ -47,11 -47,10 +47,12 @@@ storm.auth.simple-white-list.users: [
  storm.auth.simple-acl.users: []
  storm.auth.simple-acl.users.commands: []
  storm.auth.simple-acl.admins: []
 +storm.meta.serialization.delegate: "backtype.storm.serialization.DefaultSerializationDelegate"
 +storm.codedistributor.class: "backtype.storm.codedistributor.LocalFileSystemCodeDistributor"
+ storm.meta.serialization.delegate: "backtype.storm.serialization.ThriftSerializationDelegate"
  
  ### nimbus.* configs are for the master
 -nimbus.host: "localhost"
 +nimbus.seeds : ["localhost:6627"]
  nimbus.thrift.port: 6627
  nimbus.thrift.threads: 64
  nimbus.thrift.max_buffer_size: 1048576

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/clj/backtype/storm/cluster.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/cluster.clj
index c48e3c1,7987a30..333feec
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@@ -15,13 -15,14 +15,15 @@@
  ;; limitations under the License.
  
  (ns backtype.storm.cluster
-   (:import [org.apache.zookeeper.data Stat ACL Id])
+   (:import [org.apache.zookeeper.data Stat ACL Id]
 -           [backtype.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials]
++           [backtype.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials NimbusSummary]
+            [java.io Serializable])
    (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
    (:import [backtype.storm.utils Utils])
    (:import [java.security MessageDigest])
    (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
 +  (:import [backtype.storm.nimbus NimbusInfo])
-   (:use [backtype.storm util log config])
+   (:use [backtype.storm util log config converter])
    (:require [backtype.storm [zookeeper :as zk]])
    (:require [backtype.storm.daemon [common :as common]]))
  
@@@ -348,25 -322,6 +350,25 @@@
            (swap! assignment-version-callback assoc storm-id callback))
          (get-version cluster-state (assignment-path storm-id) (not-nil? callback)))
  
 +      (code-distributor
 +        [this callback]
 +        (when callback
 +          (reset! code-distributor-callback callback))
 +        (get-children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback)))
 +
 +      (nimbuses
 +        [this]
-         (map #(maybe-deserialize (get-data cluster-state (nimbus-path %1) false))
++        (map #(maybe-deserialize (get-data cluster-state (nimbus-path %1) false) NimbusSummary)
 +          (get-children cluster-state NIMBUSES-SUBTREE false)))
 +
 +      (add-nimbus-host!
 +        [this nimbus-id nimbus-summary]
 +        (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
 +
 +      (code-distributor-info
 +        [this storm-id]
 +        (map (fn [nimbus-info] (NimbusInfo/parse nimbus-info)) (get-children cluster-state (code-distributor-path storm-id) false)))
 +
        (active-storms
          [this]
          (get-children cluster-state STORMS-SUBTREE false))
@@@ -465,13 -428,9 +475,14 @@@
  
        (set-assignment!
          [this storm-id info]
-         (set-data cluster-state (assignment-path storm-id) (Utils/serialize info) acls))
+         (let [thrift-assignment (thriftify-assignment info)]
+           (set-data cluster-state (assignment-path storm-id) (Utils/serialize thrift-assignment) acls)))
  
 +      (setup-code-distributor!
 +        [this storm-id nimbusInfo]
 +        (mkdirs cluster-state (code-distributor-path storm-id) acls)
 +        (mkdirs cluster-state (str (code-distributor-path storm-id) "/" (.toHostPortString nimbusInfo)) acls))
 +
        (remove-storm!
          [this storm-id]
          (delete-node cluster-state (assignment-path storm-id))

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

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index eee0417,d1a1a59..8a2c0fb
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@@ -91,9 -77,9 +91,10 @@@
  (defn nimbus-data [conf inimbus]
    (let [forced-scheduler (.getForcedScheduler inimbus)]
      {:conf conf
 +     :nimbus-host-port-info (NimbusInfo/fromConf conf)
       :inimbus inimbus
       :authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
+      :impersonation-authorization-handler (mk-authorization-handler (conf NIMBUS-IMPERSONATION-AUTHORIZER) conf)
       :submitted-count (atom 0)
       :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
                                                                         (Utils/isZkAuthenticationConfiguredStormServer
@@@ -221,11 -195,11 +212,12 @@@
    ([nimbus storm-id event]
       (transition! nimbus storm-id event false))
    ([nimbus storm-id event error-on-no-transition?]
 -     (locking (:submit-lock nimbus)
 +    (is-leader nimbus)
 +    (locking (:submit-lock nimbus)
         (let [system-events #{:startup}
               [event & event-args] (if (keyword? event) [event] event)
-              status (topology-status nimbus storm-id)]
+              storm-base (-> nimbus :storm-cluster-state  (.storm-base storm-id nil))
+              status (:status storm-base)]
           ;; handles the case where event was scheduled but topology has been removed
           (if-not status
             (log-message "Cannot apply event " event " to " storm-id " because topology no longer exists")
@@@ -347,32 -323,9 +341,32 @@@
     (FileUtils/cleanDirectory (File. stormroot))
     (setup-jar conf tmp-jar-location stormroot)
     (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
-    (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/serialize storm-conf))
+    (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/javaSerialize storm-conf))
 +   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
     ))
  
 +(defn- wait-for-desired-code-replication [nimbus conf storm-id]
 +  (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
 +        max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
 +        total-wait-time (atom 0)
 +        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
 +  (if (:code-distributor nimbus)
 +    (while (and (> min-replication-count @current-replication-count)
 +             (or (= -1 max-replication-wait-time)
 +               (< @total-wait-time max-replication-wait-time)))
 +        (sleep-secs 1)
 +        (log-debug "waiting for desired replication to be achieved.
 +          min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
 +          "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
 +        (swap! total-wait-time inc)
 +        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
 +  (if (< min-replication-count @current-replication-count)
 +    (log-message "desired replication count "  min-replication-count " achieved,
 +      current-replication-count" @current-replication-count)
 +    (log-message "desired replication count of "  min-replication-count " not achieved but we have hit the max wait time
 +      so moving on with replication count = " @current-replication-count)
 +    )))
 +
  (defn- read-storm-topology [conf storm-id]
    (let [stormroot (master-stormdist-root conf storm-id)]
      (Utils/deserialize
@@@ -708,10 -661,9 +702,10 @@@
  ;; only keep existing slots that satisfy one of those slots. for rest, reassign them across remaining slots
  ;; edge case for slots with no executor timeout but with supervisor timeout... just treat these as valid slots that can be reassigned to. worst comes to worse the executor will timeout and won't assign here next time around
  (defnk mk-assignments [nimbus :scratch-topology-id nil]
 -  (let [conf (:conf nimbus)
 +  (if (is-leader nimbus :throw-exception false)
 +    (let [conf (:conf nimbus)
          storm-cluster-state (:storm-cluster-state nimbus)
-         ^INimbus inimbus (:inimbus nimbus) 
+         ^INimbus inimbus (:inimbus nimbus)
          ;; read all the topologies
          topology-ids (.active-storms storm-cluster-state)
          topologies (into {} (for [tid topology-ids]
@@@ -775,14 -727,14 +769,14 @@@
      (->> new-assignments
            (map (fn [[topology-id assignment]]
              (let [existing-assignment (get existing-assignments topology-id)]
-               [topology-id (map to-worker-slot (newly-added-slots existing-assignment assignment))] 
+               [topology-id (map to-worker-slot (newly-added-slots existing-assignment assignment))]
                )))
            (into {})
 -          (.assignSlots inimbus topologies))
 -    ))
 +          (.assignSlots inimbus topologies)))
 +    (log-message "not a leader, skipping assignments")))
  
  (defn- start-storm [nimbus storm-name storm-id topology-initial-status]
-   {:pre [(#{:active :inactive} topology-initial-status)]}                
+   {:pre [(#{:active :inactive} topology-initial-status)]}
    (let [storm-cluster-state (:storm-cluster-state nimbus)
          conf (:conf nimbus)
          storm-conf (read-storm-conf conf storm-id)
@@@ -1049,24 -1009,9 +1058,23 @@@
    (let [nimbus (nimbus-data conf inimbus)
         principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)]
      (.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
 +
 +    ;add to nimbuses
-     (.add-nimbus-host! (:storm-cluster-state nimbus)
-       (.toHostPortString (:nimbus-host-port-info nimbus))
-       {
-         :host (.getHost (:nimbus-host-port-info nimbus))
-         :port (.getPort (:nimbus-host-port-info nimbus))
-         :start-time-secs (current-time-secs)
-         :version (str (VersionInfo/getVersion))
-         })
++    (.add-nimbus-host! (:storm-cluster-state nimbus) (.toHostPortString (:nimbus-host-port-info nimbus))
++      (NimbusSummary.
++        (.getHost (:nimbus-host-port-info nimbus))
++        (.getPort (:nimbus-host-port-info nimbus))
++        (current-time-secs)
++        false ;is-leader
++        (str (VersionInfo/getVersion))))
 +
 +    (.addToLeaderLockQueue (:leader-elector nimbus))
      (cleanup-corrupt-topologies! nimbus)
 -    (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
 -      (transition! nimbus storm-id :startup))
 +    ;register call back for code-distributor
 +    (.code-distributor (:storm-cluster-state nimbus) (fn [] (sync-code conf nimbus)))
 +    (when (is-leader nimbus :throw-exception false)
 +      (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
 +        (transition! nimbus storm-id :startup)))
      (schedule-recurring (:timer nimbus)
                          0
                          (conf NIMBUS-MONITOR-FREQ-SECS)
@@@ -1317,14 -1253,8 +1325,16 @@@
                                                                  (count (:used-ports info))
                                                                  id )
                                              ))
 -              nimbus-uptime ((:uptime nimbus))
                bases (topology-bases storm-cluster-state)
 +              nimbuses (.nimbuses storm-cluster-state)
-               nimbuses (map #(NimbusSummary. (:host %1) (:port %1) (time-delta (:start-time-secs %1))
-                                (let [leader (.getLeader (:leader-elector nimbus))]
-                                  (and (= (.getHost leader) (:host %1)) (= (.getPort leader) (:port %1))))
-                                (:version %1))
-                          nimbuses
-                          )
++
++              ;;update the isLeader field for each nimbus summary
++              _ (let [leader (.getLeader (:leader-elector nimbus))
++                      leader-host (.getHost leader)
++                      leader-port (.getPort leader)]
++                  (doseq [nimbus-summary nimbuses]
++                    (.set_isLeader nimbus-summary (and (= leader-host (.get_host nimbus-summary)) (= leader-port (.get_port nimbus-summary))))))
++
                topology-summaries (dofor [[id base] bases :when base]
  	                                  (let [assignment (.assignment-info storm-cluster-state id nil)
                                                  topo-summ (TopologySummary. id

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index d315925,bc8b999..4fc219e
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@@ -376,7 -377,7 +382,7 @@@
              master-code-dir (if (contains? storm-code-map :data) (storm-code-map :data))
              stormroot (supervisor-stormdist-root conf storm-id)]
          (if-not (or (contains? downloaded-storm-ids storm-id) (.exists (File. stormroot)) (nil? master-code-dir))
--          (download-storm-code conf storm-id master-code-dir download-lock))
++          (download-storm-code conf storm-id master-code-dir supervisor download-lock))
          ))
  
      (wait-for-workers-launch
@@@ -725,7 -728,7 +731,7 @@@
         first ))
  
  (defmethod download-storm-code
--    :local [conf storm-id master-code-dir download-lock]
++    :local [conf storm-id master-code-dir supervisor download-lock]
      (let [stormroot (supervisor-stormdist-root conf storm-id)]
        (locking download-lock
              (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot))

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/clj/backtype/storm/thrift.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/thrift.clj
index 474ea67,6445f46..8f4c659
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@@ -20,9 -22,9 +22,10 @@@
              StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface
              ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
              GlobalStreamId ComponentObject ComponentObject$_Fields
-             ShellComponent])
+             ShellComponent SupervisorInfo])
    (:import [backtype.storm.utils Utils NimbusClient])
    (:import [backtype.storm Constants])
++  (:import [backtype.storm.security.auth ReqContext])
    (:import [backtype.storm.grouping CustomStreamGrouping])
    (:import [backtype.storm.topology TopologyBuilder])
    (:import [backtype.storm.clojure RichShellBolt RichShellSpout])
@@@ -84,13 -88,17 +89,15 @@@
  (defmacro with-configured-nimbus-connection
    [client-sym & body]
    `(let [conf# (read-storm-config)
-          nimbusClient# (NimbusClient/getConfiguredClient conf#)
 -         host# (conf# NIMBUS-HOST)
 -         port# (conf# NIMBUS-THRIFT-PORT)]
 -    (with-nimbus-connection [~client-sym host# port#]
 -      ~@body)))
++         context# (ReqContext/context)
++         user# (if (.principal context#) (.getName (.principal context#)))
++         nimbusClient# (NimbusClient/getConfiguredClientAs conf# user#)
 +         ~client-sym (.getClient nimbusClient#)
 +         conn# (.transport nimbusClient#)
 +         ]
 +     (try
 +       ~@body
 +     (finally (.close conn#)))))
  
  (defn direct-output-fields
    [fields]

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index 0f93f23,c64f35d..553434e
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -931,18 -929,19 +947,19 @@@
    (GET "/api/v1/token" [ & m]
         (json-response (format "{\"antiForgeryToken\": \"%s\"}" *anti-forgery-token*) (:callback m) :serialize-fn identity))
    (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id & m]
 +    (thrift/with-configured-nimbus-connection nimbus
+     (assert-authorized-user servlet-request "activate" (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)]
-         (assert-authorized-user servlet-request "activate" (topology-config id))
          (.activate nimbus name)
          (log-message "Activating topology '" name "'")))
 -    (json-response (topology-op-response id "deactivate") (m "callback")))
 +    (json-response (topology-op-response id "activate") (m "callback")))
    (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id & m]
 +    (thrift/with-configured-nimbus-connection nimbus
+     (assert-authorized-user servlet-request "deactivate" (topology-config id))
 -    (with-nimbus nimbus
        (let [tplg (->> (doto
                          (GetInfoOptions.)
                          (.set_num_err_choice NumErrorsChoice/NONE))
@@@ -953,7 -951,8 +969,8 @@@
          (log-message "Deactivating topology '" name "'")))
      (json-response (topology-op-response id "deactivate") (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))
@@@ -972,7 -970,8 +988,8 @@@
          (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
      (json-response (topology-op-response id "rebalance") (m "callback")))
    (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
+     (assert-authorized-user servlet-request "killTopology" (topology-config id))
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
        (let [tplg (->> (doto
                          (GetInfoOptions.)
                          (.set_num_err_choice NumErrorsChoice/NONE))

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

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index 39d3895,b171353..071d2b6
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@@ -40,31 -30,23 +40,36 @@@ public class NimbusClient extends Thrif
      private Nimbus.Client _client;
      private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class);
  
+ 
      public static NimbusClient getConfiguredClient(Map conf) {
 -        try {
 -            String nimbusHost = (String) conf.get(Config.NIMBUS_HOST);
 -            return new NimbusClient(conf, nimbusHost);
 -        } catch (TTransportException ex) {
 -            throw new RuntimeException(ex);
 -        }
++        return getConfiguredClientAs(conf, null);
+     }
+ 
+     public static NimbusClient getConfiguredClientAs(Map conf, String asUser) {
 -        try {
 -            String nimbusHost = (String) conf.get(Config.NIMBUS_HOST);
 -            return new NimbusClient(conf, nimbusHost, null, null, asUser);
 -        } catch (TTransportException ex) {
 -            throw new RuntimeException(ex);
 +        List<String> seeds = (List<String>) conf.get(Config.NIMBUS_SEEDS);
 +        for(String seed : seeds) {
 +            String[] split = seed.split(":");
 +            String host = split[0];
 +            int port = Integer.parseInt(split[1]);
 +            try {
 +                NimbusClient client = new NimbusClient(conf,host,port);
 +                ClusterSummary clusterInfo = client.getClient().getClusterInfo();
 +                List<NimbusSummary> nimbuses = clusterInfo.get_nimbuses();
 +                if(nimbuses != null) {
 +                    for(NimbusSummary nimbusSummary : nimbuses) {
 +                        if(nimbusSummary.is_isLeader()) {
-                             return new NimbusClient(conf, nimbusSummary.get_host(), nimbusSummary.get_port());
++                            return new NimbusClient(conf, nimbusSummary.get_host(), nimbusSummary.get_port(), null, asUser);
 +                        }
 +                    }
 +                }
 +                throw new RuntimeException("Found nimbuses " + nimbuses + " none of which is elected as leader, please try " +
 +                        "again after some time.");
 +            } catch (Exception e) {
 +                LOG.warn("Ignoring exception while trying to get leader nimbus info from " + seed, e);
 +            }
          }
 +        throw new RuntimeException("Could not find leader nimbus from seed hosts " + seeds +". " +
 +                "Did you specify a valid list of nimbus host:port for config " + Config.NIMBUS_SEEDS);
      }
  
      public NimbusClient(Map conf, String host, int port) throws TTransportException {

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/storm.thrift
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/cluster_test.clj
index 85aaf3b,98eae68..ffc72af
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@@ -169,12 -168,10 +169,14 @@@
  (deftest test-storm-cluster-state-basics
    (with-inprocess-zookeeper zk-port
      (let [state (mk-storm-state zk-port)
-           assignment1 (Assignment. "/aaa" {} {1 [2 2002 1]} {})
-           assignment2 (Assignment. "/aaa" {} {1 [2 2002]} {})
+           assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {})
+           assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {})
 +          nimbusInfo1 (NimbusInfo. "nimbus1" 6667 false)
 +          nimbusInfo2 (NimbusInfo. "nimbus2" 6667 false)
-           base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "")
-           base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "")]
++          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)]
        (is (= [] (.assignments state nil)))
        (.set-assignment! state "storm1" assignment1)
        (is (= assignment1 (.assignment-info state "storm1" nil)))
@@@ -204,21 -201,6 +206,21 @@@
        (.set-credentials! state "storm1" {"b" "b"} {})
        (is (= {"b" "b"} (.credentials state "storm1" nil)))
  
 +      (is (= [] (.code-distributor state nil)))
 +      (.setup-code-distributor! state "storm1" nimbusInfo1)
 +      (is (= ["storm1"] (.code-distributor state nil)))
 +      (is (= [nimbusInfo1] (.code-distributor-info state "storm1")))
 +      (.setup-code-distributor! state "storm1" nimbusInfo2)
 +      (is (= #{nimbusInfo1 nimbusInfo2} (set (.code-distributor-info state "storm1"))))
 +      (.remove-storm! state "storm1")
 +      (is (= [] (.code-distributor state nil)))
 +
 +      (is (= [] (.nimbuses state)))
-       (.add-nimbus-host! state "host:port" nimbusInfo1)
-       (is (= [nimbusInfo1] (.nimbuses state)))
-       (.add-nimbus-host! state "host1:port" nimbusInfo2)
-       (is (= #{nimbusInfo1 nimbusInfo2} (set (.nimbuses state))))
++      (.add-nimbus-host! state "nimbus1:port" nimbusSummary1)
++      (is (= [nimbusSummary1] (.nimbuses state)))
++      (.add-nimbus-host! state "nimbus2:port" nimbusSummary2)
++      (is (= #{nimbusSummary1 nimbusSummary2} (set (.nimbuses state))))
 +
        ;; TODO add tests for task info and task heartbeat setting and getting
        (.disconnect state)
        )))

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

http://git-wip-us.apache.org/repos/asf/storm/blob/57587182/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
index 416dddf,ddd5e03..cb4ccc8
--- a/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/auth_test.clj
@@@ -117,11 -119,12 +119,11 @@@
           (^TopologyInfo getTopologyInfo [this ^String storm-id]))))
    ([conf inimbus]
       (dummy-service-handler conf inimbus nil)))
-      
  
- (defn launch-server [server-port login-cfg aznClass transportPluginClass serverConf] 
+ 
+ (defn launch-server [server-port login-cfg aznClass transportPluginClass serverConf]
    (let [conf1 (merge (read-storm-config)
-                      {NIMBUS-AUTHORIZER aznClass 
+                      {NIMBUS-AUTHORIZER aznClass
 -                      NIMBUS-HOST "localhost"
                        NIMBUS-THRIFT-PORT server-port
                        STORM-THRIFT-TRANSPORT-PLUGIN transportPluginClass})
          conf2 (if login-cfg (merge conf1 {"java.security.auth.login.config" login-cfg}) conf1)


[26/50] [abbrv] storm git commit: Nimbus discovery through new thrift API instead of using zk.

Posted by bo...@apache.org.
Nimbus discovery through new thrift API instead of using zk.


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

Branch: refs/heads/master
Commit: 27ad1fb25caa5eb20f477c89cf57c801b6cec3c2
Parents: 0b454e8
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 15:47:38 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 15:47:38 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/thrift.clj  |  5 +--
 storm-core/src/clj/backtype/storm/ui/core.clj | 37 ++++++++--------------
 2 files changed, 16 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/27ad1fb2/storm-core/src/clj/backtype/storm/thrift.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/thrift.clj b/storm-core/src/clj/backtype/storm/thrift.clj
index 0cac7b8..474ea67 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -84,8 +84,9 @@
 (defmacro with-configured-nimbus-connection
   [client-sym & body]
   `(let [conf# (read-storm-config)
-         ~client-sym (NimbusClient/getConfiguredClient conf#)
-         conn# (.transport ~client-sym)
+         nimbusClient# (NimbusClient/getConfiguredClient conf#)
+         ~client-sym (.getClient nimbusClient#)
+         conn# (.transport nimbusClient#)
          ]
      (try
        ~@body

http://git-wip-us.apache.org/repos/asf/storm/blob/27ad1fb2/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 94b0311..8fd22a6 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -47,17 +47,6 @@
 
 (def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
 
-(defmacro with-nimbus
-  [nimbus-sym & body]
-  `(let [leader-elector# (zk-leader-elector *STORM-CONF*)
-    leader-nimbus# (.getLeader leader-elector#)
-    host# (.getHost leader-nimbus#)
-    port# (.getPort leader-nimbus#)
-    no-op# (.close leader-elector#)]
-  (thrift/with-nimbus-connection
-     [~nimbus-sym host# port#]
-     ~@body)))
-
 (defn assert-authorized-user
   ([servlet-request op]
     (assert-authorized-user servlet-request op nil))
@@ -468,7 +457,7 @@
 
 (defn mk-visualization-data
   [id window include-sys?]
-  (with-nimbus
+  (thrift/with-configured-nimbus-connection
     nimbus
     (let [window (if window window ":all-time")
           topology (.getTopology ^Nimbus$Client nimbus id)
@@ -490,12 +479,12 @@
        spout-comp-summs bolt-comp-summs window id))))
 
 (defn cluster-configuration []
-  (with-nimbus nimbus
+  (thrift/with-configured-nimbus-connection nimbus
     (.getNimbusConf ^Nimbus$Client nimbus)))
 
 (defn cluster-summary
   ([user]
-     (with-nimbus nimbus
+     (thrift/with-configured-nimbus-connection nimbus
         (cluster-summary (.getClusterInfo ^Nimbus$Client nimbus) user)))
   ([^ClusterSummary summ user]
      (let [sups (.get_supervisors summ)
@@ -519,7 +508,7 @@
 
 (defn nimbus-summary
   ([]
-    (with-nimbus nimbus
+    (thrift/with-configured-nimbus-connection nimbus
       (nimbus-summary
         (.get_nimbuses (.getClusterInfo ^Nimbus$Client nimbus)))))
   ([nimbuses]
@@ -535,7 +524,7 @@
 
 (defn supervisor-summary
   ([]
-   (with-nimbus nimbus
+   (thrift/with-configured-nimbus-connection nimbus
                 (supervisor-summary
                   (.get_supervisors (.getClusterInfo ^Nimbus$Client nimbus)))))
   ([summs]
@@ -549,7 +538,7 @@
 
 (defn all-topologies-summary
   ([]
-   (with-nimbus
+   (thrift/with-configured-nimbus-connection
      nimbus
      (all-topologies-summary
        (.get_topologies (.getClusterInfo ^Nimbus$Client nimbus)))))
@@ -665,7 +654,7 @@
         "failed" (get-in stats [:failed k])})))
 
 (defn topology-page [id window include-sys? user]
-  (with-nimbus nimbus
+  (thrift/with-configured-nimbus-connection nimbus
     (let [window (if window window ":all-time")
           window-hint (window-hint window)
           summ (.getTopologyInfo ^Nimbus$Client nimbus id)
@@ -848,7 +837,7 @@
 
 (defn component-page
   [topology-id component window include-sys? user]
-  (with-nimbus nimbus
+  (thrift/with-configured-nimbus-connection nimbus
     (let [window (if window window ":all-time")
           summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
           topology (.getTopology ^Nimbus$Client nimbus topology-id)
@@ -873,7 +862,7 @@
        spec errors))))
 
 (defn topology-config [topology-id]
-  (with-nimbus nimbus
+  (thrift/with-configured-nimbus-connection nimbus
      (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))))
 
 (defn check-include-sys?
@@ -923,7 +912,7 @@
          (assert-authorized-user servlet-request "getTopology" (topology-config id))
          (json-response (component-page id component (:window m) (check-include-sys? (:sys m)) user) (:callback m))))
   (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id]
-    (with-nimbus nimbus
+    (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
             name (.get_name tplg)]
         (assert-authorized-user servlet-request "activate" (topology-config id))
@@ -931,7 +920,7 @@
         (log-message "Activating topology '" name "'")))
     (resp/redirect (str "/api/v1/topology/" id)))
   (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id]
-    (with-nimbus nimbus
+    (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
             name (.get_name tplg)]
         (assert-authorized-user servlet-request "deactivate" (topology-config id))
@@ -939,7 +928,7 @@
         (log-message "Deactivating topology '" name "'")))
     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
-    (with-nimbus nimbus
+    (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
             name (.get_name tplg)
             options (RebalanceOptions.)]
@@ -949,7 +938,7 @@
         (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
   (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
-    (with-nimbus nimbus
+    (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
             name (.get_name tplg)
             options (KillOptions.)]


[17/50] [abbrv] storm git commit: STORM-166: renaming code distributor.

Posted by bo...@apache.org.
STORM-166: renaming code distributor.


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

Branch: refs/heads/master
Commit: c1e8782f7cfb3524b6c0d51fe48669fd09b87527
Parents: 58667be
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Jan 9 13:52:33 2015 -0500
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Jan 9 13:52:33 2015 -0500

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 24 ++++++++++----------
 .../clj/backtype/storm/daemon/supervisor.clj    | 16 ++++++-------
 .../test/clj/backtype/storm/nimbus_test.clj     |  2 +-
 .../storm/security/auth/nimbus_auth_test.clj    |  1 -
 .../test/clj/backtype/storm/supervisor_test.clj |  2 +-
 5 files changed, 22 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/c1e8782f/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 1f93be6..d1f5c31 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -62,7 +62,7 @@
     scheduler
     ))
 
-(defmulti mk-bt-tracker cluster-mode)
+(defmulti mk-code-distributor cluster-mode)
 (defmulti sync-code cluster-mode)
 
 (defnk is-leader [nimbus :throw-exception true]
@@ -100,7 +100,7 @@
                                  ))
      :scheduler (mk-scheduler conf inimbus)
      :leader-elector (zk-leader-elector conf)
-     :bt-tracker (mk-bt-tracker conf)
+     :bt-tracker (mk-code-distributor conf)
      :id->sched-status (atom {})
      :cred-renewers (AuthUtils/GetCredentialRenewers conf)
      :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)
@@ -336,15 +336,15 @@
    (setup-jar conf tmp-jar-location stormroot)
    (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
    (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/serialize storm-conf))
-   (if (:bt-tracker nimbus) (.upload (:bt-tracker nimbus) stormroot storm-id))
+   (if (:code-distributor nimbus) (.upload (:code-distributor nimbus) stormroot storm-id))
    ))
 
 (defn- wait-for-desired-code-replication [nimbus conf storm-id]
   (let [min-replication-count (conf NIMBUS-MIN-REPLICATION-COUNT)
         max-replication-wait-time (conf NIMBUS-MAX-REPLICATION-WAIT-TIME-SEC)
         total-wait-time (atom 0)
-        current-replication-count (atom (if (:bt-tracker nimbus) (.getReplicationCount (:bt-tracker nimbus) storm-id) 0))]
-  (if (:bt-tracker nimbus)
+        current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
+  (if (:code-distributor nimbus)
     (while (and (> min-replication-count @current-replication-count)
              (or (= -1 max-replication-wait-time)
                (< @total-wait-time max-replication-wait-time)))
@@ -353,7 +353,7 @@
           min-replication-count = " min-replication-count  " max-replication-wait-time = " max-replication-wait-time
           "current-replication-count = " @current-replication-count " total-wait-time " @total-wait-time)
         (swap! total-wait-time inc)
-        (reset! current-replication-count  (.getReplicationCount (:bt-tracker nimbus) storm-id))))
+        (reset! current-replication-count  (.getReplicationCount (:code-distributor nimbus) storm-id))))
   (if (< min-replication-count @current-replication-count)
     (log-message "desired replication count "  min-replication-count " achieved,
       current-replication-count" @current-replication-count)
@@ -897,7 +897,7 @@
         (when-not (empty? to-cleanup-ids)
           (doseq [id to-cleanup-ids]
             (log-message "Cleaning up " id)
-            (if (:bt-tracker nimbus) (.cleanup (:bt-tracker nimbus) id))
+            (if (:code-distributor nimbus) (.cleanup (:code-distributor nimbus) id))
             (.teardown-heartbeats! storm-cluster-state id)
             (.teardown-topology-errors! storm-cluster-state id)
             (rmr (master-stormdist-root conf id))
@@ -1363,19 +1363,19 @@
         (.cleanup (:downloaders nimbus))
         (.cleanup (:uploaders nimbus))
         (.close (:leader-elector nimbus))
-        (if (:bt-tracker nimbus) (.close (:bt-tracker nimbus) (:conf nimbus)))
+        (if (:code-distributor nimbus) (.close (:code-distributor nimbus) (:conf nimbus)))
         (log-message "Shut down master")
         )
       DaemonCommon
       (waiting? [this]
         (timer-waiting? (:timer nimbus))))))
 
-(defmethod mk-bt-tracker :distributed [conf]
+(defmethod mk-code-distributor :distributed [conf]
   (let [code-distributor (new-instance (conf STORM-CODE-DISTRIBUTOR-CLASS))]
     (.prepare code-distributor conf)
     code-distributor))
 
-(defmethod mk-bt-tracker :local [conf]
+(defmethod mk-code-distributor :local [conf]
   nil)
 
 (defn download-code [conf nimbus storm-id host port]
@@ -1386,8 +1386,8 @@
         local-meta-file-path (master-storm-metafile-path tmp-root)]
     (FileUtils/forceMkdir (File. tmp-root))
     (Utils/downloadFromHost conf remote-meta-file-path local-meta-file-path host port)
-    (if (:bt-tracker nimbus)
-      (.download (:bt-tracker nimbus) storm-id (File. local-meta-file-path)))
+    (if (:code-distributor nimbus)
+      (.download (:code-distributor nimbus) storm-id (File. local-meta-file-path)))
     (if (.exists (File. storm-root)) (FileUtils/forceDelete (File. storm-root)))
     (FileUtils/moveDirectory (File. tmp-root) (File. storm-root))
     (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus))))

http://git-wip-us.apache.org/repos/asf/storm/blob/c1e8782f/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 3fcf7eb..ceb098e 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -32,7 +32,7 @@
 
 (defmulti download-storm-code cluster-mode)
 (defmulti launch-worker (fn [supervisor & _] (cluster-mode (:conf supervisor))))
-(defmulti mk-bt-tracker cluster-mode)
+(defmulti mk-code-distributor cluster-mode)
 
 ;; used as part of a map from port to this
 (defrecord LocalAssignment [storm-id executors])
@@ -302,7 +302,7 @@
                                          ))
    :assignment-versions (atom {})
    :sync-retry (atom 0)
-   :bt-tracker (mk-bt-tracker conf)
+   :bt-tracker (mk-code-distributor conf)
    })
 
 (defn sync-processes [supervisor]
@@ -342,8 +342,8 @@
          ". State: " state
          ", Heartbeat: " (pr-str heartbeat))
         (shutdown-worker supervisor id)
-        (if (:bt-tracker supervisor)
-          (.cleanup (:bt-tracker supervisor) id))
+        (if (:code-distributor supervisor)
+          (.cleanup (:code-distributor supervisor) id))
         ))
 
     (doseq [id (vals new-worker-ids)]
@@ -552,8 +552,8 @@
           supervisor-meta-file-path (supervisor-storm-metafile-path tmproot)]
       (FileUtils/forceMkdir (File. tmproot))
       (Utils/downloadFromMaster conf master-meta-file-path supervisor-meta-file-path)
-      (if (:bt-tracker supervisor)
-        (.download (:bt-tracker supervisor) storm-id (File. supervisor-meta-file-path)))
+      (if (:code-distributor supervisor)
+        (.download (:code-distributor supervisor) storm-id (File. supervisor-meta-file-path)))
       (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
       (if (.exists (File. stormroot)) (FileUtils/forceDelete (File. stormroot)))
       (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
@@ -587,7 +587,7 @@
                                              (storm-conf TOPOLOGY-USERS)))))}]
     (write-log-metadata-to-yaml-file! storm-id port data conf)))
 
-(defmethod mk-bt-tracker :distributed [conf]
+(defmethod mk-code-distributor :distributed [conf]
   (let [code-distributor (new-instance (conf STORM-CODE-DISTRIBUTOR-CLASS))]
     (.prepare code-distributor conf)
     code-distributor))
@@ -719,7 +719,7 @@
               )
             )))
 
-(defmethod mk-bt-tracker :local [conf] nil)
+(defmethod mk-code-distributor :local [conf] nil)
 
 (defmethod launch-worker
     :local [supervisor storm-id port worker-id]

http://git-wip-us.apache.org/repos/asf/storm/blob/c1e8782f/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 1a94049..c937aaa 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -1245,7 +1245,7 @@
                  uptime-computer nil
                  new-instance nil
                  mk-timer nil
-                 nimbus/mk-bt-tracker nil
+                 nimbus/mk-code-distributor nil
                  zk-leader-elector nil
                  nimbus/mk-scheduler nil]
         (nimbus/nimbus-data auth-conf fake-inimbus)

http://git-wip-us.apache.org/repos/asf/storm/blob/c1e8782f/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj b/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
index a776693..2787461 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
@@ -48,7 +48,6 @@
         nimbus-server (ThriftServer. (:daemon-conf cluster-map)
                                      (Nimbus$Processor. (:nimbus cluster-map)) 
                                      ThriftConnectionType/NIMBUS)]
-    (Thread/sleep 2000)
     (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop nimbus-server))))
     (.start (Thread. #(.serve nimbus-server)))
     (wait-for-condition #(.isServing nimbus-server))

http://git-wip-us.apache.org/repos/asf/storm/blob/c1e8782f/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index 6b4328a..9328769 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -481,7 +481,7 @@
                  cluster/mk-storm-cluster-state nil
                  supervisor-state nil
                  local-hostname nil
-                 supervisor/mk-bt-tracker nil
+                 supervisor/mk-code-distributor nil
                  mk-timer nil]
         (supervisor/supervisor-data auth-conf nil fake-isupervisor)
         (verify-call-times-for cluster/mk-storm-cluster-state 1)


[48/50] [abbrv] storm git commit: Merge remote-tracking branch 'upstream/master' into ha-merge

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into ha-merge


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

Branch: refs/heads/master
Commit: 51fdc1a367dd9b8657c5245726e4edc66925cf6f
Parents: 93dbcaf 9c94188
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Aug 19 11:03:34 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 19 11:03:34 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.md                                    |  11 +
 README.markdown                                 |   4 +
 SECURITY.md                                     |   2 +-
 TODO                                            | 178 ---------
 bin/storm-config.cmd                            |  18 +-
 bin/storm.py                                    |  22 +-
 conf/defaults.yaml                              |   3 +
 dev-tools/travis/travis-script.sh               |   2 +-
 docs/documentation/FAQ.md                       |   7 +-
 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-hdfs/README.md                   |   9 +
 external/storm-hdfs/pom.xml                     |  11 +
 .../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 +-
 .../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 -
 pom.xml                                         |   2 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |   2 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  13 +-
 storm-core/src/jvm/backtype/storm/Config.java   |  22 +-
 .../jvm/backtype/storm/ConfigValidation.java    |  13 +-
 .../security/auth/SaslTransportPlugin.java      |   6 +
 .../kerberos/KerberosSaslTransportPlugin.java   |   5 +-
 .../security/auth/kerberos/NoOpTTrasport.java   |  40 ++
 .../src/jvm/backtype/storm/task/ShellBolt.java  |   8 +-
 .../test/clj/backtype/storm/config_test.clj     |   4 +-
 39 files changed, 1098 insertions(+), 375 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/storm/blob/51fdc1a3/pom.xml
----------------------------------------------------------------------

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

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


[27/50] [abbrv] storm git commit: Adding rest documentation for nimbus summary.

Posted by bo...@apache.org.
Adding rest documentation for nimbus summary.


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

Branch: refs/heads/master
Commit: 726ce80be94bbfc5e70813221b36b6f4dfce4ccf
Parents: 27ad1fb
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 15:54:27 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 15:54:27 2015 -0800

----------------------------------------------------------------------
 STORM-UI-REST-API.md | 32 +++++++++++++++++++++++++++++++-
 1 file changed, 31 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/726ce80b/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md
index ed26cd9..4801a51 100644
--- a/STORM-UI-REST-API.md
+++ b/STORM-UI-REST-API.md
@@ -72,7 +72,6 @@ Response fields:
 |Field  |Value|Description
 |---	|---	|---
 |stormVersion|String| Storm version|
-|nimbusUptime|String| Shows how long the cluster is running|
 |supervisors|Integer|  Number of supervisors running|
 |slotsTotal| Integer|Total number of available worker slots|
 |slotsUsed| Integer| Number of worker slots used|
@@ -125,6 +124,37 @@ Sample response:
 }
 ```
 
+### /api/v1/nimbus/summary (GET)
+
+Returns summary information for all nimbus hosts.
+
+Response fields:
+
+|Field  |Value|Description|
+|---	|---	|---
+|host| String | Nimbus' host name|
+|port| int| Nimbus' port number|
+|nimbusUpTime| String| Shows since how long the nimbus has been running|
+|nimbusLogLink| String| Logviewer url to view the nimbus.log|
+|version| String| Version of storm this nimbus host is running|
+
+Sample response:
+
+```json
+{
+    "nimbuses":[
+        {
+            "host":"192.168.202.1",
+            "port":6627,
+            "nimbusLogLink":"http:\/\/192.168.202.1:8000\/log?file=nimbus.log",
+            "isLeader":true,
+            "version":"0.10.0-SNAPSHOT",
+            "nimbusUpTime":"3m 33s"
+        }
+    ]
+}
+```
+
 ### /api/v1/topology/summary (GET)
 
 Returns summary information for all topologies.


[29/50] [abbrv] storm git commit: Merge remote-tracking branch 'upstream/master' into STORM-166

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into STORM-166

Conflicts:
	storm-core/src/clj/backtype/storm/daemon/nimbus.clj
	storm-core/src/clj/backtype/storm/ui/core.clj


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

Branch: refs/heads/master
Commit: 23ed53754c47a2170e58b178beb1544616f0f08a
Parents: 63f29f3 8036109
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 16:44:06 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 16:44:06 2015 -0800

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 CHANGELOG.md                                    |  25 +-
 README.markdown                                 |  13 +-
 SECURITY.md                                     |  14 +-
 STORM-UI-REST-API.md                            |  21 +
 bin/storm                                       |  36 +-
 bin/storm-config.cmd                            |  32 +-
 bin/storm.cmd                                   |  15 +-
 conf/defaults.yaml                              |   3 +-
 dev-tools/github/__init__.py                    |  11 +
 dev-tools/jira-github-join.py                   |   4 +-
 dev-tools/storm-merge.py                        |  31 +
 docs/README.md                                  |   9 +
 docs/documentation/Common-patterns.md           |  14 +-
 docs/documentation/Concepts.md                  |  13 +-
 docs/documentation/Contributing-to-Storm.md     |   2 +-
 docs/documentation/Home.md                      |   2 +-
 docs/documentation/Multilang-protocol.md        |   4 +-
 docs/documentation/Powered-By.md                |  15 +-
 docs/downloads.html                             |   3 +
 .../storm/starter/SkewedRollingTopWords.java    | 134 +++
 .../storm/starter/bolt/RollingCountAggBolt.java |  78 ++
 .../hbase/trident/state/HBaseMapState.java      |   2 +-
 .../storm/hbase/trident/state/HBaseState.java   |   2 +-
 external/storm-kafka/README.md                  |  24 +-
 external/storm-kafka/pom.xml                    |  21 +-
 .../ExponentialBackoffMsgRetryManager.java      | 167 ++++
 .../jvm/storm/kafka/FailedMsgRetryManager.java  |  26 +
 .../src/jvm/storm/kafka/PartitionManager.java   |  42 +-
 .../src/jvm/storm/kafka/SpoutConfig.java        |   8 +
 .../ExponentialBackoffMsgRetryManagerTest.java  | 194 ++++
 pom.xml                                         |  19 +-
 storm-buildtools/storm-maven-plugins/pom.xml    |  81 ++
 .../storm/maven/plugin/util/CommandExec.java    |  89 ++
 .../plugin/versioninfo/VersionInfoMojo.java     | 304 ++++++
 storm-core/pom.xml                              |  41 +-
 storm-core/src/clj/backtype/storm/cluster.clj   |  38 +-
 .../src/clj/backtype/storm/daemon/logviewer.clj |   5 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  41 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |   5 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |   6 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  39 +-
 storm-core/src/clj/backtype/storm/util.clj      |  11 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   6 +
 .../coordination/BatchSubtopologyBuilder.java   |  11 +
 .../storm/drpc/LinearDRPCInputDeclarer.java     |   5 +-
 .../storm/drpc/LinearDRPCTopologyBuilder.java   |  13 +-
 .../storm/generated/GetInfoOptions.java         | 350 +++++++
 .../jvm/backtype/storm/generated/Nimbus.java    | 974 +++++++++++++++++++
 .../storm/generated/NumErrorsChoice.java        |  64 ++
 .../storm/grouping/PartialKeyGrouping.java      |  82 ++
 .../backtype/storm/messaging/netty/Server.java  |   2 +
 .../backtype/storm/topology/InputDeclarer.java  |   3 +
 .../storm/topology/TopologyBuilder.java         |  11 +
 .../TransactionalTopologyBuilder.java           |  13 +-
 .../src/jvm/backtype/storm/utils/Monitor.java   |   8 +-
 .../jvm/backtype/storm/utils/VersionInfo.java   | 131 +++
 .../topology/TridentTopologyBuilder.java        |  13 +-
 storm-core/src/py/storm/DistributedRPC-remote   |   0
 .../py/storm/DistributedRPCInvocations-remote   |   0
 storm-core/src/py/storm/Nimbus-remote           |   7 +
 storm-core/src/py/storm/Nimbus.py               | 226 +++++
 storm-core/src/py/storm/ttypes.py               |  80 ++
 .../storm-core-version-info.properties          |  24 +
 storm-core/src/storm.thrift                     |  11 +
 .../templates/topology-page-template.html       |   8 +-
 .../clj/backtype/storm/integration_test.clj     |  10 +-
 .../scheduler/multitenant_scheduler_test.clj    |   2 +
 .../test/clj/backtype/storm/supervisor_test.clj |  34 +-
 .../storm/grouping/PartialKeyGroupingTest.java  |  66 ++
 storm-dist/binary/src/main/assembly/binary.xml  |   7 -
 71 files changed, 3647 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --cc STORM-UI-REST-API.md
index 4801a51,deee134..e677c11
--- a/STORM-UI-REST-API.md
+++ b/STORM-UI-REST-API.md
@@@ -250,8 -219,10 +250,11 @@@ Response fields
  |bolts.errorWorkerLogLink| String | Link to the worker log that reported the exception |
  |bolts.emitted| Long |Number of tuples emitted|
  |antiForgeryToken| String | CSRF token|
 +|replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
  
+ Caution: users need to unescape the antiForgeryToken value before using this token to make POST calls(simple-json escapes forward slashes)
+ [ISSUE-8](https://code.google.com/p/json-simple/issues/detail?id=8)
+ 
  
  Examples:
  

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

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index b35a8d2,af86bd0..a5543b9
--- a/pom.xml
+++ b/pom.xml
@@@ -209,9 -210,9 +210,8 @@@
          <reply.version>0.3.0</reply.version>
          <conjure.version>2.1.3</conjure.version>
          <zookeeper.version>3.4.6</zookeeper.version>
-         <conjure.version>2.1.3</conjure.version>
          <clojure-data-codec.version>0.1.0</clojure-data-codec.version>
          <clojure-contrib.version>1.2.0</clojure-contrib.version>
 -
      </properties>
  
      <profiles>

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/storm-core/pom.xml
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 26bd591,0c7612b..30d123d
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@@ -24,13 -22,12 +24,14 @@@
    (:use [backtype.storm.scheduler.DefaultScheduler])
    (:import [backtype.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
              Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
 +  (:use [backtype.storm bootstrap util zookeeper])
-   (:import [backtype.storm.generated AuthorizationException])
 +  (:import [backtype.storm.nimbus NimbusInfo])
-   (:use [backtype.storm bootstrap util])
+   (:import [backtype.storm.generated AuthorizationException GetInfoOptions
+                                      NumErrorsChoice])
 -  (:use [backtype.storm bootstrap util])
    (:use [backtype.storm.config :only [validate-configs-with-schemas]])
    (:use [backtype.storm.daemon common])
    (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
++  (:import [backtype.storm.utils VersionInfo])
    (:gen-class
      :methods [^{:static true} [launch [backtype.storm.scheduler.INimbus] void]]))
  
@@@ -1031,24 -991,9 +1039,24 @@@
    (let [nimbus (nimbus-data conf inimbus)
         principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)]
      (.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
 +
 +    ;add to nimbuses
 +    (.add-nimbus-host! (:storm-cluster-state nimbus)
 +      (.toHostPortString (:nimbus-host-port-info nimbus))
 +      {
 +        :host (.getHost (:nimbus-host-port-info nimbus))
 +        :port (.getPort (:nimbus-host-port-info nimbus))
 +        :start-time-secs (current-time-secs)
-         :version (read-storm-version)
++        :version (str (VersionInfo/getVersion))
 +        })
 +
 +    (.addToLeaderLockQueue (:leader-elector nimbus))
      (cleanup-corrupt-topologies! nimbus)
 -    (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
 -      (transition! nimbus storm-id :startup))
 +    ;register call back for code-distributor
 +    (.code-distributor (:storm-cluster-state nimbus) (fn [] (sync-code conf nimbus)))
 +    (when (is-leader nimbus :throw-exception false)
 +      (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
 +        (transition! nimbus storm-id :startup)))
      (schedule-recurring (:timer nimbus)
                          0
                          (conf NIMBUS-MONITOR-FREQ-SECS)
@@@ -1332,11 -1259,11 +1340,11 @@@
                                                 topo-summ
                                            ))]
            (ClusterSummary. supervisor-summaries
 -                           nimbus-uptime
 -                           topology-summaries)
 +                           topology-summaries
 +                           nimbuses)
            ))
        
-       (^TopologyInfo getTopologyInfo [this ^String storm-id]
+       (^TopologyInfo getTopologyInfoWithOpts [this ^String storm-id ^GetInfoOptions options]
          (let [storm-cluster-state (:storm-cluster-state nimbus)
                topology-conf (try-read-storm-conf conf storm-id)
                storm-name (topology-conf TOPOLOGY-NAME)
@@@ -1373,10 -1314,14 +1395,15 @@@
                             )]
              (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))
 +            (.set_replication_count topo-info (.getReplicationCount (:code-distributor nimbus) storm-id))
              topo-info
            ))
-       
+ 
+       (^TopologyInfo getTopologyInfo [this ^String storm-id]
+         (.getTopologyInfoWithOpts this
+                                   storm-id
+                                   (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL))))
+ 
        Shutdownable
        (shutdown [this]
          (log-message "Shutting down master")

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

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index 8fd22a6,b36a7a9..726da73
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -498,7 -504,8 +502,7 @@@
                               (map #(.get_num_executors ^TopologySummary %))
                               (reduce +))]
         {"user" user
-         "stormVersion" (read-storm-version)
+         "stormVersion" (str (VersionInfo/getVersion))
 -        "nimbusUptime" (pretty-uptime-sec (.get_nimbus_uptime_secs summ))
          "supervisors" (count sups)
          "slotsTotal" total-slots
          "slotsUsed"  used-slots
@@@ -654,10 -643,13 +658,13 @@@
          "failed" (get-in stats [:failed k])})))
  
  (defn topology-page [id window include-sys? user]
 -  (with-nimbus nimbus
 +  (thrift/with-configured-nimbus-connection nimbus
      (let [window (if window window ":all-time")
            window-hint (window-hint window)
-           summ (.getTopologyInfo ^Nimbus$Client nimbus id)
+           summ (->> (doto
+                       (GetInfoOptions.)
+                       (.set_num_err_choice NumErrorsChoice/ONE))
+                     (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
            topology (.getTopology ^Nimbus$Client nimbus id)
            topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id))
            spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
@@@ -911,25 -898,36 +918,36 @@@
         (let [user (.getUserName http-creds-handler servlet-request)]
           (assert-authorized-user servlet-request "getTopology" (topology-config id))
           (json-response (component-page id component (:window m) (check-include-sys? (:sys m)) user) (:callback m))))
+   (GET "/api/v1/token" [ & m]
+        (json-response (format "{\"antiForgeryToken\": \"%s\"}" *anti-forgery-token*) (:callback m) :serialize-fn identity))
    (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
-       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (->> (doto
+                         (GetInfoOptions.)
+                         (.set_num_err_choice NumErrorsChoice/NONE))
+                       (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
              name (.get_name tplg)]
          (assert-authorized-user servlet-request "activate" (topology-config id))
          (.activate nimbus name)
          (log-message "Activating topology '" name "'")))
-     (resp/redirect (str "/api/v1/topology/" id)))
+     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
    (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
-       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (->> (doto
+                         (GetInfoOptions.)
+                         (.set_num_err_choice NumErrorsChoice/NONE))
+                       (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
              name (.get_name tplg)]
          (assert-authorized-user servlet-request "deactivate" (topology-config id))
          (.deactivate nimbus name)
          (log-message "Deactivating topology '" name "'")))
      (resp/redirect (str "/api/v1/topology/" (url-encode id))))
    (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
-       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (->> (doto
+                         (GetInfoOptions.)
+                         (.set_num_err_choice NumErrorsChoice/NONE))
+                       (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
              name (.get_name tplg)
              options (RebalanceOptions.)]
          (assert-authorized-user servlet-request "rebalance" (topology-config id))
@@@ -938,8 -936,11 +956,11 @@@
          (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
      (resp/redirect (str "/api/v1/topology/" (url-encode id))))
    (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
-       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (->> (doto
+                         (GetInfoOptions.)
+                         (.set_num_err_choice NumErrorsChoice/NONE))
+                       (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
              name (.get_name tplg)
              options (KillOptions.)]
          (assert-authorized-user servlet-request "killTopology" (topology-config id))

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

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/storm-core/src/storm.thrift
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/23ed5375/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------


[40/50] [abbrv] storm git commit: STORM-726: Adding nimbus.host config for backward compatibility.

Posted by bo...@apache.org.
STORM-726: Adding nimbus.host config for backward compatibility.

Conflicts:
	storm-core/src/jvm/backtype/storm/utils/NimbusClient.java


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

Branch: refs/heads/master
Commit: 3e20823c199c9babf3017cf1e8a8ff0753c7f24f
Parents: d1afefd
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Mar 26 11:22:10 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 09:25:27 2015 -0700

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/Config.java             | 7 +++++++
 storm-core/src/jvm/backtype/storm/utils/NimbusClient.java | 9 ++++++++-
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3e20823c/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 3cba37c..36749ca 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -316,6 +316,13 @@ public class Config extends HashMap<String, Object> {
     public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
+     * The host that the master server is running on, only here for backward compatibility.
+     */
+    @Deprecated
+    public static final String NIMBUS_HOST = "nimbus.host";
+    public static final Object NIMBUS_HOST_SCHEMA = String.class;
+
+    /**
      * List of seed nimbus hosts:port to use for leader nimbus discovery.
      */
     public static final String NIMBUS_SEEDS = "nimbus.seeds";

http://git-wip-us.apache.org/repos/asf/storm/blob/3e20823c/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index 08610e9..60fa3aa 100644
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -29,6 +29,7 @@ import backtype.storm.security.auth.ThriftConnectionType;
 import clojure.lang.IFn;
 import clojure.lang.PersistentArrayMap;
 import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -55,7 +56,13 @@ public class NimbusClient extends ThriftClient {
         }
 
         List<String> seeds = (List<String>) conf.get(Config.NIMBUS_SEEDS);
-        for (String seed : seeds) {
+
+        if(seeds == null  || seeds.isEmpty()) {
+            LOG.warn("config {} has no value. Failing over to deprecated config {}. ", Config.NIMBUS_SEEDS, Config.NIMBUS_HOST);
+            seeds = Lists.newArrayList(conf.get(Config.NIMBUS_HOST) + ":" + conf.get(Config.NIMBUS_THRIFT_PORT));
+        }
+
+        for(String seed : seeds) {
             String[] split = seed.split(":");
             String host = split[0];
             int port = Integer.parseInt(split[1]);


[20/50] [abbrv] storm git commit: STORM-655: Adding topology replication count to thrift/rest/ui.

Posted by bo...@apache.org.
STORM-655: Adding topology replication count to thrift/rest/ui.


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

Branch: refs/heads/master
Commit: 1b6491f1547ccd796aec387b8622a5c34bda0a81
Parents: 01eee7a
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Tue Feb 3 17:30:50 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Tue Feb 3 17:33:19 2015 -0800

----------------------------------------------------------------------
 STORM-UI-REST-API.md                            |  9 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  2 +
 storm-core/src/clj/backtype/storm/ui/core.clj   |  8 +-
 .../backtype/storm/generated/TopologyInfo.java  | 93 +++++++++++++++++++-
 .../storm/generated/TopologySummary.java        | 93 +++++++++++++++++++-
 storm-core/src/py/storm/ttypes.py               | 48 +++++-----
 storm-core/src/storm.thrift                     |  2 +
 .../public/templates/index-page-template.html   |  6 ++
 .../templates/topology-page-template.html       |  6 ++
 9 files changed, 238 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --git a/STORM-UI-REST-API.md b/STORM-UI-REST-API.md
index 1671262..ed26cd9 100644
--- a/STORM-UI-REST-API.md
+++ b/STORM-UI-REST-API.md
@@ -140,7 +140,7 @@ Response fields:
 |tasksTotal| Integer |Total number of tasks for this topology|
 |workersTotal| Integer |Number of workers used for this topology|
 |executorsTotal| Integer |Number of executors used for this topology|
-
+|replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
 Sample response:
 
 ```json
@@ -153,7 +153,8 @@ Sample response:
             "uptime": "6m 5s",
             "tasksTotal": 28,
             "workersTotal": 3,
-            "executorsTotal": 28
+            "executorsTotal": 28,
+            "replicationCount": 1
         }
     ]
 }
@@ -219,6 +220,7 @@ Response fields:
 |bolts.errorWorkerLogLink| String | Link to the worker log that reported the exception |
 |bolts.emitted| Long |Number of tuples emitted|
 |antiForgeryToken| String | CSRF token|
+|replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
 
 
 Examples:
@@ -365,7 +367,8 @@ Sample response:
         "supervisor.enable": true,
         "storm.messaging.netty.server_worker_threads": 1
     },
-    "antiForgeryToken": "lAFTN\/5iSedRLwJeUNqkJ8hgYubRl2OxjXGoDf9A4Bt1nZY3rvJW0\/P4zqu9yAk\/LvDhlmn7gigw\/z8C"
+    "antiForgeryToken": "lAFTN\/5iSedRLwJeUNqkJ8hgYubRl2OxjXGoDf9A4Bt1nZY3rvJW0\/P4zqu9yAk\/LvDhlmn7gigw\/z8C",
+    "replicationCount": 1
 }
 ```
 

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/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 2528825..e354fab 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1308,6 +1308,7 @@
                                                             (extract-status-str base))]
                                                (when-let [owner (:owner base)] (.set_owner topo-summ owner))
                                                (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
+                                               (.set_replication_count topo-summ (.getReplicationCount (:code-distributor nimbus) id))
                                                topo-summ
                                           ))]
           (ClusterSummary. supervisor-summaries
@@ -1352,6 +1353,7 @@
                            )]
             (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))
+            (.set_replication_count topo-info (.getReplicationCount (:code-distributor nimbus) storm-id))
             topo-info
           ))
       

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/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 12ddb2f..3c7f578 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -576,6 +576,7 @@
        "tasksTotal" (.get_num_tasks t)
        "workersTotal" (.get_num_workers t)
        "executorsTotal" (.get_num_executors t)
+       "replicationCount" (.get_replication_count t)
        "schedulerInfo" (.get_sched_status t)})}))
 
 (defn topology-stats [id window stats]
@@ -656,7 +657,8 @@
        "tasksTotal" (sum-tasks executors)
        "workersTotal" (count workers)
        "executorsTotal" (count executors)
-       "schedulerInfo" (.get_sched_status summ)}))
+       "schedulerInfo" (.get_sched_status summ)
+       "replicationCount" (.get_replication_count summ)}))
 
 (defn spout-summary-json [topology-id id stats window]
   (let [times (stats-times (:emitted stats))
@@ -689,6 +691,7 @@
           msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)
           spouts (.get_spouts topology)
           bolts (.get_bolts topology)
+          replication-count (.get_replication_count summ)
           visualizer-data (visualization-data (merge (hashmap-to-persistent spouts)
                                                      (hashmap-to-persistent bolts))
                                               spout-comp-summs
@@ -706,7 +709,8 @@
         "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys?)
         "configuration" topology-conf
         "visualizationTable" (stream-boxes visualizer-data)
-        "antiForgeryToken" *anti-forgery-token*}))))
+        "antiForgeryToken" *anti-forgery-token*
+        "replicationCount" replication-count}))))
 
 (defn spout-output-stats
   [stream-summary window]

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/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 2c36d4e..1ba5c5f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -49,6 +49,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   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 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 String id; // required
   private String name; // required
@@ -58,6 +59,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
   private Map<String,List<ErrorInfo>> errors; // required
   private String sched_status; // required
   private String owner; // required
+  private int replication_count; // 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 {
@@ -68,7 +70,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     STATUS((short)5, "status"),
     ERRORS((short)6, "errors"),
     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>();
 
@@ -99,6 +102,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           return SCHED_STATUS;
         case 514: // OWNER
           return OWNER;
+        case 515: // REPLICATION_COUNT
+          return REPLICATION_COUNT;
         default:
           return null;
       }
@@ -140,7 +145,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
 
   // isset id assignments
   private static final int __UPTIME_SECS_ISSET_ID = 0;
-  private BitSet __isset_bit_vector = new BitSet(1);
+  private static final int __REPLICATION_COUNT_ISSET_ID = 1;
+  private BitSet __isset_bit_vector = new BitSet(2);
 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
@@ -165,6 +171,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         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, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap);
   }
@@ -237,6 +245,7 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     if (other.is_set_owner()) {
       this.owner = other.owner;
     }
+    this.replication_count = other.replication_count;
   }
 
   public TopologyInfo deepCopy() {
@@ -254,6 +263,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     this.errors = null;
     this.sched_status = null;
     this.owner = null;
+    set_replication_count_isSet(false);
+    this.replication_count = 0;
   }
 
   public String get_id() {
@@ -465,6 +476,28 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     }
   }
 
+  public int get_replication_count() {
+    return this.replication_count;
+  }
+
+  public void set_replication_count(int replication_count) {
+    this.replication_count = replication_count;
+    set_replication_count_isSet(true);
+  }
+
+  public void unset_replication_count() {
+    __isset_bit_vector.clear(__REPLICATION_COUNT_ISSET_ID);
+  }
+
+  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_count() {
+    return __isset_bit_vector.get(__REPLICATION_COUNT_ISSET_ID);
+  }
+
+  public void set_replication_count_isSet(boolean value) {
+    __isset_bit_vector.set(__REPLICATION_COUNT_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -531,6 +564,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       break;
 
+    case REPLICATION_COUNT:
+      if (value == null) {
+        unset_replication_count();
+      } else {
+        set_replication_count((Integer)value);
+      }
+      break;
+
     }
   }
 
@@ -560,6 +601,9 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     case OWNER:
       return get_owner();
 
+    case REPLICATION_COUNT:
+      return Integer.valueOf(get_replication_count());
+
     }
     throw new IllegalStateException();
   }
@@ -587,6 +631,8 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       return is_set_sched_status();
     case OWNER:
       return is_set_owner();
+    case REPLICATION_COUNT:
+      return is_set_replication_count();
     }
     throw new IllegalStateException();
   }
@@ -676,6 +722,15 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return false;
     }
 
+    boolean this_present_replication_count = true && this.is_set_replication_count();
+    boolean that_present_replication_count = true && that.is_set_replication_count();
+    if (this_present_replication_count || that_present_replication_count) {
+      if (!(this_present_replication_count && that_present_replication_count))
+        return false;
+      if (this.replication_count != that.replication_count)
+        return false;
+    }
+
     return true;
   }
 
@@ -723,6 +778,11 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
     if (present_owner)
       builder.append(owner);
 
+    boolean present_replication_count = true && (is_set_replication_count());
+    builder.append(present_replication_count);
+    if (present_replication_count)
+      builder.append(replication_count);
+
     return builder.toHashCode();
   }
 
@@ -814,6 +874,16 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(typedOther.is_set_replication_count());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_count()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, typedOther.replication_count);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -922,6 +992,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 515: // REPLICATION_COUNT
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.replication_count = iprot.readI32();
+            set_replication_count_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
       }
@@ -999,6 +1077,11 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         oprot.writeFieldEnd();
       }
     }
+    if (is_set_replication_count()) {
+      oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC);
+      oprot.writeI32(this.replication_count);
+      oprot.writeFieldEnd();
+    }
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
@@ -1071,6 +1154,12 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       }
       first = false;
     }
+    if (is_set_replication_count()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_count:");
+      sb.append(this.replication_count);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
index fea2137..bc09e68 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
@@ -50,6 +50,7 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   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)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 String id; // required
   private String name; // required
@@ -60,6 +61,7 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   private String status; // required
   private String sched_status; // required
   private String owner; // required
+  private int replication_count; // 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 {
@@ -71,7 +73,8 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     UPTIME_SECS((short)6, "uptime_secs"),
     STATUS((short)7, "status"),
     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>();
 
@@ -104,6 +107,8 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
           return SCHED_STATUS;
         case 514: // OWNER
           return OWNER;
+        case 515: // REPLICATION_COUNT
+          return REPLICATION_COUNT;
         default:
           return null;
       }
@@ -148,7 +153,8 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
   private static final int __NUM_EXECUTORS_ISSET_ID = 1;
   private static final int __NUM_WORKERS_ISSET_ID = 2;
   private static final int __UPTIME_SECS_ISSET_ID = 3;
-  private BitSet __isset_bit_vector = new BitSet(4);
+  private static final int __REPLICATION_COUNT_ISSET_ID = 4;
+  private BitSet __isset_bit_vector = new BitSet(5);
 
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
@@ -171,6 +177,8 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         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, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap);
   }
@@ -226,6 +234,7 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     if (other.is_set_owner()) {
       this.owner = other.owner;
     }
+    this.replication_count = other.replication_count;
   }
 
   public TopologySummary deepCopy() {
@@ -247,6 +256,8 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     this.status = null;
     this.sched_status = null;
     this.owner = null;
+    set_replication_count_isSet(false);
+    this.replication_count = 0;
   }
 
   public String get_id() {
@@ -452,6 +463,28 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     }
   }
 
+  public int get_replication_count() {
+    return this.replication_count;
+  }
+
+  public void set_replication_count(int replication_count) {
+    this.replication_count = replication_count;
+    set_replication_count_isSet(true);
+  }
+
+  public void unset_replication_count() {
+    __isset_bit_vector.clear(__REPLICATION_COUNT_ISSET_ID);
+  }
+
+  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_count() {
+    return __isset_bit_vector.get(__REPLICATION_COUNT_ISSET_ID);
+  }
+
+  public void set_replication_count_isSet(boolean value) {
+    __isset_bit_vector.set(__REPLICATION_COUNT_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -526,6 +559,14 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       }
       break;
 
+    case REPLICATION_COUNT:
+      if (value == null) {
+        unset_replication_count();
+      } else {
+        set_replication_count((Integer)value);
+      }
+      break;
+
     }
   }
 
@@ -558,6 +599,9 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     case OWNER:
       return get_owner();
 
+    case REPLICATION_COUNT:
+      return Integer.valueOf(get_replication_count());
+
     }
     throw new IllegalStateException();
   }
@@ -587,6 +631,8 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       return is_set_sched_status();
     case OWNER:
       return is_set_owner();
+    case REPLICATION_COUNT:
+      return is_set_replication_count();
     }
     throw new IllegalStateException();
   }
@@ -685,6 +731,15 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         return false;
     }
 
+    boolean this_present_replication_count = true && this.is_set_replication_count();
+    boolean that_present_replication_count = true && that.is_set_replication_count();
+    if (this_present_replication_count || that_present_replication_count) {
+      if (!(this_present_replication_count && that_present_replication_count))
+        return false;
+      if (this.replication_count != that.replication_count)
+        return false;
+    }
+
     return true;
   }
 
@@ -737,6 +792,11 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
     if (present_owner)
       builder.append(owner);
 
+    boolean present_replication_count = true && (is_set_replication_count());
+    builder.append(present_replication_count);
+    if (present_replication_count)
+      builder.append(replication_count);
+
     return builder.toHashCode();
   }
 
@@ -838,6 +898,16 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(typedOther.is_set_replication_count());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_count()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, typedOther.replication_count);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -922,6 +992,14 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
           }
           break;
+        case 515: // REPLICATION_COUNT
+          if (field.type == org.apache.thrift.protocol.TType.I32) {
+            this.replication_count = iprot.readI32();
+            set_replication_count_isSet(true);
+          } else { 
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
         default:
           org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
       }
@@ -976,6 +1054,11 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
         oprot.writeFieldEnd();
       }
     }
+    if (is_set_replication_count()) {
+      oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC);
+      oprot.writeI32(this.replication_count);
+      oprot.writeFieldEnd();
+    }
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
@@ -1044,6 +1127,12 @@ public class TopologySummary implements org.apache.thrift.TBase<TopologySummary,
       }
       first = false;
     }
+    if (is_set_replication_count()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_count:");
+      sb.append(this.replication_count);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/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 1bbaf37..e4fb751 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -1,19 +1,3 @@
-# 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.7.0)
 #
@@ -1571,6 +1555,7 @@ class TopologySummary:
    - status
    - sched_status
    - owner
+   - replication_count
   """
 
   thrift_spec = (
@@ -2089,12 +2074,13 @@ class TopologySummary:
     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 __hash__(self):
-    return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status) + hash(self.sched_status) + hash(self.owner)
+    return 0 + hash(self.id) + hash(self.name) + hash(self.num_tasks) + hash(self.num_executors) + hash(self.num_workers) + hash(self.uptime_secs) + hash(self.status) + hash(self.sched_status) + hash(self.owner) + hash(self.replication_count)
 
-  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None,):
+  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None, replication_count=None,):
     self.id = id
     self.name = name
     self.num_tasks = num_tasks
@@ -2104,6 +2090,7 @@ class TopologySummary:
     self.status = status
     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:
@@ -2159,6 +2146,11 @@ class TopologySummary:
           self.owner = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 515:
+        if ftype == TType.I32:
+          self.replication_count = iprot.readI32();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2205,6 +2197,10 @@ class TopologySummary:
       oprot.writeFieldBegin('owner', TType.STRING, 514)
       oprot.writeString(self.owner.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.replication_count is not None:
+      oprot.writeFieldBegin('replication_count', TType.I32, 515)
+      oprot.writeI32(self.replication_count)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3388,6 +3384,7 @@ class TopologyInfo:
    - errors
    - sched_status
    - owner
+   - replication_count
   """
 
   thrift_spec = (
@@ -3906,12 +3903,13 @@ class TopologyInfo:
     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 __hash__(self):
-    return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors) + hash(self.sched_status) + hash(self.owner)
+    return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.executors) + hash(self.status) + hash(self.errors) + hash(self.sched_status) + hash(self.owner) + hash(self.replication_count)
 
-  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, sched_status=None, owner=None, replication_count=None,):
     self.id = id
     self.name = name
     self.uptime_secs = uptime_secs
@@ -3920,6 +3918,7 @@ class TopologyInfo:
     self.errors = errors
     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:
@@ -3988,6 +3987,11 @@ class TopologyInfo:
           self.owner = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 515:
+        if ftype == TType.I32:
+          self.replication_count = iprot.readI32();
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -4040,6 +4044,10 @@ class TopologyInfo:
       oprot.writeFieldBegin('owner', TType.STRING, 514)
       oprot.writeString(self.owner.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.replication_count is not None:
+      oprot.writeFieldBegin('replication_count', TType.I32, 515)
+      oprot.writeI32(self.replication_count)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index f807b74..3c49d39 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -145,6 +145,7 @@ struct TopologySummary {
   7: required string status;
 513: optional string sched_status;
 514: optional string owner;
+515: optional i32 replication_count;
 }
 
 struct SupervisorSummary {
@@ -218,6 +219,7 @@ struct TopologyInfo {
   6: required map<string, list<ErrorInfo>> errors;
 513: optional string sched_status;
 514: optional string owner;
+515: optional i32 replication_count;
 }
 
 struct KillOptions {

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index f301ac5..0a4fdf3 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -153,6 +153,11 @@
           </span>
         </th>
         <th>
+          <span class="tip above" title="Number of nimbus hosts on which this topology's code is replicated. ">
+            Replication count
+          </span>
+        </th>
+        <th>
           <span class="tip left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
             Scheduler Info
           </span>
@@ -170,6 +175,7 @@
         <td>{{workersTotal}}</td>
         <td>{{executorsTotal}}</td>
         <td>{{tasksTotal}}</td>
+        <td>{{replicationCount}}</td>
         <td>{{schedulerInfo}}</td>
       </tr>
       {{/topologies}}

http://git-wip-us.apache.org/repos/asf/storm/blob/1b6491f1/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 b74edd2..2b42197 100644
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@ -59,6 +59,11 @@
           </span>
         </th>
         <th>
+          <span class="tip above" title="Number of nimbus hosts on which this topology's code is replicated. ">
+            Replication count
+          </span>
+        </th>
+        <th>
           <span class="tip left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
             Scheduler Info
           </span>
@@ -75,6 +80,7 @@
         <td>{{workersTotal}}</td>
         <td>{{executorsTotal}}</td>
         <td>{{tasksTotal}}</td>
+        <td>{{replicationCount}}</td>
         <td>{{schedulerInfo}}</td>
       </tr>
     </tbody>


[24/50] [abbrv] storm git commit: Changing nimbus discovery to use thrift API instead of using zookeeper.

Posted by bo...@apache.org.
Changing nimbus discovery to use thrift API instead of using zookeeper.


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

Branch: refs/heads/master
Commit: 7cae523e3c8e86688a371da9bae809cb4c244358
Parents: aa24375
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 14:57:26 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 14:57:26 2015 -0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |  1 +
 storm-core/src/clj/backtype/storm/thrift.clj    | 13 +++---
 storm-core/src/jvm/backtype/storm/Config.java   |  6 +++
 .../jvm/backtype/storm/utils/NimbusClient.java  | 42 +++++++++++++-------
 4 files changed, 40 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7cae523e/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index fbea948..2d8b62c 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -51,6 +51,7 @@ storm.meta.serialization.delegate: "backtype.storm.serialization.DefaultSerializ
 storm.codedistributor.class: "backtype.storm.codedistributor.LocalFileSystemCodeDistributor"
 
 ### nimbus.* configs are for the master
+nimbus.seeds : ["localhost:6627"]
 nimbus.thrift.port: 6627
 nimbus.thrift.threads: 64
 nimbus.thrift.max_buffer_size: 1048576

http://git-wip-us.apache.org/repos/asf/storm/blob/7cae523e/storm-core/src/clj/backtype/storm/thrift.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/thrift.clj b/storm-core/src/clj/backtype/storm/thrift.clj
index 2b860b3..0cac7b8 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -84,13 +84,12 @@
 (defmacro with-configured-nimbus-connection
   [client-sym & body]
   `(let [conf# (read-storm-config)
-         zk-leader-elector# (zk-leader-elector conf#)
-         leader-nimbus# (.getLeader zk-leader-elector#)
-         host# (.getHost leader-nimbus#)
-         port# (.getPort leader-nimbus#)
-         no-op# (.close zk-leader-elector#)]
-     (with-nimbus-connection [~client-sym host# port#]
-       ~@body )))
+         ~client-sym (NimbusClient/getConfiguredClient conf#)
+         conn# (.transport ~client-sym)
+         ]
+     (try
+       ~@body
+     (finally (.close conn#)))))
 
 (defn direct-output-fields
   [fields]

http://git-wip-us.apache.org/repos/asf/storm/blob/7cae523e/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 c0668ee..106f1f3 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -307,6 +307,12 @@ public class Config extends HashMap<String, Object> {
     public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
+     * List of seed nimbus hosts:port to use for leader nimbus discovery.
+     */
+    public static final String NIMBUS_SEEDS = "nimbus.seeds";
+    public static final Object NIMBUS_SEEDS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
      * Which port the Thrift interface of Nimbus should run on. Clients should
      * connect to this port to upload jars and submit topologies.
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/7cae523e/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index da10a4f..e4222e4 100644
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -18,17 +18,22 @@
 package backtype.storm.utils;
 
 
+import backtype.storm.Config;
+import backtype.storm.generated.ClusterSummary;
 import backtype.storm.generated.Nimbus;
+import backtype.storm.generated.NimbusSummary;
 import backtype.storm.nimbus.ILeaderElector;
 import backtype.storm.nimbus.NimbusInfo;
 import backtype.storm.security.auth.ThriftClient;
 import backtype.storm.security.auth.ThriftConnectionType;
 import clojure.lang.IFn;
 import clojure.lang.PersistentArrayMap;
+import com.google.common.base.Splitter;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.List;
 import java.util.Map;
 
 public class NimbusClient extends ThriftClient {
@@ -36,23 +41,30 @@ public class NimbusClient extends ThriftClient {
     private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class);
 
     public static NimbusClient getConfiguredClient(Map conf) {
-        ILeaderElector zkLeaderElector = null;
-        try {
-            IFn zkLeaderElectorFn = Utils.loadClojureFn("backtype.storm.zookeeper", "zk-leader-elector");
-            zkLeaderElector = (ILeaderElector) zkLeaderElectorFn.invoke(PersistentArrayMap.create(conf));
-            NimbusInfo leaderInfo = zkLeaderElector.getLeader();
-            String nimbusHost = leaderInfo.getHost();
-            int nimbusPort = leaderInfo.getPort();
-            return new NimbusClient(conf, nimbusHost, nimbusPort);
-        } catch (TTransportException ex) {
-            throw new RuntimeException(ex);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        } finally {
-            if(zkLeaderElector != null) {
-                zkLeaderElector.close();
+        List<String> seeds = (List<String>) conf.get(Config.NIMBUS_SEEDS);
+        for(String seed : seeds) {
+            String[] split = seed.split(":");
+            String host = split[0];
+            int port = Integer.parseInt(split[1]);
+            try {
+                NimbusClient client = new NimbusClient(conf,host,port);
+                ClusterSummary clusterInfo = client.getClient().getClusterInfo();
+                List<NimbusSummary> nimbuses = clusterInfo.get_nimbuses();
+                if(nimbuses != null) {
+                    for(NimbusSummary nimbusSummary : nimbuses) {
+                        if(nimbusSummary.is_isLeader()) {
+                            return new NimbusClient(conf, nimbusSummary.get_host(), nimbusSummary.get_port());
+                        }
+                    }
+                }
+                throw new RuntimeException("Found nimbuses " + nimbuses + " none of which is elected as leader, please try " +
+                        "again after some time.");
+            } catch (Exception e) {
+                LOG.warn("Ignoring exception while trying to get leader nimbus info from {}", seed);
             }
         }
+        throw new RuntimeException("Could not find leader nimbus from seed hosts " + seeds +". " +
+                "Did you specify a valid list of nimbus host:port for config " + Config.NIMBUS_SEEDS);
     }
 
     public NimbusClient(Map conf, String host, int port) throws TTransportException {


[34/50] [abbrv] storm git commit: Regenerated NimbusSummary using thrift-0.9.2

Posted by bo...@apache.org.
Regenerated NimbusSummary using thrift-0.9.2


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

Branch: refs/heads/master
Commit: f7205d29f7afeb9e4cf03a31be0ac5737652b07c
Parents: 85e4d5e
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Mon Mar 2 15:26:41 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Mon Mar 2 15:52:00 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/ui/core.clj   |   8 +-
 .../backtype/storm/generated/NimbusSummary.java | 381 +++++++++++--------
 2 files changed, 231 insertions(+), 158 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f7205d29/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 7fc9273..0f93f23 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -25,7 +25,8 @@
                                               ACKER-FAIL-STREAM-ID system-id? mk-authorization-handler]]])
   (:use [ring.middleware.anti-forgery])
   (: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
             ErrorInfo ClusterSummary SupervisorSummary TopologySummary
@@ -41,7 +42,6 @@
             [ring.util.response :as resp]
             [backtype.storm [thrift :as thrift]])
   (:import [org.apache.commons.lang StringEscapeUtils])
-  (:import [backtype.storm.nimbus NimbusInfo])
   (:gen-class))
 
 (def ^:dynamic *STORM-CONF* (read-storm-config))
@@ -520,14 +520,14 @@
         (.get_nimbuses (.getClusterInfo ^Nimbus$Client nimbus)))))
   ([nimbuses]
     {"nimbuses"
-     (for [^NimbusInfo n nimbuses]
+     (for [^NimbusSummary n nimbuses]
        {
         "host" (.get_host n)
         "port" (.get_port n)
         "nimbusLogLink" (nimbus-log-link (.get_host n) (.get_port n))
         "isLeader" (.is_isLeader n)
         "version" (.get_version n)
-        "nimbusUpTime" (pretty-uptime-sec (.get_uptimeSecs n))})}))
+        "nimbusUpTime" (pretty-uptime-sec (.get_uptime_secs n))})}))
 
 (defn supervisor-summary
   ([]

http://git-wip-us.apache.org/repos/asf/storm/blob/f7205d29/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
index 195048a..1d0f3fb 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
@@ -16,13 +16,24 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.7.0)
+ * 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.commons.lang.builder.HashCodeBuilder;
+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;
@@ -35,21 +46,30 @@ 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;
 
-public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, NimbusSummary._Fields>, java.io.Serializable, Cloneable {
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-2")
+public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, NimbusSummary._Fields>, java.io.Serializable, Cloneable, Comparable<NimbusSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusSummary");
 
   private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)2);
-  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptimeSecs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)3);
   private static final org.apache.thrift.protocol.TField IS_LEADER_FIELD_DESC = new org.apache.thrift.protocol.TField("isLeader", org.apache.thrift.protocol.TType.BOOL, (short)4);
   private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)5);
 
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new NimbusSummaryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new NimbusSummaryTupleSchemeFactory());
+  }
+
   private String host; // required
   private int port; // required
-  private int uptimeSecs; // required
+  private int uptime_secs; // required
   private boolean isLeader; // required
   private String version; // required
 
@@ -57,7 +77,7 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     HOST((short)1, "host"),
     PORT((short)2, "port"),
-    UPTIME_SECS((short)3, "uptimeSecs"),
+    UPTIME_SECS((short)3, "uptime_secs"),
     IS_LEADER((short)4, "isLeader"),
     VERSION((short)5, "version");
 
@@ -125,10 +145,9 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
 
   // isset id assignments
   private static final int __PORT_ISSET_ID = 0;
-  private static final int __UPTIMESECS_ISSET_ID = 1;
+  private static final int __UPTIME_SECS_ISSET_ID = 1;
   private static final int __ISLEADER_ISSET_ID = 2;
-  private BitSet __isset_bit_vector = new BitSet(3);
-
+  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);
@@ -136,7 +155,7 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptimeSecs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.IS_LEADER, new org.apache.thrift.meta_data.FieldMetaData("isLeader", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
@@ -152,7 +171,7 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
   public NimbusSummary(
     String host,
     int port,
-    int uptimeSecs,
+    int uptime_secs,
     boolean isLeader,
     String version)
   {
@@ -160,8 +179,8 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
     this.host = host;
     this.port = port;
     set_port_isSet(true);
-    this.uptimeSecs = uptimeSecs;
-    set_uptimeSecs_isSet(true);
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
     this.isLeader = isLeader;
     set_isLeader_isSet(true);
     this.version = version;
@@ -171,13 +190,12 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
    * Performs a deep copy on <i>other</i>.
    */
   public NimbusSummary(NimbusSummary other) {
-    __isset_bit_vector.clear();
-    __isset_bit_vector.or(other.__isset_bit_vector);
+    __isset_bitfield = other.__isset_bitfield;
     if (other.is_set_host()) {
       this.host = other.host;
     }
     this.port = other.port;
-    this.uptimeSecs = other.uptimeSecs;
+    this.uptime_secs = other.uptime_secs;
     this.isLeader = other.isLeader;
     if (other.is_set_version()) {
       this.version = other.version;
@@ -193,8 +211,8 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
     this.host = null;
     set_port_isSet(false);
     this.port = 0;
-    set_uptimeSecs_isSet(false);
-    this.uptimeSecs = 0;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
     set_isLeader_isSet(false);
     this.isLeader = false;
     this.version = null;
@@ -233,38 +251,38 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
   }
 
   public void unset_port() {
-    __isset_bit_vector.clear(__PORT_ISSET_ID);
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID);
   }
 
   /** Returns true if field port is set (has been assigned a value) and false otherwise */
   public boolean is_set_port() {
-    return __isset_bit_vector.get(__PORT_ISSET_ID);
+    return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID);
   }
 
   public void set_port_isSet(boolean value) {
-    __isset_bit_vector.set(__PORT_ISSET_ID, value);
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value);
   }
 
-  public int get_uptimeSecs() {
-    return this.uptimeSecs;
+  public int get_uptime_secs() {
+    return this.uptime_secs;
   }
 
-  public void set_uptimeSecs(int uptimeSecs) {
-    this.uptimeSecs = uptimeSecs;
-    set_uptimeSecs_isSet(true);
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
   }
 
-  public void unset_uptimeSecs() {
-    __isset_bit_vector.clear(__UPTIMESECS_ISSET_ID);
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
   }
 
-  /** Returns true if field uptimeSecs is set (has been assigned a value) and false otherwise */
-  public boolean is_set_uptimeSecs() {
-    return __isset_bit_vector.get(__UPTIMESECS_ISSET_ID);
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
   }
 
-  public void set_uptimeSecs_isSet(boolean value) {
-    __isset_bit_vector.set(__UPTIMESECS_ISSET_ID, value);
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
   }
 
   public boolean is_isLeader() {
@@ -277,16 +295,16 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
   }
 
   public void unset_isLeader() {
-    __isset_bit_vector.clear(__ISLEADER_ISSET_ID);
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISLEADER_ISSET_ID);
   }
 
   /** Returns true if field isLeader is set (has been assigned a value) and false otherwise */
   public boolean is_set_isLeader() {
-    return __isset_bit_vector.get(__ISLEADER_ISSET_ID);
+    return EncodingUtils.testBit(__isset_bitfield, __ISLEADER_ISSET_ID);
   }
 
   public void set_isLeader_isSet(boolean value) {
-    __isset_bit_vector.set(__ISLEADER_ISSET_ID, value);
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISLEADER_ISSET_ID, value);
   }
 
   public String get_version() {
@@ -332,9 +350,9 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
 
     case UPTIME_SECS:
       if (value == null) {
-        unset_uptimeSecs();
+        unset_uptime_secs();
       } else {
-        set_uptimeSecs((Integer)value);
+        set_uptime_secs((Integer)value);
       }
       break;
 
@@ -366,7 +384,7 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
       return Integer.valueOf(get_port());
 
     case UPTIME_SECS:
-      return Integer.valueOf(get_uptimeSecs());
+      return Integer.valueOf(get_uptime_secs());
 
     case IS_LEADER:
       return Boolean.valueOf(is_isLeader());
@@ -390,7 +408,7 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
     case PORT:
       return is_set_port();
     case UPTIME_SECS:
-      return is_set_uptimeSecs();
+      return is_set_uptime_secs();
     case IS_LEADER:
       return is_set_isLeader();
     case VERSION:
@@ -430,12 +448,12 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
         return false;
     }
 
-    boolean this_present_uptimeSecs = true;
-    boolean that_present_uptimeSecs = true;
-    if (this_present_uptimeSecs || that_present_uptimeSecs) {
-      if (!(this_present_uptimeSecs && that_present_uptimeSecs))
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
         return false;
-      if (this.uptimeSecs != that.uptimeSecs)
+      if (this.uptime_secs != that.uptime_secs)
         return false;
     }
 
@@ -462,90 +480,90 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
+    List<Object> list = new ArrayList<Object>();
 
     boolean present_host = true && (is_set_host());
-    builder.append(present_host);
+    list.add(present_host);
     if (present_host)
-      builder.append(host);
+      list.add(host);
 
     boolean present_port = true;
-    builder.append(present_port);
+    list.add(present_port);
     if (present_port)
-      builder.append(port);
+      list.add(port);
 
-    boolean present_uptimeSecs = true;
-    builder.append(present_uptimeSecs);
-    if (present_uptimeSecs)
-      builder.append(uptimeSecs);
+    boolean present_uptime_secs = true;
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
 
     boolean present_isLeader = true;
-    builder.append(present_isLeader);
+    list.add(present_isLeader);
     if (present_isLeader)
-      builder.append(isLeader);
+      list.add(isLeader);
 
     boolean present_version = true && (is_set_version());
-    builder.append(present_version);
+    list.add(present_version);
     if (present_version)
-      builder.append(version);
+      list.add(version);
 
-    return builder.toHashCode();
+    return list.hashCode();
   }
 
+  @Override
   public int compareTo(NimbusSummary other) {
     if (!getClass().equals(other.getClass())) {
       return getClass().getName().compareTo(other.getClass().getName());
     }
 
     int lastComparison = 0;
-    NimbusSummary typedOther = (NimbusSummary)other;
 
-    lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host());
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_host()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, typedOther.host);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port());
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_port()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, typedOther.port);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_uptimeSecs()).compareTo(typedOther.is_set_uptimeSecs());
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (is_set_uptimeSecs()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptimeSecs, typedOther.uptimeSecs);
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_isLeader()).compareTo(typedOther.is_set_isLeader());
+    lastComparison = Boolean.valueOf(is_set_isLeader()).compareTo(other.is_set_isLeader());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_isLeader()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isLeader, typedOther.isLeader);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isLeader, other.isLeader);
       if (lastComparison != 0) {
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(is_set_version()).compareTo(typedOther.is_set_version());
+    lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version());
     if (lastComparison != 0) {
       return lastComparison;
     }
     if (is_set_version()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, typedOther.version);
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -558,87 +576,11 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
   }
 
   public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    org.apache.thrift.protocol.TField field;
-    iprot.readStructBegin();
-    while (true)
-    {
-      field = iprot.readFieldBegin();
-      if (field.type == org.apache.thrift.protocol.TType.STOP) { 
-        break;
-      }
-      switch (field.id) {
-        case 1: // HOST
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.host = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 2: // PORT
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.port = iprot.readI32();
-            set_port_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 3: // UPTIME_SECS
-          if (field.type == org.apache.thrift.protocol.TType.I32) {
-            this.uptimeSecs = iprot.readI32();
-            set_uptimeSecs_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 4: // IS_LEADER
-          if (field.type == org.apache.thrift.protocol.TType.BOOL) {
-            this.isLeader = iprot.readBool();
-            set_isLeader_isSet(true);
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        case 5: // VERSION
-          if (field.type == org.apache.thrift.protocol.TType.STRING) {
-            this.version = iprot.readString();
-          } else { 
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-          }
-          break;
-        default:
-          org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
-      }
-      iprot.readFieldEnd();
-    }
-    iprot.readStructEnd();
-    validate();
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
   }
 
   public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    validate();
-
-    oprot.writeStructBegin(STRUCT_DESC);
-    if (this.host != null) {
-      oprot.writeFieldBegin(HOST_FIELD_DESC);
-      oprot.writeString(this.host);
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldBegin(PORT_FIELD_DESC);
-    oprot.writeI32(this.port);
-    oprot.writeFieldEnd();
-    oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
-    oprot.writeI32(this.uptimeSecs);
-    oprot.writeFieldEnd();
-    oprot.writeFieldBegin(IS_LEADER_FIELD_DESC);
-    oprot.writeBool(this.isLeader);
-    oprot.writeFieldEnd();
-    if (this.version != null) {
-      oprot.writeFieldBegin(VERSION_FIELD_DESC);
-      oprot.writeString(this.version);
-      oprot.writeFieldEnd();
-    }
-    oprot.writeFieldStop();
-    oprot.writeStructEnd();
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
   }
 
   @Override
@@ -658,8 +600,8 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
     sb.append(this.port);
     first = false;
     if (!first) sb.append(", ");
-    sb.append("uptimeSecs:");
-    sb.append(this.uptimeSecs);
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
     first = false;
     if (!first) sb.append(", ");
     sb.append("isLeader:");
@@ -687,8 +629,8 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString());
     }
 
-    if (!is_set_uptimeSecs()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptimeSecs' is unset! Struct:" + toString());
+    if (!is_set_uptime_secs()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
     }
 
     if (!is_set_isLeader()) {
@@ -699,6 +641,7 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
       throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString());
     }
 
+    // check for sub-struct validity
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -712,12 +655,142 @@ public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, Nim
   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_bit_vector = new BitSet(1);
+      __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 NimbusSummaryStandardSchemeFactory implements SchemeFactory {
+    public NimbusSummaryStandardScheme getScheme() {
+      return new NimbusSummaryStandardScheme();
+    }
+  }
+
+  private static class NimbusSummaryStandardScheme extends StandardScheme<NimbusSummary> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, NimbusSummary 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: // HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.host = iprot.readString();
+              struct.set_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.port = iprot.readI32();
+              struct.set_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // IS_LEADER
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isLeader = iprot.readBool();
+              struct.set_isLeader_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.version = iprot.readString();
+              struct.set_version_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, NimbusSummary struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.host != null) {
+        oprot.writeFieldBegin(HOST_FIELD_DESC);
+        oprot.writeString(struct.host);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(PORT_FIELD_DESC);
+      oprot.writeI32(struct.port);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(IS_LEADER_FIELD_DESC);
+      oprot.writeBool(struct.isLeader);
+      oprot.writeFieldEnd();
+      if (struct.version != null) {
+        oprot.writeFieldBegin(VERSION_FIELD_DESC);
+        oprot.writeString(struct.version);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class NimbusSummaryTupleSchemeFactory implements SchemeFactory {
+    public NimbusSummaryTupleScheme getScheme() {
+      return new NimbusSummaryTupleScheme();
+    }
+  }
+
+  private static class NimbusSummaryTupleScheme extends TupleScheme<NimbusSummary> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, NimbusSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.host);
+      oprot.writeI32(struct.port);
+      oprot.writeI32(struct.uptime_secs);
+      oprot.writeBool(struct.isLeader);
+      oprot.writeString(struct.version);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, NimbusSummary struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.host = iprot.readString();
+      struct.set_host_isSet(true);
+      struct.port = iprot.readI32();
+      struct.set_port_isSet(true);
+      struct.uptime_secs = iprot.readI32();
+      struct.set_uptime_secs_isSet(true);
+      struct.isLeader = iprot.readBool();
+      struct.set_isLeader_isSet(true);
+      struct.version = iprot.readString();
+      struct.set_version_isSet(true);
+    }
+  }
+
 }
 


[43/50] [abbrv] storm git commit: Fixing the uptime in nimbusSummary.

Posted by bo...@apache.org.
Fixing the uptime in nimbusSummary.


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

Branch: refs/heads/master
Commit: 95fb6809fad6a5de8af67fec2f6af0556bd18ddb
Parents: 16293e4
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Apr 1 18:19:58 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 09:29:53 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/daemon/nimbus.clj              | 1 +
 storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/95fb6809/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 5d9a038..9aa098a 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1353,6 +1353,7 @@
                       leader-host (.getHost leader)
                       leader-port (.getPort leader)]
                   (doseq [nimbus-summary nimbuses]
+                    (.set_uptime_secs nimbus-summary (time-delta (.get_uptime_secs nimbus-summary)))
                     (.set_isLeader nimbus-summary (and (= leader-host (.get_host nimbus-summary)) (= leader-port (.get_port nimbus-summary))))))
 
               topology-summaries (dofor [[id base] bases :when base]

http://git-wip-us.apache.org/repos/asf/storm/blob/95fb6809/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj
index 7188d7a..f7d3802 100644
--- a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj
+++ b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj
@@ -20,7 +20,7 @@
   (:use [backtype.storm.daemon.worker :only [is-connection-ready]])
   (:import [java.util ArrayList]))
 
-(def port 6700)
+(def port (available-port))
 (def task 1)
 
 ;; In a "real" cluster (or an integration test), Storm itself would ensure that a topology's workers would only be


[36/50] [abbrv] storm git commit: Fixing a broker unit test that started failing after merge.

Posted by bo...@apache.org.
Fixing a broker unit test that started failing after merge.


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

Branch: refs/heads/master
Commit: eb3a8375aed6eea6302383839676634f663acc46
Parents: 5758718
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Mar 20 10:38:05 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Mar 20 10:38:05 2015 -0700

----------------------------------------------------------------------
 storm-core/test/clj/backtype/storm/cluster_test.clj | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/eb3a8375/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 ffc72af..e5ffc17 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -17,6 +17,7 @@
   (:import [java.util Arrays]
            [backtype.storm.nimbus NimbusInfo])
   (:import [backtype.storm.daemon.common Assignment StormBase SupervisorInfo])
+  (:import [backtype.storm.generated NimbusSummary])
   (:import [org.apache.zookeeper ZooDefs ZooDefs$Ids])
   (:import [org.mockito Mockito])
   (:import [org.mockito.exceptions.base MockitoAssertionError])


[28/50] [abbrv] storm git commit: Adding cluster-test for nimbus summary storage.

Posted by bo...@apache.org.
Adding cluster-test for nimbus summary storage.


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

Branch: refs/heads/master
Commit: 63f29f349b469746fd1a4507151ec47942e03e58
Parents: 726ce80
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Feb 12 16:14:14 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Feb 12 16:14:14 2015 -0800

----------------------------------------------------------------------
 storm-core/test/clj/backtype/storm/cluster_test.clj | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/63f29f34/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 b7630b1..85aaf3b 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -213,6 +213,12 @@
       (.remove-storm! state "storm1")
       (is (= [] (.code-distributor state nil)))
 
+      (is (= [] (.nimbuses state)))
+      (.add-nimbus-host! state "host:port" nimbusInfo1)
+      (is (= [nimbusInfo1] (.nimbuses state)))
+      (.add-nimbus-host! state "host1:port" nimbusInfo2)
+      (is (= #{nimbusInfo1 nimbusInfo2} (set (.nimbuses state))))
+
       ;; TODO add tests for task info and task heartbeat setting and getting
       (.disconnect state)
       )))


[33/50] [abbrv] storm git commit: Merge remote-tracking branch 'upstream/master' into STORM-166

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into STORM-166

Conflicts:
	STORM-UI-REST-API.md
	pom.xml
	storm-core/src/clj/backtype/storm/daemon/nimbus.clj
	storm-core/src/clj/backtype/storm/daemon/supervisor.clj
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
	storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
	storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
	storm-core/src/py/storm/ttypes.py
	storm-core/test/clj/backtype/storm/nimbus_test.clj


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

Branch: refs/heads/master
Commit: 85e4d5efce66352469cb4c9947922731e834ca5d
Parents: 8d4e561 559f0f2
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Mon Mar 2 15:16:05 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Mon Mar 2 15:16:05 2015 -0800

----------------------------------------------------------------------
 .gitignore                                      |    4 +-
 BYLAWS.md                                       |   31 +-
 CHANGELOG.md                                    |   18 +
 DEVELOPER.md                                    |   24 +-
 README.markdown                                 |    3 +
 SECURITY.md                                     |   10 +-
 STORM-UI-REST-API.md                            |   46 +-
 bin/storm-config.cmd                            |   14 +-
 bin/storm.cmd                                   |   33 +-
 conf/defaults.yaml                              |    2 +-
 docs/documentation/Trident-API-Overview.md      |    2 +-
 docs/images/architecture.png                    |  Bin 0 -> 69825 bytes
 docs/images/architecture.svg                    | 1458 +++
 docs/images/bolt.png                            |  Bin 0 -> 24796 bytes
 docs/images/bolt.svg                            |  743 ++
 docs/images/security.png                        |  Bin 0 -> 72415 bytes
 docs/images/security.svg                        | 1779 ++++
 docs/images/spout.png                           |  Bin 0 -> 22911 bytes
 docs/images/spout.svg                           |  833 ++
 docs/images/storm.svg                           | 1326 +++
 docs/images/topology.png                        |  Bin 59837 -> 64740 bytes
 docs/images/topology.svg                        | 1044 ++
 docs/images/topology_dark.png                   |  Bin 0 -> 49692 bytes
 docs/images/topology_dark.svg                   | 1101 +++
 .../jvm/storm/starter/BasicDRPCTopology.java    |    3 +-
 external/storm-hdfs/pom.xml                     |   18 +-
 .../storm/hdfs/bolt/HdfsFileTopology.java       |    6 +-
 .../storm/hdfs/bolt/SequenceFileTopology.java   |    4 +-
 .../storm/hdfs/trident/TridentFileTopology.java |    2 +-
 .../hdfs/trident/TridentSequenceTopology.java   |    6 +-
 external/storm-hive/README.md                   |  113 +
 external/storm-hive/pom.xml                     |  143 +
 .../org/apache/storm/hive/bolt/HiveBolt.java    |  289 +
 .../bolt/mapper/DelimitedRecordHiveMapper.java  |  143 +
 .../storm/hive/bolt/mapper/HiveMapper.java      |   81 +
 .../hive/bolt/mapper/JsonRecordHiveMapper.java  |  132 +
 .../apache/storm/hive/common/HiveOptions.java   |  146 +
 .../org/apache/storm/hive/common/HiveUtils.java |   76 +
 .../apache/storm/hive/common/HiveWriter.java    |  420 +
 .../apache/storm/hive/trident/HiveState.java    |  306 +
 .../storm/hive/trident/HiveStateFactory.java    |   31 +
 .../apache/storm/hive/trident/HiveUpdater.java  |   14 +
 .../apache/storm/hive/bolt/HiveSetupUtil.java   |  220 +
 .../apache/storm/hive/bolt/HiveTopology.java    |  150 +
 .../hive/bolt/HiveTopologyPartitioned.java      |  153 +
 .../apache/storm/hive/bolt/TestHiveBolt.java    |  330 +
 .../storm/hive/common/TestHiveWriter.java       |  193 +
 .../storm/hive/trident/TridentHiveTopology.java |  190 +
 external/storm-jdbc/LICENSE                     |  202 +
 external/storm-jdbc/README.md                   |  240 +
 external/storm-jdbc/pom.xml                     |  125 +
 .../storm/jdbc/bolt/AbstractJdbcBolt.java       |   57 +
 .../apache/storm/jdbc/bolt/JdbcInsertBolt.java  |   71 +
 .../apache/storm/jdbc/bolt/JdbcLookupBolt.java  |   76 +
 .../org/apache/storm/jdbc/common/Column.java    |  111 +
 .../apache/storm/jdbc/common/JdbcClient.java    |  228 +
 .../java/org/apache/storm/jdbc/common/Util.java |   75 +
 .../storm/jdbc/mapper/JdbcLookupMapper.java     |   26 +
 .../apache/storm/jdbc/mapper/JdbcMapper.java    |   33 +
 .../jdbc/mapper/SimpleJdbcLookupMapper.java     |   46 +
 .../storm/jdbc/mapper/SimpleJdbcMapper.java     |   92 +
 .../storm/jdbc/trident/state/JdbcQuery.java     |   40 +
 .../storm/jdbc/trident/state/JdbcState.java     |  145 +
 .../jdbc/trident/state/JdbcStateFactory.java    |   40 +
 .../storm/jdbc/trident/state/JdbcUpdater.java   |   32 +
 .../storm/jdbc/common/JdbcClientTest.java       |   88 +
 .../org/apache/storm/jdbc/common/UtilTest.java  |   69 +
 .../org/apache/storm/jdbc/spout/UserSpout.java  |   90 +
 .../jdbc/topology/AbstractUserTopology.java     |  106 +
 .../jdbc/topology/UserPersistanceTopology.java  |   48 +
 .../UserPersistanceTridentTopology.java         |   61 +
 external/storm-jdbc/src/test/sql/test.sql       |    1 +
 .../src/jvm/storm/kafka/PartitionManager.java   |    5 +-
 external/storm-redis/LICENSE                    |  202 +
 external/storm-redis/README.md                  |  137 +
 external/storm-redis/pom.xml                    |   65 +
 .../storm/redis/bolt/AbstractRedisBolt.java     |   67 +
 .../trident/mapper/TridentTupleMapper.java      |   27 +
 .../trident/state/RedisClusterMapState.java     |  294 +
 .../redis/trident/state/RedisClusterState.java  |   80 +
 .../trident/state/RedisClusterStateQuerier.java |   78 +
 .../trident/state/RedisClusterStateUpdater.java |   76 +
 .../redis/trident/state/RedisMapState.java      |  323 +
 .../storm/redis/trident/state/RedisState.java   |   83 +
 .../redis/trident/state/RedisStateQuerier.java  |   70 +
 .../state/RedisStateSetCountQuerier.java        |   74 +
 .../trident/state/RedisStateSetUpdater.java     |   80 +
 .../redis/trident/state/RedisStateUpdater.java  |   75 +
 .../redis/util/config/JedisClusterConfig.java   |   82 +
 .../redis/util/config/JedisPoolConfig.java      |   97 +
 .../util/container/JedisClusterContainer.java   |   47 +
 .../JedisCommandsContainerBuilder.java          |   38 +
 .../JedisCommandsInstanceContainer.java         |   25 +
 .../redis/util/container/JedisContainer.java    |   65 +
 .../storm/redis/topology/LookupWordCount.java   |  127 +
 .../redis/topology/PersistentWordCount.java     |  117 +
 .../storm/redis/topology/WordCounter.java       |   58 +
 .../apache/storm/redis/topology/WordSpout.java  |   88 +
 .../storm/redis/trident/PrintFunction.java      |   40 +
 .../redis/trident/WordCountTridentRedis.java    |   97 +
 .../trident/WordCountTridentRedisCluster.java   |  103 +
 .../WordCountTridentRedisClusterMap.java        |  101 +
 .../redis/trident/WordCountTridentRedisMap.java |   95 +
 .../redis/trident/WordCountTupleMapper.java     |   16 +
 pom.xml                                         |   15 +-
 storm-core/pom.xml                              |   13 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj |   64 -
 storm-core/src/clj/backtype/storm/clojure.clj   |    2 +-
 storm-core/src/clj/backtype/storm/config.clj    |    3 +-
 .../src/clj/backtype/storm/daemon/drpc.clj      |    6 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |   26 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   26 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |  179 +-
 .../src/clj/backtype/storm/daemon/task.clj      |   16 +-
 .../src/clj/backtype/storm/daemon/worker.clj    |   70 +-
 .../src/clj/backtype/storm/messaging/local.clj  |    2 +-
 storm-core/src/clj/backtype/storm/tuple.clj     |    4 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |   61 +-
 .../src/clj/backtype/storm/ui/helpers.clj       |    1 +
 storm-core/src/clj/backtype/storm/util.clj      |    3 +
 .../storm/drpc/DRPCInvocationsClient.java       |    6 +
 .../src/jvm/backtype/storm/drpc/DRPCSpout.java  |   14 +-
 .../jvm/backtype/storm/drpc/ReturnResults.java  |    8 +-
 .../storm/generated/AlreadyAliveException.java  |  149 +-
 .../storm/generated/AuthorizationException.java |  149 +-
 .../src/jvm/backtype/storm/generated/Bolt.java  |  194 +-
 .../jvm/backtype/storm/generated/BoltStats.java |  908 +-
 .../storm/generated/ClusterSummary.java         |  383 +-
 .../storm/generated/ComponentCommon.java        |  426 +-
 .../storm/generated/ComponentObject.java        |   86 +-
 .../backtype/storm/generated/Credentials.java   |  220 +-
 .../storm/generated/DRPCExecutionException.java |  149 +-
 .../backtype/storm/generated/DRPCRequest.java   |  185 +-
 .../storm/generated/DistributedRPC.java         |  529 +-
 .../generated/DistributedRPCInvocations.java    | 1199 ++-
 .../jvm/backtype/storm/generated/ErrorInfo.java |  300 +-
 .../backtype/storm/generated/ExecutorInfo.java  |  198 +-
 .../storm/generated/ExecutorSpecificStats.java  |   72 +-
 .../backtype/storm/generated/ExecutorStats.java |  486 +-
 .../storm/generated/ExecutorSummary.java        |  371 +-
 .../storm/generated/GetInfoOptions.java         |  166 +-
 .../storm/generated/GlobalStreamId.java         |  185 +-
 .../jvm/backtype/storm/generated/Grouping.java  |  163 +-
 .../generated/InvalidTopologyException.java     |  149 +-
 .../backtype/storm/generated/JavaObject.java    |  239 +-
 .../backtype/storm/generated/JavaObjectArg.java |  108 +-
 .../backtype/storm/generated/KillOptions.java   |  176 +-
 .../jvm/backtype/storm/generated/Nimbus.java    | 9177 +++++++++++++-----
 .../storm/generated/NotAliveException.java      |  149 +-
 .../backtype/storm/generated/NullStruct.java    |  112 +-
 .../storm/generated/NumErrorsChoice.java        |    3 +-
 .../storm/generated/RebalanceOptions.java       |  348 +-
 .../storm/generated/ShellComponent.java         |  202 +-
 .../jvm/backtype/storm/generated/SpoutSpec.java |  194 +-
 .../backtype/storm/generated/SpoutStats.java    |  614 +-
 .../storm/generated/StateSpoutSpec.java         |  194 +-
 .../backtype/storm/generated/StormTopology.java |  410 +-
 .../backtype/storm/generated/StreamInfo.java    |  249 +-
 .../backtype/storm/generated/SubmitOptions.java |  208 +-
 .../storm/generated/SupervisorSummary.java      |  309 +-
 .../backtype/storm/generated/TopologyInfo.java  |  661 +-
 .../storm/generated/TopologyInitialStatus.java  |    3 +-
 .../storm/generated/TopologySummary.java        |  538 +-
 .../storm/messaging/ConnectionWithStatus.java   |   32 +
 .../backtype/storm/messaging/netty/Client.java  |  712 +-
 .../messaging/netty/SaslStormClientHandler.java |    5 +-
 .../backtype/storm/messaging/netty/Server.java  |  182 +-
 .../netty/StormClientPipelineFactory.java       |    5 +-
 .../security/auth/SimpleTransportPlugin.java    |    2 +-
 .../trident/drpc/ReturnResultsReducer.java      |    4 +-
 .../trident/spout/RichSpoutBatchExecutor.java   |    1 +
 storm-core/src/py/storm/DistributedRPC-remote   |   35 +-
 storm-core/src/py/storm/DistributedRPC.py       |   38 +-
 .../py/storm/DistributedRPCInvocations-remote   |   43 +-
 .../src/py/storm/DistributedRPCInvocations.py   |   95 +-
 storm-core/src/py/storm/Nimbus-remote           |  111 +-
 storm-core/src/py/storm/Nimbus.py               |  640 +-
 storm-core/src/py/storm/constants.py            |    6 +-
 storm-core/src/py/storm/ttypes.py               |  470 +-
 storm-core/src/ui/public/index.html             |    1 +
 .../test/clj/backtype/storm/clojure_test.clj    |    9 +-
 .../test/clj/backtype/storm/drpc_test.clj       |    8 +-
 .../test/clj/backtype/storm/grouping_test.clj   |    9 +-
 .../clj/backtype/storm/integration_test.clj     |   10 +-
 .../storm/messaging/netty_integration_test.clj  |    5 +-
 .../storm/messaging/netty_unit_test.clj         |   73 +-
 .../test/clj/backtype/storm/messaging_test.clj  |    6 +-
 .../test/clj/backtype/storm/metrics_test.clj    |    8 +-
 .../test/clj/backtype/storm/multilang_test.clj  |    6 +-
 .../test/clj/backtype/storm/nimbus_test.clj     |   22 +-
 .../scheduler/multitenant_scheduler_test.clj    |    4 +-
 .../test/clj/backtype/storm/scheduler_test.clj  |    4 +-
 .../storm/security/auth/AuthUtils_test.clj      |    6 +-
 .../backtype/storm/security/auth/auth_test.clj  |   13 +-
 .../storm/security/auth/drpc_auth_test.clj      |    7 +-
 .../storm/security/auth/nimbus_auth_test.clj    |   10 +-
 .../clj/backtype/storm/subtopology_test.clj     |    9 +-
 .../test/clj/backtype/storm/supervisor_test.clj |   16 +-
 .../test/clj/backtype/storm/tick_tuple_test.clj |    7 +-
 .../clj/backtype/storm/transactional_test.clj   |   14 +-
 .../test/clj/backtype/storm/worker_test.clj     |   37 +
 storm-dist/binary/src/main/assembly/binary.xml  |   38 +-
 202 files changed, 32950 insertions(+), 7507 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/STORM-UI-REST-API.md
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/clj/backtype/storm/config.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 30d123d,f1b4082..eee0417
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@@ -14,24 -14,32 +14,36 @@@
  ;; See the License for the specific language governing permissions and
  ;; limitations under the License.
  (ns backtype.storm.daemon.nimbus
+   (:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
+   (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
+   (:import [org.apache.thrift.exception])
+   (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
+   (:import [org.apache.commons.io FileUtils])
    (:import [java.nio ByteBuffer]
-            [java.util Collections]
 -           [java.util Collections HashMap])
++           [java.util Collections HashMap]
 +           [backtype.storm.generated NimbusSummary])
-   (:import [java.io FileNotFoundException])
+   (:import [java.io FileNotFoundException File FileOutputStream])
 +  (:import [java.net InetAddress])
    (:import [java.nio.channels Channels WritableByteChannel])
    (:import [backtype.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils])
    (:use [backtype.storm.scheduler.DefaultScheduler])
    (:import [backtype.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
              Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
-   (:use [backtype.storm bootstrap util zookeeper])
 +  (:import [backtype.storm.nimbus NimbusInfo])
-   (:import [backtype.storm.generated AuthorizationException GetInfoOptions
-                                      NumErrorsChoice])
-   (:use [backtype.storm.config :only [validate-configs-with-schemas]])
+   (:import [backtype.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ThriftTopologyUtils
+             BufferFileInputStream])
+   (:import [backtype.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
+             ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
+             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 [clojure.set :as set])
+   (:import [backtype.storm.daemon.common StormBase Assignment])
    (:use [backtype.storm.daemon common])
    (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
 +  (:import [backtype.storm.utils VersionInfo])
    (:gen-class
      :methods [^{:static true} [launch [backtype.storm.scheduler.INimbus] void]]))
  

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index e53cb52,b9f9632..d315925
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@@ -28,11 -35,8 +35,9 @@@
    (:gen-class
      :methods [^{:static true} [launch [backtype.storm.scheduler.ISupervisor] void]]))
  
- (bootstrap)
- 
  (defmulti download-storm-code cluster-mode)
  (defmulti launch-worker (fn [supervisor & _] (cluster-mode (:conf supervisor))))
 +(defmulti mk-code-distributor cluster-mode)
  
  ;; used as part of a map from port to this
  (defrecord LocalAssignment [storm-id executors])
@@@ -302,7 -306,7 +307,8 @@@
                                           ))
     :assignment-versions (atom {})
     :sync-retry (atom 0)
 +   :code-distributor (mk-code-distributor conf)
+    :download-lock (Object.)
     })
  
  (defn sync-processes [supervisor]
@@@ -431,16 -458,7 +463,7 @@@
        (doseq [[storm-id master-code-dir] storm-code-map]
          (when (and (not (downloaded-storm-ids storm-id))
                     (assigned-storm-ids storm-id))
-           (log-message "Downloading code for storm id "
-              storm-id
-              " from "
-              master-code-dir)
-           (download-storm-code conf storm-id master-code-dir supervisor)
-           (log-message "Finished downloading code for storm id "
-              storm-id
-              " from "
-              master-code-dir)
-           ))
 -          (download-storm-code conf storm-id master-code-dir download-lock)))
++          (download-storm-code conf storm-id master-code-dir supervisor download-lock)))
  
        (log-debug "Writing new assignment "
                   (pr-str new-assignment))
@@@ -544,21 -562,30 +567,23 @@@
  
  ;; distributed implementation
  (defmethod download-storm-code
-     :distributed [conf storm-id master-code-dir supervisor]
 -    :distributed [conf storm-id master-code-dir download-lock]
++    :distributed [conf storm-id master-code-dir supervisor download-lock]
      ;; Downloading to permanent location is atomic
      (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid))
 -          stormroot (supervisor-stormdist-root conf storm-id)]
 +          stormroot (supervisor-stormdist-root conf storm-id)
 +          master-meta-file-path (master-storm-metafile-path master-code-dir)
 +          supervisor-meta-file-path (supervisor-storm-metafile-path tmproot)]
-       (FileUtils/forceMkdir (File. tmproot))
-       (Utils/downloadFromMaster conf master-meta-file-path supervisor-meta-file-path)
-       (if (:code-distributor supervisor)
-         (.download (:code-distributor supervisor) storm-id (File. supervisor-meta-file-path)))
-       (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
-       (if (.exists (File. stormroot)) (FileUtils/forceDelete (File. stormroot)))
-       (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
-       (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)      
-     ))
+       (locking download-lock
 -            (log-message "Downloading code for storm id "
 -                         storm-id
 -                         " from "
 -                         master-code-dir)
 -            (FileUtils/forceMkdir (File. tmproot))
 -
 -            (Utils/downloadFromMaster conf (master-stormjar-path master-code-dir) (supervisor-stormjar-path tmproot))
 -            (Utils/downloadFromMaster conf (master-stormcode-path master-code-dir) (supervisor-stormcode-path tmproot))
 -            (Utils/downloadFromMaster conf (master-stormconf-path master-code-dir) (supervisor-stormconf-path tmproot))
 -            (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
 -            (if-not (.exists (File. stormroot))
 -              (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
 -              (FileUtils/deleteDirectory (File. tmproot)))
 -            (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
 -            (log-message "Finished downloading code for storm id "
 -                         storm-id
 -                         " from "
 -                         master-code-dir))
 -      ))
++        (log-message "Downloading code for storm id " storm-id " from " master-code-dir)
++        (FileUtils/forceMkdir (File. tmproot))
++        (Utils/downloadFromMaster conf master-meta-file-path supervisor-meta-file-path)
++        (if (:code-distributor supervisor)
++          (.download (:code-distributor supervisor) storm-id (File. supervisor-meta-file-path)))
++        (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
++        (if (.exists (File. stormroot)) (FileUtils/forceDelete (File. stormroot)))
++        (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
++        (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) stormroot)
++        (log-message "Finished downloading code for storm id " storm-id " from " master-code-dir))))
  
  (defn write-log-metadata-to-yaml-file! [storm-id port data conf]
    (let [file (get-log-metadata-file storm-id port)]

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index 726da73,f656b12..7fc9273
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -17,8 -17,9 +17,9 @@@
  (ns backtype.storm.ui.core
    (:use compojure.core)
    (:use ring.middleware.reload)
+   (: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 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]]])
@@@ -492,18 -495,21 +493,20 @@@
          (cluster-summary (.getClusterInfo ^Nimbus$Client nimbus) user)))
    ([^ClusterSummary summ user]
       (let [sups (.get_supervisors summ)
-         used-slots (reduce + (map #(.get_num_used_workers ^SupervisorSummary %) sups))
-         total-slots (reduce + (map #(.get_num_workers ^SupervisorSummary %) sups))
-         free-slots (- total-slots used-slots)
-         total-tasks (->> (.get_topologies summ)
-                          (map #(.get_num_tasks ^TopologySummary %))
-                          (reduce +))
-         total-executors (->> (.get_topologies summ)
-                              (map #(.get_num_executors ^TopologySummary %))
-                              (reduce +))]
+            used-slots (reduce + (map #(.get_num_used_workers ^SupervisorSummary %) sups))
+            total-slots (reduce + (map #(.get_num_workers ^SupervisorSummary %) sups))
+            free-slots (- total-slots used-slots)
+            topologies (.get_topologies_size summ)
+            total-tasks (->> (.get_topologies summ)
+                             (map #(.get_num_tasks ^TopologySummary %))
+                             (reduce +))
+            total-executors (->> (.get_topologies summ)
+                                 (map #(.get_num_executors ^TopologySummary %))
+                                 (reduce +))]
         {"user" user
          "stormVersion" (str (VersionInfo/getVersion))
 -        "nimbusUptime" (pretty-uptime-sec (.get_nimbus_uptime_secs summ))
          "supervisors" (count sups)
+         "topologies" topologies
          "slotsTotal" total-slots
          "slotsUsed"  used-slots
          "slotsFree" free-slots
@@@ -869,9 -855,15 +872,15 @@@
         spec errors))))
  
  (defn topology-config [topology-id]
 -  (with-nimbus nimbus
 -    (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))))
 +  (thrift/with-configured-nimbus-connection nimbus
 +     (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))))
  
+ (defn topology-op-response [topology-id op]
+   {"topologyOperation" op,
+    "topologyId" topology-id,
+    "status" "success"
+    })
+ 
  (defn check-include-sys?
    [sys?]
    (if (or (nil? sys?) (= "false" sys?)) false true))
@@@ -920,8 -910,8 +930,8 @@@
           (json-response (component-page id component (:window m) (check-include-sys? (:sys m)) user) (:callback m))))
    (GET "/api/v1/token" [ & m]
         (json-response (format "{\"antiForgeryToken\": \"%s\"}" *anti-forgery-token*) (:callback m) :serialize-fn identity))
-   (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id]
+   (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id & m]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
        (let [tplg (->> (doto
                          (GetInfoOptions.)
                          (.set_num_err_choice NumErrorsChoice/NONE))
@@@ -930,9 -920,9 +940,9 @@@
          (assert-authorized-user servlet-request "activate" (topology-config id))
          (.activate nimbus name)
          (log-message "Activating topology '" name "'")))
-     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
-   (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id]
 -    (json-response (topology-op-response id "deactivate") (m "callback")))
++    (json-response (topology-op-response id "activate") (m "callback")))
+   (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id & m]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
        (let [tplg (->> (doto
                          (GetInfoOptions.)
                          (.set_num_err_choice NumErrorsChoice/NONE))
@@@ -941,9 -931,9 +951,9 @@@
          (assert-authorized-user servlet-request "deactivate" (topology-config id))
          (.deactivate nimbus name)
          (log-message "Deactivating topology '" name "'")))
-     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
-   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
+     (json-response (topology-op-response id "deactivate") (m "callback")))
+   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
        (let [tplg (->> (doto
                          (GetInfoOptions.)
                          (.set_num_err_choice NumErrorsChoice/NONE))
@@@ -952,11 -943,16 +963,16 @@@
              options (RebalanceOptions.)]
          (assert-authorized-user servlet-request "rebalance" (topology-config id))
          (.set_wait_secs options (Integer/parseInt wait-time))
+         (if (and (not-nil? rebalance-options) (contains? rebalance-options "numWorkers"))
+           (.set_num_workers options (Integer/parseInt (.toString (rebalance-options "numWorkers")))))
+         (if (and (not-nil? rebalance-options) (contains? rebalance-options "executors"))
+           (doseq [keyval (rebalance-options "executors")]
+             (.put_to_num_executors options (key keyval) (Integer/parseInt (.toString (val keyval))))))
          (.rebalance nimbus name options)
          (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
-     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
-   (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
+     (json-response (topology-op-response id "rebalance") (m "callback")))
+   (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
 -    (with-nimbus nimbus
 +    (thrift/with-configured-nimbus-connection nimbus
        (let [tplg (->> (doto
                          (GetInfoOptions.)
                          (.set_num_err_choice NumErrorsChoice/NONE))

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
index 7e32c72,ee25ac8..e5651df
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
@@@ -38,16 -50,24 +50,24 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, ClusterSummary._Fields>, java.io.Serializable, Cloneable {
+ @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-3-2")
+ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, ClusterSummary._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterSummary> {
    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterSummary");
  
    private static final org.apache.thrift.protocol.TField SUPERVISORS_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisors", org.apache.thrift.protocol.TType.LIST, (short)1);
 -  private static final org.apache.thrift.protocol.TField NIMBUS_UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus_uptime_secs", org.apache.thrift.protocol.TType.I32, (short)2);
    private static final org.apache.thrift.protocol.TField TOPOLOGIES_FIELD_DESC = new org.apache.thrift.protocol.TField("topologies", org.apache.thrift.protocol.TType.LIST, (short)3);
 +  private static final org.apache.thrift.protocol.TField NIMBUSES_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbuses", org.apache.thrift.protocol.TType.LIST, (short)4);
  
+   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+   static {
+     schemes.put(StandardScheme.class, new ClusterSummaryStandardSchemeFactory());
+     schemes.put(TupleScheme.class, new ClusterSummaryTupleSchemeFactory());
+   }
+ 
    private List<SupervisorSummary> supervisors; // required
 -  private int nimbus_uptime_secs; // required
    private List<TopologySummary> topologies; // required
 +  private List<NimbusSummary> nimbuses; // 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 {
@@@ -114,7 -134,8 +134,6 @@@
    }
  
    // isset id assignments
- 
 -  private static final int __NIMBUS_UPTIME_SECS_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);
@@@ -149,27 -170,22 +168,27 @@@
     * Performs a deep copy on <i>other</i>.
     */
    public ClusterSummary(ClusterSummary other) {
 -    __isset_bitfield = other.__isset_bitfield;
      if (other.is_set_supervisors()) {
-       List<SupervisorSummary> __this__supervisors = new ArrayList<SupervisorSummary>();
+       List<SupervisorSummary> __this__supervisors = new ArrayList<SupervisorSummary>(other.supervisors.size());
        for (SupervisorSummary other_element : other.supervisors) {
          __this__supervisors.add(new SupervisorSummary(other_element));
        }
        this.supervisors = __this__supervisors;
      }
 -    this.nimbus_uptime_secs = other.nimbus_uptime_secs;
      if (other.is_set_topologies()) {
-       List<TopologySummary> __this__topologies = new ArrayList<TopologySummary>();
+       List<TopologySummary> __this__topologies = new ArrayList<TopologySummary>(other.topologies.size());
        for (TopologySummary other_element : other.topologies) {
          __this__topologies.add(new TopologySummary(other_element));
        }
        this.topologies = __this__topologies;
      }
 +    if (other.is_set_nimbuses()) {
-       List<NimbusSummary> __this__nimbuses = new ArrayList<NimbusSummary>();
++      List<NimbusSummary> __this__nimbuses = new ArrayList<NimbusSummary>(other.nimbuses.size());
 +      for (NimbusSummary other_element : other.nimbuses) {
 +        __this__nimbuses.add(new NimbusSummary(other_element));
 +      }
 +      this.nimbuses = __this__nimbuses;
 +    }
    }
  
    public ClusterSummary deepCopy() {
@@@ -403,26 -404,27 +422,27 @@@
  
    @Override
    public int hashCode() {
-     HashCodeBuilder builder = new HashCodeBuilder();
+     List<Object> list = new ArrayList<Object>();
  
      boolean present_supervisors = true && (is_set_supervisors());
-     builder.append(present_supervisors);
+     list.add(present_supervisors);
      if (present_supervisors)
-       builder.append(supervisors);
+       list.add(supervisors);
  
 -    boolean present_nimbus_uptime_secs = true;
 -    list.add(present_nimbus_uptime_secs);
 -    if (present_nimbus_uptime_secs)
 -      list.add(nimbus_uptime_secs);
 -
      boolean present_topologies = true && (is_set_topologies());
-     builder.append(present_topologies);
+     list.add(present_topologies);
      if (present_topologies)
-       builder.append(topologies);
+       list.add(topologies);
  
 +    boolean present_nimbuses = true && (is_set_nimbuses());
-     builder.append(present_nimbuses);
++    list.add(present_nimbuses);
 +    if (present_nimbuses)
-       builder.append(nimbuses);
++      list.add(nimbuses);
 +
-     return builder.toHashCode();
+     return list.hashCode();
    }
  
+   @Override
    public int compareTo(ClusterSummary other) {
      if (!getClass().equals(other.getClass())) {
        return getClass().getName().compareTo(other.getClass().getName());
@@@ -441,22 -442,22 +460,22 @@@
          return lastComparison;
        }
      }
-     lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(typedOther.is_set_topologies());
 -    lastComparison = Boolean.valueOf(is_set_nimbus_uptime_secs()).compareTo(other.is_set_nimbus_uptime_secs());
++    lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(other.is_set_topologies());
      if (lastComparison != 0) {
        return lastComparison;
      }
 -    if (is_set_nimbus_uptime_secs()) {
 -      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus_uptime_secs, other.nimbus_uptime_secs);
 +    if (is_set_topologies()) {
-       lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, typedOther.topologies);
++      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, other.topologies);
        if (lastComparison != 0) {
          return lastComparison;
        }
      }
-     lastComparison = Boolean.valueOf(is_set_nimbuses()).compareTo(typedOther.is_set_nimbuses());
 -    lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(other.is_set_topologies());
++    lastComparison = Boolean.valueOf(is_set_nimbuses()).compareTo(other.is_set_nimbuses());
      if (lastComparison != 0) {
        return lastComparison;
      }
 -    if (is_set_topologies()) {
 -      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, other.topologies);
 +    if (is_set_nimbuses()) {
-       lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbuses, typedOther.nimbuses);
++      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbuses, other.nimbuses);
        if (lastComparison != 0) {
          return lastComparison;
        }
@@@ -627,10 -519,7 +537,11 @@@
        throw new org.apache.thrift.protocol.TProtocolException("Required field 'topologies' is unset! Struct:" + toString());
      }
  
 +    if (!is_set_nimbuses()) {
 +      throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbuses' is unset! Struct:" + toString());
 +    }
 +
+     // check for sub-struct validity
    }
  
    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@@ -649,5 -540,175 +560,211 @@@
      }
    }
  
+   private static class ClusterSummaryStandardSchemeFactory implements SchemeFactory {
+     public ClusterSummaryStandardScheme getScheme() {
+       return new ClusterSummaryStandardScheme();
+     }
+   }
+ 
+   private static class ClusterSummaryStandardScheme extends StandardScheme<ClusterSummary> {
+ 
+     public void read(org.apache.thrift.protocol.TProtocol iprot, ClusterSummary 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: // SUPERVISORS
+             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+               {
+                 org.apache.thrift.protocol.TList _list74 = iprot.readListBegin();
+                 struct.supervisors = new ArrayList<SupervisorSummary>(_list74.size);
+                 SupervisorSummary _elem75;
+                 for (int _i76 = 0; _i76 < _list74.size; ++_i76)
+                 {
+                   _elem75 = new SupervisorSummary();
+                   _elem75.read(iprot);
+                   struct.supervisors.add(_elem75);
+                 }
+                 iprot.readListEnd();
+               }
+               struct.set_supervisors_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
 -          case 2: // NIMBUS_UPTIME_SECS
 -            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
 -              struct.nimbus_uptime_secs = iprot.readI32();
 -              struct.set_nimbus_uptime_secs_isSet(true);
 -            } else { 
 -              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 -            }
 -            break;
+           case 3: // TOPOLOGIES
+             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+               {
+                 org.apache.thrift.protocol.TList _list77 = iprot.readListBegin();
+                 struct.topologies = new ArrayList<TopologySummary>(_list77.size);
+                 TopologySummary _elem78;
+                 for (int _i79 = 0; _i79 < _list77.size; ++_i79)
+                 {
+                   _elem78 = new TopologySummary();
+                   _elem78.read(iprot);
+                   struct.topologies.add(_elem78);
+                 }
+                 iprot.readListEnd();
+               }
+               struct.set_topologies_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
++          case 4: // NIMBUSES
++            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
++              {
++                org.apache.thrift.protocol.TList _list80 = iprot.readListBegin();
++                struct.nimbuses = new ArrayList<NimbusSummary>(_list80.size);
++                NimbusSummary _elem81;
++                for (int _i82 = 0; _i82 < _list80.size; ++_i82)
++                {
++                  _elem81 = new NimbusSummary();
++                  _elem81.read(iprot);
++                  struct.nimbuses.add(_elem81);
++                }
++                iprot.readListEnd();
++              }
++              struct.set_nimbuses_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, ClusterSummary struct) throws org.apache.thrift.TException {
+       struct.validate();
+ 
+       oprot.writeStructBegin(STRUCT_DESC);
+       if (struct.supervisors != null) {
+         oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC);
+         {
+           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.supervisors.size()));
 -          for (SupervisorSummary _iter80 : struct.supervisors)
++          for (SupervisorSummary _iter83 : struct.supervisors)
+           {
 -            _iter80.write(oprot);
++            _iter83.write(oprot);
+           }
+           oprot.writeListEnd();
+         }
+         oprot.writeFieldEnd();
+       }
 -      oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC);
 -      oprot.writeI32(struct.nimbus_uptime_secs);
 -      oprot.writeFieldEnd();
+       if (struct.topologies != null) {
+         oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC);
+         {
+           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topologies.size()));
 -          for (TopologySummary _iter81 : struct.topologies)
++          for (TopologySummary _iter84 : struct.topologies)
++          {
++            _iter84.write(oprot);
++          }
++          oprot.writeListEnd();
++        }
++        oprot.writeFieldEnd();
++      }
++      if (struct.nimbuses != null) {
++        oprot.writeFieldBegin(NIMBUSES_FIELD_DESC);
++        {
++          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.nimbuses.size()));
++          for (NimbusSummary _iter85 : struct.nimbuses)
+           {
 -            _iter81.write(oprot);
++            _iter85.write(oprot);
+           }
+           oprot.writeListEnd();
+         }
+         oprot.writeFieldEnd();
+       }
+       oprot.writeFieldStop();
+       oprot.writeStructEnd();
+     }
+ 
+   }
+ 
+   private static class ClusterSummaryTupleSchemeFactory implements SchemeFactory {
+     public ClusterSummaryTupleScheme getScheme() {
+       return new ClusterSummaryTupleScheme();
+     }
+   }
+ 
+   private static class ClusterSummaryTupleScheme extends TupleScheme<ClusterSummary> {
+ 
+     @Override
+     public void write(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException {
+       TTupleProtocol oprot = (TTupleProtocol) prot;
+       {
+         oprot.writeI32(struct.supervisors.size());
 -        for (SupervisorSummary _iter82 : struct.supervisors)
++        for (SupervisorSummary _iter86 : struct.supervisors)
+         {
 -          _iter82.write(oprot);
++          _iter86.write(oprot);
+         }
+       }
 -      oprot.writeI32(struct.nimbus_uptime_secs);
+       {
+         oprot.writeI32(struct.topologies.size());
 -        for (TopologySummary _iter83 : struct.topologies)
++        for (TopologySummary _iter87 : struct.topologies)
++        {
++          _iter87.write(oprot);
++        }
++      }
++      {
++        oprot.writeI32(struct.nimbuses.size());
++        for (NimbusSummary _iter88 : struct.nimbuses)
+         {
 -          _iter83.write(oprot);
++          _iter88.write(oprot);
+         }
+       }
+     }
+ 
+     @Override
+     public void read(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException {
+       TTupleProtocol iprot = (TTupleProtocol) prot;
+       {
 -        org.apache.thrift.protocol.TList _list84 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -        struct.supervisors = new ArrayList<SupervisorSummary>(_list84.size);
 -        SupervisorSummary _elem85;
 -        for (int _i86 = 0; _i86 < _list84.size; ++_i86)
++        org.apache.thrift.protocol.TList _list89 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++        struct.supervisors = new ArrayList<SupervisorSummary>(_list89.size);
++        SupervisorSummary _elem90;
++        for (int _i91 = 0; _i91 < _list89.size; ++_i91)
+         {
 -          _elem85 = new SupervisorSummary();
 -          _elem85.read(iprot);
 -          struct.supervisors.add(_elem85);
++          _elem90 = new SupervisorSummary();
++          _elem90.read(iprot);
++          struct.supervisors.add(_elem90);
+         }
+       }
+       struct.set_supervisors_isSet(true);
 -      struct.nimbus_uptime_secs = iprot.readI32();
 -      struct.set_nimbus_uptime_secs_isSet(true);
+       {
 -        org.apache.thrift.protocol.TList _list87 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -        struct.topologies = new ArrayList<TopologySummary>(_list87.size);
 -        TopologySummary _elem88;
 -        for (int _i89 = 0; _i89 < _list87.size; ++_i89)
++        org.apache.thrift.protocol.TList _list92 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++        struct.topologies = new ArrayList<TopologySummary>(_list92.size);
++        TopologySummary _elem93;
++        for (int _i94 = 0; _i94 < _list92.size; ++_i94)
+         {
 -          _elem88 = new TopologySummary();
 -          _elem88.read(iprot);
 -          struct.topologies.add(_elem88);
++          _elem93 = new TopologySummary();
++          _elem93.read(iprot);
++          struct.topologies.add(_elem93);
+         }
+       }
+       struct.set_topologies_isSet(true);
++      {
++        org.apache.thrift.protocol.TList _list95 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++        struct.nimbuses = new ArrayList<NimbusSummary>(_list95.size);
++        NimbusSummary _elem96;
++        for (int _i97 = 0; _i97 < _list95.size; ++_i97)
++        {
++          _elem96 = new NimbusSummary();
++          _elem96.read(iprot);
++          struct.nimbuses.add(_elem96);
++        }
++      }
++      struct.set_nimbuses_isSet(true);
+     }
+   }
+ 
  }
  

http://git-wip-us.apache.org/repos/asf/storm/blob/85e4d5ef/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index 1ba5c5f,6e1a503..37c5c63
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@@ -38,7 -50,9 +50,9 @@@ import javax.annotation.Generated
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable {
+ @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-3-2")
+ 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");
  
    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
@@@ -49,17 -63,21 +63,23 @@@
    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 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 {
+     schemes.put(StandardScheme.class, new TopologyInfoStandardSchemeFactory());
+     schemes.put(TupleScheme.class, new TopologyInfoTupleSchemeFactory());
+   }
+ 
    private String id; // required
    private String name; // required
    private int uptime_secs; // required
    private List<ExecutorSummary> executors; // required
    private String status; // required
    private Map<String,List<ErrorInfo>> errors; // required
-   private String sched_status; // required
-   private String owner; // required
-   private int replication_count; // required
+   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 {
@@@ -145,9 -160,8 +165,9 @@@
  
    // isset id assignments
    private static final int __UPTIME_SECS_ISSET_ID = 0;
 +  private static final int __REPLICATION_COUNT_ISSET_ID = 1;
-   private BitSet __isset_bit_vector = new BitSet(2);
- 
+   private byte __isset_bitfield = 0;
 -  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER};
++  private static final _Fields optionals[] = {_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);
@@@ -476,28 -484,6 +495,28 @@@
      }
    }
  
 +  public int get_replication_count() {
 +    return this.replication_count;
 +  }
 +
 +  public void set_replication_count(int replication_count) {
 +    this.replication_count = replication_count;
 +    set_replication_count_isSet(true);
 +  }
 +
 +  public void unset_replication_count() {
-     __isset_bit_vector.clear(__REPLICATION_COUNT_ISSET_ID);
++    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
 +  }
 +
 +  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
 +  public boolean is_set_replication_count() {
-     return __isset_bit_vector.get(__REPLICATION_COUNT_ISSET_ID);
++    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
 +  }
 +
 +  public void set_replication_count_isSet(boolean value) {
-     __isset_bit_vector.set(__REPLICATION_COUNT_ISSET_ID, value);
++    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
 +  }
 +
    public void setFieldValue(_Fields field, Object value) {
      switch (field) {
      case ID:
@@@ -736,56 -700,52 +755,57 @@@
  
    @Override
    public int hashCode() {
-     HashCodeBuilder builder = new HashCodeBuilder();
+     List<Object> list = new ArrayList<Object>();
  
      boolean present_id = true && (is_set_id());
-     builder.append(present_id);
+     list.add(present_id);
      if (present_id)
-       builder.append(id);
+       list.add(id);
  
      boolean present_name = true && (is_set_name());
-     builder.append(present_name);
+     list.add(present_name);
      if (present_name)
-       builder.append(name);
+       list.add(name);
  
      boolean present_uptime_secs = true;
-     builder.append(present_uptime_secs);
+     list.add(present_uptime_secs);
      if (present_uptime_secs)
-       builder.append(uptime_secs);
+       list.add(uptime_secs);
  
      boolean present_executors = true && (is_set_executors());
-     builder.append(present_executors);
+     list.add(present_executors);
      if (present_executors)
-       builder.append(executors);
+       list.add(executors);
  
      boolean present_status = true && (is_set_status());
-     builder.append(present_status);
+     list.add(present_status);
      if (present_status)
-       builder.append(status);
+       list.add(status);
  
      boolean present_errors = true && (is_set_errors());
-     builder.append(present_errors);
+     list.add(present_errors);
      if (present_errors)
-       builder.append(errors);
+       list.add(errors);
  
      boolean present_sched_status = true && (is_set_sched_status());
-     builder.append(present_sched_status);
+     list.add(present_sched_status);
      if (present_sched_status)
-       builder.append(sched_status);
+       list.add(sched_status);
  
      boolean present_owner = true && (is_set_owner());
-     builder.append(present_owner);
+     list.add(present_owner);
      if (present_owner)
-       builder.append(owner);
+       list.add(owner);
  
 +    boolean present_replication_count = true && (is_set_replication_count());
-     builder.append(present_replication_count);
++    list.add(present_replication_count);
 +    if (present_replication_count)
-       builder.append(replication_count);
++      list.add(replication_count);
 +
-     return builder.toHashCode();
+     return list.hashCode();
    }
  
+   @Override
    public int compareTo(TopologyInfo other) {
      if (!getClass().equals(other.getClass())) {
        return getClass().getName().compareTo(other.getClass().getName());
@@@ -874,16 -833,6 +893,16 @@@
          return lastComparison;
        }
      }
-     lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(typedOther.is_set_replication_count());
++    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(other.is_set_replication_count());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (is_set_replication_count()) {
-       lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, typedOther.replication_count);
++      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, other.replication_count);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
      return 0;
    }
  
@@@ -1210,5 -967,314 +1043,337 @@@
      }
    }
  
+   private static class TopologyInfoStandardSchemeFactory implements SchemeFactory {
+     public TopologyInfoStandardScheme getScheme() {
+       return new TopologyInfoStandardScheme();
+     }
+   }
+ 
+   private static class TopologyInfoStandardScheme extends StandardScheme<TopologyInfo> {
+ 
+     public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyInfo struct) throws org.apache.thrift.TException {
+       org.apache.thrift.protocol.TField schemeField;
+       iprot.readStructBegin();
+       while (true)
+       {
+         schemeField = iprot.readFieldBegin();
+         if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+           break;
+         }
+         switch (schemeField.id) {
+           case 1: // ID
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.id = iprot.readString();
+               struct.set_id_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 2: // NAME
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.name = iprot.readString();
+               struct.set_name_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 3: // UPTIME_SECS
+             if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+               struct.uptime_secs = iprot.readI32();
+               struct.set_uptime_secs_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 4: // EXECUTORS
+             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+               {
 -                org.apache.thrift.protocol.TList _list290 = iprot.readListBegin();
 -                struct.executors = new ArrayList<ExecutorSummary>(_list290.size);
 -                ExecutorSummary _elem291;
 -                for (int _i292 = 0; _i292 < _list290.size; ++_i292)
++                org.apache.thrift.protocol.TList _list298 = iprot.readListBegin();
++                struct.executors = new ArrayList<ExecutorSummary>(_list298.size);
++                ExecutorSummary _elem299;
++                for (int _i300 = 0; _i300 < _list298.size; ++_i300)
+                 {
 -                  _elem291 = new ExecutorSummary();
 -                  _elem291.read(iprot);
 -                  struct.executors.add(_elem291);
++                  _elem299 = new ExecutorSummary();
++                  _elem299.read(iprot);
++                  struct.executors.add(_elem299);
+                 }
+                 iprot.readListEnd();
+               }
+               struct.set_executors_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 5: // STATUS
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.status = iprot.readString();
+               struct.set_status_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 6: // ERRORS
+             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+               {
 -                org.apache.thrift.protocol.TMap _map293 = iprot.readMapBegin();
 -                struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map293.size);
 -                String _key294;
 -                List<ErrorInfo> _val295;
 -                for (int _i296 = 0; _i296 < _map293.size; ++_i296)
++                org.apache.thrift.protocol.TMap _map301 = iprot.readMapBegin();
++                struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map301.size);
++                String _key302;
++                List<ErrorInfo> _val303;
++                for (int _i304 = 0; _i304 < _map301.size; ++_i304)
+                 {
 -                  _key294 = iprot.readString();
++                  _key302 = iprot.readString();
+                   {
 -                    org.apache.thrift.protocol.TList _list297 = iprot.readListBegin();
 -                    _val295 = new ArrayList<ErrorInfo>(_list297.size);
 -                    ErrorInfo _elem298;
 -                    for (int _i299 = 0; _i299 < _list297.size; ++_i299)
++                    org.apache.thrift.protocol.TList _list305 = iprot.readListBegin();
++                    _val303 = new ArrayList<ErrorInfo>(_list305.size);
++                    ErrorInfo _elem306;
++                    for (int _i307 = 0; _i307 < _list305.size; ++_i307)
+                     {
 -                      _elem298 = new ErrorInfo();
 -                      _elem298.read(iprot);
 -                      _val295.add(_elem298);
++                      _elem306 = new ErrorInfo();
++                      _elem306.read(iprot);
++                      _val303.add(_elem306);
+                     }
+                     iprot.readListEnd();
+                   }
 -                  struct.errors.put(_key294, _val295);
++                  struct.errors.put(_key302, _val303);
+                 }
+                 iprot.readMapEnd();
+               }
+               struct.set_errors_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();
+               struct.set_sched_status_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
+           case 514: // OWNER
+             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+               struct.owner = iprot.readString();
+               struct.set_owner_isSet(true);
+             } else { 
+               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+             }
+             break;
++          case 515: // REPLICATION_COUNT
++            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
++              struct.replication_count = iprot.readI32();
++              struct.set_replication_count_isSet(true);
++            } else { 
++              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
++            }
++            break;
+           default:
+             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+         }
+         iprot.readFieldEnd();
+       }
+       iprot.readStructEnd();
+       struct.validate();
+     }
+ 
+     public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyInfo struct) throws org.apache.thrift.TException {
+       struct.validate();
+ 
+       oprot.writeStructBegin(STRUCT_DESC);
+       if (struct.id != null) {
+         oprot.writeFieldBegin(ID_FIELD_DESC);
+         oprot.writeString(struct.id);
+         oprot.writeFieldEnd();
+       }
+       if (struct.name != null) {
+         oprot.writeFieldBegin(NAME_FIELD_DESC);
+         oprot.writeString(struct.name);
+         oprot.writeFieldEnd();
+       }
+       oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+       oprot.writeI32(struct.uptime_secs);
+       oprot.writeFieldEnd();
+       if (struct.executors != null) {
+         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 _iter308 : struct.executors)
+           {
 -            _iter300.write(oprot);
++            _iter308.write(oprot);
+           }
+           oprot.writeListEnd();
+         }
+         oprot.writeFieldEnd();
+       }
+       if (struct.status != null) {
+         oprot.writeFieldBegin(STATUS_FIELD_DESC);
+         oprot.writeString(struct.status);
+         oprot.writeFieldEnd();
+       }
+       if (struct.errors != null) {
+         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>> _iter309 : struct.errors.entrySet())
+           {
 -            oprot.writeString(_iter301.getKey());
++            oprot.writeString(_iter309.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, _iter309.getValue().size()));
++              for (ErrorInfo _iter310 : _iter309.getValue())
+               {
 -                _iter302.write(oprot);
++                _iter310.write(oprot);
+               }
+               oprot.writeListEnd();
+             }
+           }
+           oprot.writeMapEnd();
+         }
+         oprot.writeFieldEnd();
+       }
+       if (struct.sched_status != null) {
+         if (struct.is_set_sched_status()) {
+           oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
+           oprot.writeString(struct.sched_status);
+           oprot.writeFieldEnd();
+         }
+       }
+       if (struct.owner != null) {
+         if (struct.is_set_owner()) {
+           oprot.writeFieldBegin(OWNER_FIELD_DESC);
+           oprot.writeString(struct.owner);
+           oprot.writeFieldEnd();
+         }
+       }
++      if (struct.is_set_replication_count()) {
++        oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC);
++        oprot.writeI32(struct.replication_count);
++        oprot.writeFieldEnd();
++      }
+       oprot.writeFieldStop();
+       oprot.writeStructEnd();
+     }
+ 
+   }
+ 
+   private static class TopologyInfoTupleSchemeFactory implements SchemeFactory {
+     public TopologyInfoTupleScheme getScheme() {
+       return new TopologyInfoTupleScheme();
+     }
+   }
+ 
+   private static class TopologyInfoTupleScheme extends TupleScheme<TopologyInfo> {
+ 
+     @Override
+     public void write(org.apache.thrift.protocol.TProtocol prot, TopologyInfo struct) throws org.apache.thrift.TException {
+       TTupleProtocol oprot = (TTupleProtocol) prot;
+       oprot.writeString(struct.id);
+       oprot.writeString(struct.name);
+       oprot.writeI32(struct.uptime_secs);
+       {
+         oprot.writeI32(struct.executors.size());
 -        for (ExecutorSummary _iter303 : struct.executors)
++        for (ExecutorSummary _iter311 : struct.executors)
+         {
 -          _iter303.write(oprot);
++          _iter311.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>> _iter312 : struct.errors.entrySet())
+         {
 -          oprot.writeString(_iter304.getKey());
++          oprot.writeString(_iter312.getKey());
+           {
 -            oprot.writeI32(_iter304.getValue().size());
 -            for (ErrorInfo _iter305 : _iter304.getValue())
++            oprot.writeI32(_iter312.getValue().size());
++            for (ErrorInfo _iter313 : _iter312.getValue())
+             {
 -              _iter305.write(oprot);
++              _iter313.write(oprot);
+             }
+           }
+         }
+       }
+       BitSet optionals = new BitSet();
+       if (struct.is_set_sched_status()) {
+         optionals.set(0);
+       }
+       if (struct.is_set_owner()) {
+         optionals.set(1);
+       }
 -      oprot.writeBitSet(optionals, 2);
++      if (struct.is_set_replication_count()) {
++        optionals.set(2);
++      }
++      oprot.writeBitSet(optionals, 3);
+       if (struct.is_set_sched_status()) {
+         oprot.writeString(struct.sched_status);
+       }
+       if (struct.is_set_owner()) {
+         oprot.writeString(struct.owner);
+       }
++      if (struct.is_set_replication_count()) {
++        oprot.writeI32(struct.replication_count);
++      }
+     }
+ 
+     @Override
+     public void read(org.apache.thrift.protocol.TProtocol prot, TopologyInfo struct) throws org.apache.thrift.TException {
+       TTupleProtocol iprot = (TTupleProtocol) prot;
+       struct.id = iprot.readString();
+       struct.set_id_isSet(true);
+       struct.name = iprot.readString();
+       struct.set_name_isSet(true);
+       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 _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)
+         {
 -          _elem307 = new ExecutorSummary();
 -          _elem307.read(iprot);
 -          struct.executors.add(_elem307);
++          _elem315 = new ExecutorSummary();
++          _elem315.read(iprot);
++          struct.executors.add(_elem315);
+         }
+       }
+       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 _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)
+         {
 -          _key310 = iprot.readString();
++          _key318 = 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 _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)
+             {
 -              _elem314 = new ErrorInfo();
 -              _elem314.read(iprot);
 -              _val311.add(_elem314);
++              _elem322 = new ErrorInfo();
++              _elem322.read(iprot);
++              _val319.add(_elem322);
+             }
+           }
 -          struct.errors.put(_key310, _val311);
++          struct.errors.put(_key318, _val319);
+         }
+       }
+       struct.set_errors_isSet(true);
 -      BitSet incoming = iprot.readBitSet(2);
++      BitSet incoming = iprot.readBitSet(3);
+       if (incoming.get(0)) {
+         struct.sched_status = iprot.readString();
+         struct.set_sched_status_isSet(true);
+       }
+       if (incoming.get(1)) {
+         struct.owner = iprot.readString();
+         struct.set_owner_isSet(true);
+       }
++      if (incoming.get(2)) {
++        struct.replication_count = iprot.readI32();
++        struct.set_replication_count_isSet(true);
++      }
+     }
+   }
+ 
  }
  


[46/50] [abbrv] storm git commit: BUG-40864: Handing zookeeper failures that can result in nimbus ephemeral entries getting deleted. Adding a sleep before cody-sycn thread executes ls /code-distributor/topology-id to ensure it gets the correct id back so

Posted by bo...@apache.org.
BUG-40864: Handing zookeeper failures that can result in nimbus ephemeral entries getting deleted. Adding a sleep before cody-sycn thread executes ls /code-distributor/topology-id to ensure it gets the correct id back so users dont have to wait for upto 5 minutes to submit topology.


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

Branch: refs/heads/master
Commit: ef3cee6d5d9d673246dd1c7d2caa44240546e228
Parents: 21ba9c1
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Tue Jul 7 12:16:22 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Aug 12 09:58:57 2015 -0700

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/cluster.clj   | 21 ++++++++++++++++++--
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  2 ++
 storm-core/src/clj/backtype/storm/zookeeper.clj | 14 ++++++++++++-
 3 files changed, 34 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/ef3cee6d/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 65cdb47..04f5b89 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -19,6 +19,8 @@
            [backtype.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials NimbusSummary]
            [java.io Serializable])
   (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
+  (:import [org.apache.curator.framework.state ConnectionStateListener ConnectionState])
+  (:import [org.apache.curator.framework CuratorFramework])
   (:import [backtype.storm.utils Utils])
   (:import [java.security MessageDigest])
   (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
@@ -41,7 +43,8 @@
   (exists-node? [this path watch?])
   (close [this])
   (register [this callback])
-  (unregister [this id]))
+  (unregister [this id])
+  (add-listener [this listener]))
 
 (defn mk-topo-only-acls
   [topo-conf]
@@ -137,7 +140,12 @@
      (close
        [this]
        (reset! active false)
-       (.close zk)))))
+       (.close zk))
+
+      (add-listener
+        [this listener]
+        (zk/add-listener zk listener))
+      )))
 
 (defprotocol StormClusterState
   (assignments [this callback])
@@ -368,6 +376,15 @@
         [this nimbus-id nimbus-summary]
         ;explicit delete for ephmeral node to ensure this session creates the entry.
         (delete-node cluster-state (nimbus-path nimbus-id))
+
+        (add-listener cluster-state (reify ConnectionStateListener
+                        (^void stateChanged[this ^CuratorFramework client ^ConnectionState newState]
+                          (log-message "Connection state listener invoked, zookeeper connection state has changed to " newState)
+                          (if (.equals newState ConnectionState/RECONNECTED)
+                            (do
+                              (log-message "Connection state has changed to reconnected so setting nimbuses entry one more time")
+                              (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))))))
+
         (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
 
       (code-distributor-info

http://git-wip-us.apache.org/repos/asf/storm/blob/ef3cee6d/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 4f516ce..fa3d8ec 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -1492,6 +1492,8 @@
         (.removeFromLeaderLockQueue (:leader-elector nimbus))
         (doseq [missing missing-topologies]
           (log-message "missing topology " missing " has state on zookeeper but doesn't have a local dir on this host.")
+          ;; complete heck to get around zookeeper eventual consistency issue. zk/sync is not helping us so adding a sleep.
+          (sleep-secs 5)
           (let [nimbuses-with-missing (.code-distributor-info storm-cluster-state missing)]
             (log-message "trying to download missing topology code from " (clojure.string/join "," nimbuses-with-missing))
             (doseq [nimbus-host-port nimbuses-with-missing]

http://git-wip-us.apache.org/repos/asf/storm/blob/ef3cee6d/storm-core/src/clj/backtype/storm/zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj
index f714a9e..26def33 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -18,6 +18,7 @@
   (:import [org.apache.curator.retry RetryNTimes]
            [backtype.storm Config])
   (:import [org.apache.curator.framework.api CuratorEvent CuratorEventType CuratorListener UnhandledErrorListener])
+  (:import [org.apache.curator.framework.state ConnectionStateListener])
   (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory])
   (:import [org.apache.curator.framework.recipes.leader LeaderLatch LeaderLatch$State Participant LeaderLatchListener])
   (:import [org.apache.zookeeper ZooKeeper Watcher KeeperException$NoNodeException
@@ -128,6 +129,17 @@
           ))
       )))
 
+
+(defn sync-path
+  [^CuratorFramework zk ^String path]
+  (try
+    (.. zk (sync) (forPath (normalize-path path)))
+    (catch Exception e (throw (wrap-in-runtime e)))))
+
+
+(defn add-listener [^CuratorFramework zk ^ConnectionStateListener listener]
+  (.. zk (getConnectionStateListenable) (addListener listener)))
+
 (defn get-data
   [^CuratorFramework zk ^String path watch?]
   (let [path (normalize-path path)]
@@ -146,7 +158,7 @@
   (let [stats (org.apache.zookeeper.data.Stat. )
         path (normalize-path path)]
     (try-cause
-     (if-let [data 
+     (if-let [data
               (if (exists-node? zk path watch?)
                 (if watch?
                   (.. zk (getData) (watched) (storingStatIn stats) (forPath path))


[15/50] [abbrv] storm git commit: STORM-166: removing unused imports that causes build to require java-1.7.

Posted by bo...@apache.org.
STORM-166: removing unused imports that causes build to require java-1.7.


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

Branch: refs/heads/master
Commit: 5658bbbc84ca1076381ac2398ef423b4af30a1b4
Parents: 7681686
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Jan 8 11:52:34 2015 -0500
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Jan 8 11:52:34 2015 -0500

----------------------------------------------------------------------
 .../src/jvm/backtype/storm/codedistributor/ICodeDistributor.java    | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5658bbbc/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java b/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
index f536a2a..25e578b 100644
--- a/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
+++ b/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
@@ -3,7 +3,6 @@ package backtype.storm.codedistributor;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.file.Path;
 import java.util.List;
 import java.util.Map;
 


[21/50] [abbrv] storm git commit: Adding nimbus summary info to zookeeper.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 3c49d39..4445dac 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -156,10 +156,18 @@ struct SupervisorSummary {
   5: required string supervisor_id;
 }
 
+struct NimbusSummary {
+  1: required string host;
+  2: required i32 port;
+  3: required i32 uptime_secs;
+  4: required bool isLeader;
+  5: required string version;
+}
+
 struct ClusterSummary {
   1: required list<SupervisorSummary> supervisors;
-  2: required i32 nimbus_uptime_secs;
   3: required list<TopologySummary> topologies;
+  4: required list<NimbusSummary> nimbuses;
 }
 
 struct ErrorInfo {

http://git-wip-us.apache.org/repos/asf/storm/blob/4502bffb/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index 0a4fdf3..eb4581a 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -24,11 +24,6 @@
         </span>
       </th>
       <th>
-        <span class="tip right" title="The duration the current Nimbus instance has been running. (Note that the storm cluster may have been deployed and available for a much longer period than the current Nimbus process has been running.)">
-          Nimbus uptime
-        </span>
-      </th>
-      <th>
         <span class="tip above" title="The number of nodes in the cluster currently.">
           Supervisors
       </span>
@@ -63,7 +58,6 @@
   <tbody>
     <tr>
       <td>{{stormVersion}}</td>
-      <td>{{nimbusUptime}}</td>
       <td>{{supervisors}}</td>
       <td>{{slotsUsed}}</td>
       <td>{{slotsFree}}</td>
@@ -80,19 +74,19 @@
         <thead>
         <tr>
             <th>
-                    <span class="tip right" title="nimbus host name.">
-                        Host
-                    </span>
+                <span class="tip right" title="nimbus host name.">Host</span>
             </th>
             <th>
-        <span class="tip above" title="Nimbus Thrift port number.">
-          Port
-        </span>
+                <span class="tip above" title="Nimbus Thrift port number.">Port</span>
             </th>
             <th>
-        <span class="tip above" title="True if this host is leader, false otherwise.">
-          IsLeader
-        </span>
+                <span class="tip above" title="True if this host is leader, false otherwise.">IsLeader</span>
+            </th>
+            <th>
+                <span class="tip left" title="Storm version that this nimbus host is running.">Version</span>
+            </th>
+            <th>
+                <span class="tip left" title="Number of seconds this nimbus host has been running the nimbus process.">UpTime Seconds</span>
             </th>
         </tr>
         </thead>
@@ -102,6 +96,8 @@
             <td>{{host}}</td>
             <td><a href="{{nimbusLogLink}}">{{port}}</a></td>
             <td>{{isLeader}}</td>
+            <td>{{version}}</td>
+            <td>{{nimbusUpTime}}</td>
         </tr>
         {{/nimbuses}}
         </tbody>


[38/50] [abbrv] storm git commit: Merge remote-tracking branch 'apache/master' into nimbus-ha

Posted by bo...@apache.org.
Merge remote-tracking branch 'apache/master' into nimbus-ha

Conflicts:
	storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
	storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj


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

Branch: refs/heads/master
Commit: 765e4c2fabafbfe31fb0f43e5b670fd3b91d1a2e
Parents: a11fcc3 2aaa718
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Apr 1 10:44:56 2015 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Apr 1 10:44:56 2015 -0700

----------------------------------------------------------------------
 CHANGELOG.md                                    |  13 ++
 README.markdown                                 |   2 +
 STORM-UI-REST-API.md                            |  30 +++
 docs/_posts/2015-03-25-storm094-released.md     |  24 ++
 docs/downloads.html                             |  64 ++++--
 examples/storm-starter/pom.xml                  |   2 +-
 external/storm-hbase/pom.xml                    |   2 +-
 .../org/apache/storm/hbase/bolt/HBaseBolt.java  |   2 +-
 .../storm/hbase/bolt/HBaseLookupBolt.java       |   4 +-
 .../hbase/trident/state/HBaseMapState.java      |   2 +
 .../storm/hbase/trident/state/HBaseState.java   |   2 +-
 external/storm-hdfs/pom.xml                     |   2 +-
 .../org/apache/storm/hdfs/bolt/HdfsBolt.java    |   2 +-
 .../storm/hdfs/bolt/SequenceFileBolt.java       |   2 +-
 external/storm-hive/pom.xml                     |   2 +-
 external/storm-jdbc/pom.xml                     |   2 +-
 external/storm-kafka/pom.xml                    |   2 +-
 .../src/jvm/storm/kafka/KafkaUtils.java         |  16 +-
 external/storm-redis/README.md                  | 108 ++++++++-
 external/storm-redis/pom.xml                    |   2 +-
 .../storm/redis/bolt/AbstractRedisBolt.java     |   8 +-
 .../storm/redis/bolt/RedisLookupBolt.java       | 112 ++++++++++
 .../apache/storm/redis/bolt/RedisStoreBolt.java | 100 +++++++++
 .../redis/common/config/JedisClusterConfig.java |  82 +++++++
 .../redis/common/config/JedisPoolConfig.java    |  97 +++++++++
 .../common/container/JedisClusterContainer.java |  47 ++++
 .../JedisCommandsContainerBuilder.java          |  38 ++++
 .../JedisCommandsInstanceContainer.java         |  25 +++
 .../redis/common/container/JedisContainer.java  |  65 ++++++
 .../common/mapper/RedisDataTypeDescription.java |  50 +++++
 .../redis/common/mapper/RedisLookupMapper.java  |  40 ++++
 .../storm/redis/common/mapper/RedisMapper.java  |  22 ++
 .../redis/common/mapper/RedisStoreMapper.java   |  21 ++
 .../storm/redis/common/mapper/TupleMapper.java  |  27 +++
 .../trident/mapper/TridentTupleMapper.java      |  27 ---
 .../trident/state/RedisClusterMapState.java     |   2 +-
 .../redis/trident/state/RedisClusterState.java  |   2 +-
 .../trident/state/RedisClusterStateQuerier.java |  10 +-
 .../trident/state/RedisClusterStateUpdater.java |  10 +-
 .../redis/trident/state/RedisMapState.java      |  21 +-
 .../storm/redis/trident/state/RedisState.java   |   2 +-
 .../redis/trident/state/RedisStateQuerier.java  |  10 +-
 .../state/RedisStateSetCountQuerier.java        |  74 -------
 .../trident/state/RedisStateSetUpdater.java     |  80 -------
 .../redis/trident/state/RedisStateUpdater.java  |  10 +-
 .../redis/util/config/JedisClusterConfig.java   |  82 -------
 .../redis/util/config/JedisPoolConfig.java      |  97 ---------
 .../util/container/JedisClusterContainer.java   |  47 ----
 .../JedisCommandsContainerBuilder.java          |  38 ----
 .../JedisCommandsInstanceContainer.java         |  25 ---
 .../redis/util/container/JedisContainer.java    |  65 ------
 .../storm/redis/topology/LookupWordCount.java   | 115 ++++++----
 .../redis/topology/PersistentWordCount.java     |  81 ++++---
 .../storm/redis/topology/WordCounter.java       |  19 +-
 .../redis/trident/WordCountTridentRedis.java    |   7 +-
 .../trident/WordCountTridentRedisCluster.java   |   6 +-
 .../WordCountTridentRedisClusterMap.java        |   8 +-
 .../redis/trident/WordCountTridentRedisMap.java |   9 +-
 .../redis/trident/WordCountTupleMapper.java     |  10 +-
 pom.xml                                         |   2 +-
 .../maven-shade-clojure-transformer/pom.xml     |   2 +-
 storm-buildtools/storm-maven-plugins/pom.xml    |   2 +-
 storm-core/pom.xml                              |   2 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  62 +++++-
 .../src/clj/backtype/storm/ui/helpers.clj       |  15 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   6 +
 .../jvm/backtype/storm/utils/NimbusClient.java  |  29 ++-
 storm-core/src/ui/public/css/style.css          |  62 ++++++
 .../storm/security/auth/nimbus_auth_test.clj    | 217 ++++++++++---------
 storm-dist/binary/pom.xml                       |   2 +-
 storm-dist/source/pom.xml                       |   2 +-
 71 files changed, 1425 insertions(+), 854 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/765e4c2f/STORM-UI-REST-API.md
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/765e4c2f/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/765e4c2f/storm-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/765e4c2f/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index 553434e,ab2b2d9..02c3d90
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -16,10 -16,12 +16,12 @@@
  
  (ns backtype.storm.ui.core
    (:use compojure.core)
-   (:use ring.middleware.reload)
+   (:use [clojure.java.shell :only [sh]])
+   (:use ring.middleware.reload
+         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 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]]])
@@@ -499,8 -505,41 +501,41 @@@
                (hashmap-to-persistent bolts))
         spout-comp-summs bolt-comp-summs window id))))
  
+ (defn validate-tplg-submit-params [params]
+   (let [tplg-jar-file (params :topologyJar)
+         tplg-config (if (not-nil? (params :topologyConfig)) (from-json (params :topologyConfig)))]
+     (cond
+      (nil? tplg-jar-file) {:valid false :error "missing topology jar file"}
+      (nil? tplg-config) {:valid false :error "missing topology config"}
+      (nil? (tplg-config "topologyMainClass")) {:valid false :error "topologyMainClass missing in topologyConfig"}
+      :else {:valid true})))
+ 
+ (defn run-tplg-submit-cmd [tplg-jar-file tplg-config user]
+   (let [tplg-main-class (if (not-nil? tplg-config) (trim (tplg-config "topologyMainClass")))
+         tplg-main-class-args (if (not-nil? tplg-config) (clojure.string/join " " (tplg-config "topologyMainClassArgs")))
+         tplg-jvm-opts (if (not-nil? tplg-config) (clojure.string/join " " (tplg-config "topologyJvmOpts")))
+         storm-home (System/getProperty "storm.home")
+         storm-conf-dir (str storm-home file-path-separator "conf")
+         storm-log-dir (if (not-nil? (*STORM-CONF* "storm.log.dir")) (*STORM-CONF* "storm.log.dir")
+                           (str storm-home file-path-separator "logs"))
+         storm-libs (str storm-home file-path-separator "lib" file-path-separator "*")
+         java-cmd (str (System/getProperty "java.home") file-path-separator "bin" file-path-separator "java")
+         storm-cmd (str storm-home file-path-separator "bin" file-path-separator "storm")
+         tplg-cmd-response (sh storm-cmd "jar" tplg-jar-file
+                               tplg-main-class
+                               tplg-main-class-args
+                               (if (not= user "unknown") (str "-c storm.doAsUser=" user) ""))]
+     (log-message "tplg-cmd-response " tplg-cmd-response)
+     (cond
+      (= (tplg-cmd-response :exit) 0) {"status" "success"}
+      (and (not= (tplg-cmd-response :exit) 0)
+           (not-nil? (re-find #"already exists on cluster" (tplg-cmd-response :err)))) {"status" "failed" "error" "Topology with the same name exists in cluster"}
+           (not= (tplg-cmd-response :exit) 0) {"status" "failed" "error" (clojure.string/trim-newline (tplg-cmd-response :err))}
+           :else {"status" "success" "response" "topology deployed"}
+           )))
+ 
  (defn cluster-configuration []
 -  (with-nimbus nimbus
 +  (thrift/with-configured-nimbus-connection nimbus
      (.getNimbusConf ^Nimbus$Client nimbus)))
  
  (defn cluster-summary

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

http://git-wip-us.apache.org/repos/asf/storm/blob/765e4c2f/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index 071d2b6,6314deb..08610e9
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@@ -46,34 -41,23 +46,43 @@@ public class NimbusClient extends Thrif
      }
  
      public static NimbusClient getConfiguredClientAs(Map conf, String asUser) {
 -        try {
 -            if(conf.containsKey(Config.STORM_DO_AS_USER)) {
 -                if(asUser != null && !asUser.isEmpty()) {
 -                    LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence."
 -                            , asUser, conf.get(Config.STORM_DO_AS_USER));
++        if (conf.containsKey(Config.STORM_DO_AS_USER)) {
++            if (asUser != null && !asUser.isEmpty()) {
++                LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence."
++                        , asUser, conf.get(Config.STORM_DO_AS_USER));
++            }
++            asUser = (String) conf.get(Config.STORM_DO_AS_USER);
++        }
++
 +        List<String> seeds = (List<String>) conf.get(Config.NIMBUS_SEEDS);
-         for(String seed : seeds) {
++        for (String seed : seeds) {
 +            String[] split = seed.split(":");
 +            String host = split[0];
 +            int port = Integer.parseInt(split[1]);
 +            try {
-                 NimbusClient client = new NimbusClient(conf,host,port);
++                NimbusClient client = new NimbusClient(conf, host, port);
 +                ClusterSummary clusterInfo = client.getClient().getClusterInfo();
 +                List<NimbusSummary> nimbuses = clusterInfo.get_nimbuses();
-                 if(nimbuses != null) {
-                     for(NimbusSummary nimbusSummary : nimbuses) {
-                         if(nimbusSummary.is_isLeader()) {
++                if (nimbuses != null) {
++                    for (NimbusSummary nimbusSummary : nimbuses) {
++                        if (nimbusSummary.is_isLeader()) {
 +                            return new NimbusClient(conf, nimbusSummary.get_host(), nimbusSummary.get_port(), null, asUser);
 +                        }
 +                    }
++                    throw new RuntimeException("Found nimbuses " + nimbuses + " none of which is elected as leader, please try " +
++                            "again after some time.");
                  }
-                 throw new RuntimeException("Found nimbuses " + nimbuses + " none of which is elected as leader, please try " +
-                         "again after some time.");
 -                asUser = (String) conf.get(Config.STORM_DO_AS_USER);
 +            } catch (Exception e) {
-                 LOG.warn("Ignoring exception while trying to get leader nimbus info from " + seed, e);
++                LOG.warn("Ignoring exception while trying to get leader nimbus info from " + seed
++                        + ". will retry with a different seed host.", e);
              }
 -            String nimbusHost = (String) conf.get(Config.NIMBUS_HOST);
 -            return new NimbusClient(conf, nimbusHost, null, null, asUser);
 -        } catch (TTransportException ex) {
 -            throw new RuntimeException(ex);
          }
-         throw new RuntimeException("Could not find leader nimbus from seed hosts " + seeds +". " +
++        throw new RuntimeException("Could not find leader nimbus from seed hosts " + seeds + ". " +
 +                "Did you specify a valid list of nimbus host:port for config " + Config.NIMBUS_SEEDS);
      }
  
      public NimbusClient(Map conf, String host, int port) throws TTransportException {
--        this(conf, host, port, null);
++        this(conf, host, port, null, null);
      }
  
      public NimbusClient(Map conf, String host, int port, Integer timeout) throws TTransportException {

http://git-wip-us.apache.org/repos/asf/storm/blob/765e4c2f/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
index 8908593,0a9c280..dff3042
--- a/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/nimbus_auth_test.clj
@@@ -56,120 -56,127 +56,125 @@@
        (testing/kill-local-storm-cluster cluster-map#)
        (.stop nimbus-server#)))
  
 -(deftest Simple-authentication-test 
 +(deftest Simple-authentication-test
-   (with-test-cluster [6627 nil nil "backtype.storm.security.auth.SimpleTransportPlugin"]
-     (let [storm-conf (merge (read-storm-config)
-                             {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
-                              STORM-NIMBUS-RETRY-TIMES 0})
-           client (NimbusClient. storm-conf "localhost" 6627 nimbus-timeout)
-           nimbus_client (.getClient client)]
-       (testing "(Positive authorization) Simple protocol w/o authentication/authorization enforcement"
-                (is (thrown-cause? NotAliveException
-                             (.activate nimbus_client "topo-name"))))
-       (.close client))))
+   (let [port (available-port)]
+     (with-test-cluster [port nil nil "backtype.storm.security.auth.SimpleTransportPlugin"]
+       (let [storm-conf (merge (read-storm-config)
+                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
+                                STORM-NIMBUS-RETRY-TIMES 0})
+             client (NimbusClient. storm-conf "localhost" port nimbus-timeout)
+             nimbus_client (.getClient client)]
+         (testing "(Positive authorization) Simple protocol w/o authentication/authorization enforcement"
+                  (is (thrown-cause? NotAliveException
+                               (.activate nimbus_client "topo-name"))))
+         (.close client)))))
 -  
 +
  (deftest test-noop-authorization-w-simple-transport
-   (with-test-cluster [6628 nil
-                 "backtype.storm.security.auth.authorizer.NoopAuthorizer"
-                 "backtype.storm.security.auth.SimpleTransportPlugin"]
-     (let [storm-conf (merge (read-storm-config)
-                              {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
-                               STORM-NIMBUS-RETRY-TIMES 0})
-           client (NimbusClient. storm-conf "localhost" 6628 nimbus-timeout)
-           nimbus_client (.getClient client)]
-       (testing "(Positive authorization) Authorization plugin should accept client request"
-                (is (thrown-cause? NotAliveException
-                             (.activate nimbus_client "topo-name"))))
-       (.close client))))
+   (let [port (available-port)]
+     (with-test-cluster [port nil
+                   "backtype.storm.security.auth.authorizer.NoopAuthorizer"
+                   "backtype.storm.security.auth.SimpleTransportPlugin"]
+       (let [storm-conf (merge (read-storm-config)
+                                {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
+                                 STORM-NIMBUS-RETRY-TIMES 0})
+             client (NimbusClient. storm-conf "localhost" port nimbus-timeout)
+             nimbus_client (.getClient client)]
+         (testing "(Positive authorization) Authorization plugin should accept client request"
+                  (is (thrown-cause? NotAliveException
+                               (.activate nimbus_client "topo-name"))))
+         (.close client)))))
  
  (deftest test-deny-authorization-w-simple-transport
-   (with-test-cluster [6629 nil
-                 "backtype.storm.security.auth.authorizer.DenyAuthorizer"
-                 "backtype.storm.security.auth.SimpleTransportPlugin"]
-     (let [storm-conf (merge (read-storm-config)
-                              {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
-                              Config/NIMBUS_THRIFT_PORT 6629
-                              STORM-NIMBUS-RETRY-TIMES 0})
-           client (NimbusClient. storm-conf "localhost" 6629 nimbus-timeout)
-           nimbus_client (.getClient client)
-           topologyInitialStatus (TopologyInitialStatus/findByValue 2)
-           submitOptions (SubmitOptions. topologyInitialStatus)]
-       (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client  "topo-name" nil nil nil)))
-       (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client  "topo-name" nil nil nil submitOptions)))
-       (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client)))
+   (let [port (available-port)]
+     (with-test-cluster [port nil
+                   "backtype.storm.security.auth.authorizer.DenyAuthorizer"
+                   "backtype.storm.security.auth.SimpleTransportPlugin"]
+       (let [storm-conf (merge (read-storm-config)
+                                {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.SimpleTransportPlugin"
 -                               Config/NIMBUS_HOST "localhost"
+                                Config/NIMBUS_THRIFT_PORT port
+                                STORM-NIMBUS-RETRY-TIMES 0})
 -            client (NimbusClient/getConfiguredClient storm-conf)
++            client (NimbusClient. storm-conf "localhost" port nimbus-timeout)
+             nimbus_client (.getClient client)
+             topologyInitialStatus (TopologyInitialStatus/findByValue 2)
+             submitOptions (SubmitOptions. topologyInitialStatus)]
+         (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client  "topo-name" nil nil nil)))
+         (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client  "topo-name" nil nil nil submitOptions)))
+         (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client)))
 +
-       (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil)))
-       (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil)))
-       (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil)))
-       (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil)))
-       (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client)))
-       (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client)))
-       (stubbing [nimbus/check-storm-active! nil
-                  nimbus/try-read-storm-conf-from-name {}]
-         (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name")))
-         (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.))))
-         (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name")))
-         (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name")))
-         (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil)))
-       )
-       (stubbing [nimbus/try-read-storm-conf {}]
-         (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID")))
-         (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID")))
-         (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID")))
-         (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID"))))
-       (.close client))))
+         (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil)))
+         (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil)))
+         (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil)))
+         (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil)))
+         (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client)))
+         (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client)))
+         (stubbing [nimbus/check-storm-active! nil
+                    nimbus/try-read-storm-conf-from-name {}]
+           (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name")))
+           (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.))))
+           (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name")))
+           (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name")))
+           (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil)))
+         )
+         (stubbing [nimbus/try-read-storm-conf {}]
+           (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID")))
+           (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID")))
+           (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID")))
+           (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID"))))
+         (.close client)))))
  
  (deftest test-noop-authorization-w-sasl-digest
-   (with-test-cluster [6630
-                 "test/clj/backtype/storm/security/auth/jaas_digest.conf"
-                 "backtype.storm.security.auth.authorizer.NoopAuthorizer"
-                 "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"]
-     (let [storm-conf (merge (read-storm-config)
-                             {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                              "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
-                              Config/NIMBUS_THRIFT_PORT 6630
-                              STORM-NIMBUS-RETRY-TIMES 0})
-           client (NimbusClient. storm-conf "localhost" 6630 nimbus-timeout)
-           nimbus_client (.getClient client)]
-       (testing "(Positive authorization) Authorization plugin should accept client request"
-                (is (thrown-cause? NotAliveException
-                             (.activate nimbus_client "topo-name"))))
-       (.close client))))
+   (let [port (available-port)]
+     (with-test-cluster [port
+                   "test/clj/backtype/storm/security/auth/jaas_digest.conf"
+                   "backtype.storm.security.auth.authorizer.NoopAuthorizer"
+                   "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"]
+       (let [storm-conf (merge (read-storm-config)
+                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
+                                "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
 -                               Config/NIMBUS_HOST "localhost"
+                                Config/NIMBUS_THRIFT_PORT port
+                                STORM-NIMBUS-RETRY-TIMES 0})
 -            client (NimbusClient/getConfiguredClient storm-conf)
++            client (NimbusClient. storm-conf "localhost" port nimbus-timeout)
+             nimbus_client (.getClient client)]
+         (testing "(Positive authorization) Authorization plugin should accept client request"
+                  (is (thrown-cause? NotAliveException
+                               (.activate nimbus_client "topo-name"))))
+         (.close client)))))
  
  (deftest test-deny-authorization-w-sasl-digest
-   (with-test-cluster [6631
-                 "test/clj/backtype/storm/security/auth/jaas_digest.conf"
-                 "backtype.storm.security.auth.authorizer.DenyAuthorizer"
-                 "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"]
-     (let [storm-conf (merge (read-storm-config)
-                             {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
-                              "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
-                              Config/NIMBUS_THRIFT_PORT 6631
-                              STORM-NIMBUS-RETRY-TIMES 0})
-           client (NimbusClient. storm-conf "localhost" 6631 nimbus-timeout)
-           nimbus_client (.getClient client)
-           topologyInitialStatus (TopologyInitialStatus/findByValue 2)
-           submitOptions (SubmitOptions. topologyInitialStatus)]
-       (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client  "topo-name" nil nil nil)))
-       (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client  "topo-name" nil nil nil submitOptions)))
-       (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client)))
-       (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil)))
-       (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil)))
-       (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil)))
-       (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil)))
-       (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client)))
-       (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client)))
-       (stubbing [nimbus/check-storm-active! nil
-                  nimbus/try-read-storm-conf-from-name {}]
-         (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name")))
-         (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.))))
-         (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name")))
-         (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name")))
-         (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil))))
-       (stubbing [nimbus/try-read-storm-conf {}]
-         (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID")))
-         (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID")))
-         (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID")))
-         (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID"))))
-       (.close client))))
+   (let [port (available-port)]
+     (with-test-cluster [port
+                   "test/clj/backtype/storm/security/auth/jaas_digest.conf"
+                   "backtype.storm.security.auth.authorizer.DenyAuthorizer"
+                   "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"]
+       (let [storm-conf (merge (read-storm-config)
+                               {STORM-THRIFT-TRANSPORT-PLUGIN "backtype.storm.security.auth.digest.DigestSaslTransportPlugin"
+                                "java.security.auth.login.config" "test/clj/backtype/storm/security/auth/jaas_digest.conf"
 -                               Config/NIMBUS_HOST "localhost"
+                                Config/NIMBUS_THRIFT_PORT port
+                                STORM-NIMBUS-RETRY-TIMES 0})
 -            client (NimbusClient/getConfiguredClient storm-conf)
++            client (NimbusClient. storm-conf "localhost" port nimbus-timeout)
+             nimbus_client (.getClient client)
+             topologyInitialStatus (TopologyInitialStatus/findByValue 2)
+             submitOptions (SubmitOptions. topologyInitialStatus)]
+         (is (thrown-cause? AuthorizationException (.submitTopology nimbus_client  "topo-name" nil nil nil)))
+         (is (thrown-cause? AuthorizationException (.submitTopologyWithOpts nimbus_client  "topo-name" nil nil nil submitOptions)))
+         (is (thrown-cause? AuthorizationException (.beginFileUpload nimbus_client)))
+         (is (thrown-cause? AuthorizationException (.uploadChunk nimbus_client nil nil)))
+         (is (thrown-cause? AuthorizationException (.finishFileUpload nimbus_client nil)))
+         (is (thrown-cause? AuthorizationException (.beginFileDownload nimbus_client nil)))
+         (is (thrown-cause? AuthorizationException (.downloadChunk nimbus_client nil)))
+         (is (thrown-cause? AuthorizationException (.getNimbusConf nimbus_client)))
+         (is (thrown-cause? AuthorizationException (.getClusterInfo nimbus_client)))
+         (stubbing [nimbus/check-storm-active! nil
+                    nimbus/try-read-storm-conf-from-name {}]
+           (is (thrown-cause? AuthorizationException (.killTopology nimbus_client "topo-name")))
+           (is (thrown-cause? AuthorizationException (.killTopologyWithOpts nimbus_client "topo-name" (KillOptions.))))
+           (is (thrown-cause? AuthorizationException (.activate nimbus_client "topo-name")))
+           (is (thrown-cause? AuthorizationException (.deactivate nimbus_client "topo-name")))
+           (is (thrown-cause? AuthorizationException (.rebalance nimbus_client "topo-name" nil))))
+         (stubbing [nimbus/try-read-storm-conf {}]
+           (is (thrown-cause? AuthorizationException (.getTopologyConf nimbus_client "topo-ID")))
+           (is (thrown-cause? AuthorizationException (.getTopology nimbus_client "topo-ID")))
+           (is (thrown-cause? AuthorizationException (.getUserTopology nimbus_client "topo-ID")))
+           (is (thrown-cause? AuthorizationException (.getTopologyInfo nimbus_client "topo-ID"))))
+         (.close client)))))
  


[13/50] [abbrv] storm git commit: STORM-166: renew thread should only renew creds on leader nimbus.

Posted by bo...@apache.org.
STORM-166: renew thread should only renew creds on leader nimbus.


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

Branch: refs/heads/master
Commit: a17167e4eb6cea78d7da7fa2632c0cbbb3a40c7a
Parents: eb242e0
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 17:09:36 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 17:09:36 2014 -0800

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 36 +++++++++++---------
 1 file changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a17167e4/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 0281e73..0eb3c7f 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -984,23 +984,25 @@
 )
 
 (defn renew-credentials [nimbus]
-  (let [storm-cluster-state (:storm-cluster-state nimbus)
-        renewers (:cred-renewers nimbus)
-        update-lock (:cred-update-lock nimbus)
-        assigned-ids (set (.active-storms storm-cluster-state))]
-    (when-not (empty? assigned-ids)
-      (doseq [id assigned-ids]
-        (locking update-lock
-          (let [orig-creds (.credentials storm-cluster-state id nil)
-                topology-conf (try-read-storm-conf (:conf nimbus) id)]
-            (if orig-creds
-              (let [new-creds (HashMap. orig-creds)]
-                (doseq [renewer renewers]
-                  (log-message "Renewing Creds For " id " with " renewer)
-                  (.renew renewer new-creds (Collections/unmodifiableMap topology-conf)))
-                (when-not (= orig-creds new-creds)
-                  (.set-credentials! storm-cluster-state id new-creds topology-conf)
-                  )))))))))
+  (if (is-leader nimbus :throw-exception false)
+    (let [storm-cluster-state (:storm-cluster-state nimbus)
+          renewers (:cred-renewers nimbus)
+          update-lock (:cred-update-lock nimbus)
+          assigned-ids (set (.active-storms storm-cluster-state))]
+      (when-not (empty? assigned-ids)
+        (doseq [id assigned-ids]
+          (locking update-lock
+            (let [orig-creds (.credentials storm-cluster-state id nil)
+                  topology-conf (try-read-storm-conf (:conf nimbus) id)]
+              (if orig-creds
+                (let [new-creds (HashMap. orig-creds)]
+                  (doseq [renewer renewers]
+                    (log-message "Renewing Creds For " id " with " renewer)
+                    (.renew renewer new-creds (Collections/unmodifiableMap topology-conf)))
+                  (when-not (= orig-creds new-creds)
+                    (.set-credentials! storm-cluster-state id new-creds topology-conf)
+                    ))))))))
+    (log-message "not a leader skipping , credential renweal.")))
 
 (defn validate-topology-size [topo-conf nimbus-conf topology]
   (let [workers-count (get topo-conf TOPOLOGY-WORKERS)


[18/50] [abbrv] storm git commit: STORM-166: renaming bt-tracker to code-distributor.

Posted by bo...@apache.org.
STORM-166: renaming bt-tracker to code-distributor.


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

Branch: refs/heads/master
Commit: 27a381aef63b0929aedc5e85b6536f8ac29a14fd
Parents: c1e8782
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Thu Jan 22 23:10:02 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Thu Jan 22 23:10:02 2015 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/daemon/nimbus.clj     | 2 +-
 storm-core/src/clj/backtype/storm/daemon/supervisor.clj | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/27a381ae/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 d1f5c31..e07e6d1 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -100,7 +100,7 @@
                                  ))
      :scheduler (mk-scheduler conf inimbus)
      :leader-elector (zk-leader-elector conf)
-     :bt-tracker (mk-code-distributor conf)
+     :code-distributor (mk-code-distributor conf)
      :id->sched-status (atom {})
      :cred-renewers (AuthUtils/GetCredentialRenewers conf)
      :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)

http://git-wip-us.apache.org/repos/asf/storm/blob/27a381ae/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index ceb098e..7e87d3d 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -302,7 +302,7 @@
                                          ))
    :assignment-versions (atom {})
    :sync-retry (atom 0)
-   :bt-tracker (mk-code-distributor conf)
+   :code-distributor (mk-code-distributor conf)
    })
 
 (defn sync-processes [supervisor]


[09/50] [abbrv] storm git commit: STORM-166: removing unnecessary TODO.

Posted by bo...@apache.org.
STORM-166: removing unnecessary TODO.


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

Branch: refs/heads/master
Commit: a75c72cfc75a150dc3e1a7c1b1b0b6471a4f99ab
Parents: a92a1e9
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 13:59:41 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 13:59:41 2014 -0800

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/utils/Utils.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a75c72cf/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index 1c4c567..8c8fc1b 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -261,7 +261,6 @@ public class Utils {
     }
 
     public static void downloadFromHost(Map conf, String file, String localFile, String host, int port) throws IOException, TException, AuthorizationException {
-        //TODO : instead of null as last arg we probably need some real timeout, check what is the default and if its ok to reuse.
         NimbusClient client = new NimbusClient (conf, host, port, null);
         download(client, file, localFile);
     }


[06/50] [abbrv] storm git commit: STORM-166: adding formConf method to NimbusInfo.

Posted by bo...@apache.org.
STORM-166: adding formConf method to NimbusInfo.


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

Branch: refs/heads/master
Commit: c82482d845de996515047ee001d61b9a50322a9f
Parents: 9cd52c8
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 13:54:52 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 13:54:52 2014 -0800

----------------------------------------------------------------------
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  2 +-
 .../jvm/backtype/storm/nimbus/NimbusInfo.java   | 38 ++++++++++++++++++++
 .../test/clj/backtype/storm/nimbus_test.clj     |  3 +-
 3 files changed, 41 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/c82482d8/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 c652b41..b638e52 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -79,7 +79,7 @@
 (defn nimbus-data [conf inimbus]
   (let [forced-scheduler (.getForcedScheduler inimbus)]
     {:conf conf
-     :nimbus-host-port-info (NimbusInfo. (.getCanonicalHostName (InetAddress/getLocalHost)) (conf NIMBUS-THRIFT-PORT) false)
+     :nimbus-host-port-info (NimbusInfo/fromConf conf)
      :inimbus inimbus
      :authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
      :submitted-count (atom 0)

http://git-wip-us.apache.org/repos/asf/storm/blob/c82482d8/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
index e31090f..1b96bbf 100644
--- a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
+++ b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
@@ -1,6 +1,11 @@
 package backtype.storm.nimbus;
 
+import backtype.storm.Config;
+
 import java.io.Serializable;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
 
 public class NimbusInfo implements Serializable {
     private static final String DELIM = ":";
@@ -24,6 +29,17 @@ public class NimbusInfo implements Serializable {
         }
     }
 
+    public static NimbusInfo fromConf(Map conf) {
+        try {
+            String host = InetAddress.getLocalHost().getCanonicalHostName();
+            int port = Integer.parseInt(conf.get(Config.NIMBUS_THRIFT_PORT).toString());
+            return new NimbusInfo(host, port, false);
+
+        } catch (UnknownHostException e) {
+            throw new RuntimeException("Something wrong with network/dns config, host cant figure out its name", e);
+        }
+    }
+
     public String toHostPortString() {
         return String.format("%s%s%s",host,DELIM,port);
     }
@@ -41,6 +57,28 @@ public class NimbusInfo implements Serializable {
     }
 
     @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof NimbusInfo)) return false;
+
+        NimbusInfo that = (NimbusInfo) o;
+
+        if (isLeader != that.isLeader) return false;
+        if (port != that.port) return false;
+        if (!host.equals(that.host)) return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = host.hashCode();
+        result = 31 * result + port;
+        result = 31 * result + (isLeader ? 1 : 0);
+        return result;
+    }
+
+    @Override
     public String toString() {
         return "NimbusInfo{" +
                 "host='" + host + '\'' +

http://git-wip-us.apache.org/repos/asf/storm/blob/c82482d8/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 e408c17..1a94049 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -1235,7 +1235,8 @@
     (let [scheme "digest"
           digest "storm:thisisapoorpassword"
           auth-conf {STORM-ZOOKEEPER-AUTH-SCHEME scheme
-                     STORM-ZOOKEEPER-AUTH-PAYLOAD digest}
+                     STORM-ZOOKEEPER-AUTH-PAYLOAD digest
+                     NIMBUS-THRIFT-PORT 6666}
           expected-acls nimbus/NIMBUS-ZK-ACLS
           fake-inimbus (reify INimbus (getForcedScheduler [this] nil))]
       (stubbing [mk-authorization-handler nil


[31/50] [abbrv] storm git commit: Adding stacktrace to the log. Modifying the design doc with nimbus discovery APIs.

Posted by bo...@apache.org.
Adding stacktrace to the log. Modifying the design doc with nimbus discovery APIs.


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

Branch: refs/heads/master
Commit: 8d4e5618efa8a2e0a0ef9d5f199f0a644f31604c
Parents: a8aacca
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Tue Feb 17 14:56:44 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Tue Feb 17 14:56:44 2015 -0800

----------------------------------------------------------------------
 docs/documentation/nimbus-ha-design.md          | 54 +++++++++-----------
 .../jvm/backtype/storm/utils/NimbusClient.java  |  2 +-
 2 files changed, 25 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/8d4e5618/docs/documentation/nimbus-ha-design.md
----------------------------------------------------------------------
diff --git a/docs/documentation/nimbus-ha-design.md b/docs/documentation/nimbus-ha-design.md
index 00fd115..672eece 100644
--- a/docs/documentation/nimbus-ha-design.md
+++ b/docs/documentation/nimbus-ha-design.md
@@ -169,35 +169,31 @@ The following sequence diagram describes the communication between different com
 ![Nimbus HA Topology Submission](images/nimbus_ha_topology_submission.png)
 
 ##Thrift and Rest API 
+In order to avoid workers/supervisors/ui talking to zookeeper for getting master nimbus address we are going to modify the 
+`getClusterInfo` API so it can also return nimbus information. getClusterInfo currently returns `ClusterSummary` instance
+which has a list of `supervisorSummary` and a list of 'topologySummary` instances. We will add a list of `NimbusSummary` 
+to the `ClusterSummary`. See the structures below:
+
+```thrift
+struct ClusterSummary {
+  1: required list<SupervisorSummary> supervisors;
+  3: required list<TopologySummary> topologies;
+  4: required list<NimbusSummary> nimbuses;
+}
 
-This section only exists to track and document how we can reduce the added load on zookeeper for nimbus discovery if the 
-performance numbers indicated any degradation. The actual implementation will not be part of nimbus HA unless we have 
-performance tests to indicate degradation.  
-
-In order to avoid workers/supervisors/ui talking to zookeeper for getting master nimbus address we can add following new API:
-
-```java
-/**
-* Returns list of all nimbus hosts that are either currently in queue or has
-* the leadership lock.
-*/
-List<NimbusInfo> getNimbusHosts();
-
-/**
-* NimbusInfo
-*/
-Class NimbusInfo {
-	String host;
-	short port;
-	boolean isLeader;
+struct NimbusSummary {
+  1: required string host;
+  2: required i32 port;
+  3: required i32 uptime_secs;
+  4: required bool isLeader;
+  5: required string version;
 }
 ```
 
-These apis will be used by StormSubmitter, Nimbus clients,supervisors and ui to discover the current leaders and participating 
+This will be used by StormSubmitter, Nimbus clients,supervisors and ui to discover the current leaders and participating 
 nimbus hosts. Any nimbus host will be able to respond to these requests. The nimbus hosts can read this information once 
-from zookeeper and cache it and keep updating the cache when the watchers are fired to indicate any changes,which should be 
-rare in general case. In addition we should update all the existing thrift and rest apis’s to throw redirect 
-exceptions when a non leader receives a request that only a leader should serve.
+from zookeeper and cache it and keep updating the cache when the watchers are fired to indicate any changes,which should 
+be rare in general case.
 
 ## Configuration
 You can use nimbus ha with default configuration , however the default configuration assumes a single nimbus host so it
@@ -210,14 +206,12 @@ actual code/config and to get the current replication count. An alternative is t
 "org.apache.storm.hdfs.ha.codedistributor.HDFSCodeDistributor" which relies on HDFS but does not add extra load on zookeeper and will 
 make topology submission faster.
 * topology.min.replication.count : Minimum number of nimbus hosts where the code must be replicated before leader nimbus
-can mark the topology as active and create assignments. Default is 1. in case of HDFSCodeDistributor this represents number
-of data nodes instead of nimbus hosts where code must be replicated before activating topology.
+can mark the topology as active and create assignments. Default is 1.
 * topology.max.replication.wait.time.sec: Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
 Once this time is elapsed nimbus will go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. 
 The default is 60 seconds, a value of -1 indicates to wait for ever.
-*nimbus.code.sync.freq.secs: frequency at which the background thread which syncs code for locally missing topologies will run. default is 5 minutes.
+*nimbus.code.sync.freq.secs: frequency at which the background thread on nimbus which syncs code for locally missing topologies will run. default is 5 minutes.
 
 Note: Even though all nimbus hosts have watchers on zookeeper to be notified immediately as soon as a new topology is available for code
-download, due to eventual consistency of zookeeper the callback pretty much never results in code download. In practice we have observed that
-the desired replication is only achieved once the background-thread runs. So you should expect your topology submission time to be somewhere between
-0 to (2 * nimbus.code.sync.freq.secs) for any nimbus.min.replication.count > 0.
\ No newline at end of file
+download, the callback pretty much never results in code download. In practice we have observed that the desired replication is only achieved once the background-thread runs. 
+So you should expect your topology submission time to be somewhere between 0 to (2 * nimbus.code.sync.freq.secs) for any nimbus.min.replication.count > 1.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/8d4e5618/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index e4222e4..39d3895 100644
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -60,7 +60,7 @@ public class NimbusClient extends ThriftClient {
                 throw new RuntimeException("Found nimbuses " + nimbuses + " none of which is elected as leader, please try " +
                         "again after some time.");
             } catch (Exception e) {
-                LOG.warn("Ignoring exception while trying to get leader nimbus info from {}", seed);
+                LOG.warn("Ignoring exception while trying to get leader nimbus info from " + seed, e);
             }
         }
         throw new RuntimeException("Could not find leader nimbus from seed hosts " + seeds +". " +


[12/50] [abbrv] storm git commit: Leveraging NimbusInfo/parse.

Posted by bo...@apache.org.
Leveraging NimbusInfo/parse.


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

Branch: refs/heads/master
Commit: eb242e0f6e1681825d005dbadb19649684025e01
Parents: dc24e44
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 15:52:17 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 15:52:17 2014 -0800

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/zookeeper.clj          | 9 ++++-----
 storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java | 4 ++++
 2 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/eb242e0f/storm-core/src/clj/backtype/storm/zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/zookeeper.clj b/storm-core/src/clj/backtype/storm/zookeeper.clj
index a058098..ae017e6 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -221,10 +221,9 @@
     [id (if (clojure.string/blank? (.getId participant))
           (throw (RuntimeException. "No nimbus leader participant host found, have you started your nimbus hosts?"))
           (.getId participant))
-     server (first (.split id ":"))
-     port (Integer/parseInt (last (.split id ":")))
-     is-leader (.isLeader participant)]
-    (NimbusInfo. server port is-leader)))
+     nimbus-info (NimbusInfo/parse id)]
+    (.setLeader nimbus-info (.isLeader participant))
+    nimbus-info))
 
 (defn leader-latch-listener-impl
   "Leader latch listener that will be invoked when we either gain or lose leadership"
@@ -254,7 +253,7 @@
   (let [servers (conf STORM-ZOOKEEPER-SERVERS)
         zk (mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf conf)
         leader-lock-path (str (conf STORM-ZOOKEEPER-ROOT) "/leader-lock")
-        id (str (.getCanonicalHostName (InetAddress/getLocalHost)) ":" (conf NIMBUS-THRIFT-PORT))
+        id (.toHostPortString (NimbusInfo/fromConf conf))
         leader-latch (atom (LeaderLatch. zk leader-lock-path id))
         leader-latch-listener (atom (leader-latch-listener-impl conf zk @leader-latch))
         ]

http://git-wip-us.apache.org/repos/asf/storm/blob/eb242e0f/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
index 1b96bbf..bf56e2d 100644
--- a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
+++ b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
@@ -56,6 +56,10 @@ public class NimbusInfo implements Serializable {
         return host;
     }
 
+    public void setLeader(boolean isLeader) {
+        this.isLeader = isLeader;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;


[10/50] [abbrv] storm git commit: STORM-166: modifed some config key names to match the convention, added the documentation for configuration.

Posted by bo...@apache.org.
STORM-166: modifed some config key names to match the convention, added the documentation for configuration.


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

Branch: refs/heads/master
Commit: 27d6b4ca5ab4d4f9d7753d66b2a28c982b47c7e7
Parents: a75c72c
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 14:31:49 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 14:31:49 2014 -0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |  4 ++--
 docs/documentation/nimbus-ha-design.md          | 23 ++++++++++++++++++++
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  4 ++--
 storm-core/src/jvm/backtype/storm/Config.java   | 13 +++++------
 4 files changed, 33 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/27d6b4ca/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 07230dc..e189966 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -70,8 +70,8 @@ nimbus.task.launch.secs: 120
 nimbus.reassign: true
 nimbus.file.copy.expiration.secs: 600
 nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator"
-min.replication.count: 0
-max.replication.wait.time.sec: 0
+nimbus.min.replication.count: 0
+nimbus.max.replication.wait.time.sec: 0
 nimbus.credential.renewers.freq.secs: 600
 
 ### ui.* configs are for the master

http://git-wip-us.apache.org/repos/asf/storm/blob/27d6b4ca/docs/documentation/nimbus-ha-design.md
----------------------------------------------------------------------
diff --git a/docs/documentation/nimbus-ha-design.md b/docs/documentation/nimbus-ha-design.md
index cb6332f..9c38851 100644
--- a/docs/documentation/nimbus-ha-design.md
+++ b/docs/documentation/nimbus-ha-design.md
@@ -198,3 +198,26 @@ nimbus hosts. Any nimbus host will be able to respond to these requests. The nim
 from zookeeper and cache it and keep updating the cache when the watchers are fired to indicate any changes,which should be 
 rare in general case. In addition we should update all the existing thrift and rest apis’s to throw redirect 
 exceptions when a non leader receives a request that only a leader should serve.
+
+## Configuration
+You can use nimbus ha with default configuration , however the default configuration assumes a single nimbus host so it
+trades off replication for lower topology submission latency. Depending on your use case you can adjust following configurations:
+* storm.codedistributor.class : This is a string representing fully qualified class name of a class that implements
+backtype.storm.codedistributor.ICodeDistributor. The default is set to "backtype.storm.codedistributor.LocalFileSystemCodeDistributor".
+This class leverages local file system to store both meta files and code/configs. This class adds extra load on zookeeper as even after
+downloading the code-distrbutor meta file it contacts zookeeper in order to figure out hosts from where it can download
+actual code/config and to get the current replication count. An alternative is to use 
+"org.apache.storm.hdfs.ha.codedistributor.HDFSCodeDistributor" which relies on HDFS but does not add extra load on zookeeper and will 
+make topology submission faster.
+* nimbus.min.replication.count : Minimum number of nimbus hosts where the code must be replicated before leader nimbus
+can mark the topology as active and create assignments. Default is 0. in case of HDFSCodeDistributor this represents number
+of data nodes insted of nimbus hosts where code must be replicated before activating topology.
+* nimbus.max.replication.wait.time.sec: Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
+Once this time is elapsed nimbus will go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. 
+The default is 0 seconds, a value of -1 indicates to wait for ever.
+*nimbus.code.sync.freq.secs: frequency at which the background thread which syncs code for locally missing topologies will run. default is 5 minutes.
+
+Note: Even though all nimbus hosts have watchers on zookeeper to be notified immediately as soon as a new topology is available for code
+download, due to eventual consistency of zookeeper the callback pretty much never results in code download. In practice we have observed that
+the desired replication is only achieved once the background-thread runs. So you should expect your topology submission time to be somewhere between
+0 to (2 * nimbus.code.sync.freq.secs) for any nimbus.min.replication.count > 0.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/27d6b4ca/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 b638e52..0281e73 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -340,8 +340,8 @@
    ))
 
 (defn- wait-for-desired-code-replication [nimbus conf storm-id]
-  (let [min-replication-count (conf MIN-REPLICATION-COUNT)
-        max-replication-wait-time (conf MAX-REPLICATION-WAIT-TIME-SEC)
+  (let [min-replication-count (conf NIMBUS-MIN-REPLICATION-COUNT)
+        max-replication-wait-time (conf NIMBUS-MAX-REPLICATION-WAIT-TIME-SEC)
         total-wait-time (atom 0)
         current-replication-count (atom (if (:bt-tracker nimbus) (.getReplicationCount (:bt-tracker nimbus) storm-id) 0))]
   (if (:bt-tracker nimbus)

http://git-wip-us.apache.org/repos/asf/storm/blob/27d6b4ca/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 ead144f..9746565 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1288,18 +1288,17 @@ public class Config extends HashMap<String, Object> {
      * is allowed to perform topology activation tasks like setting up heartbeats/assignments
      * and marking the topology as active. default is 0.
      */
-    public static final String MIN_REPLICATION_COUNT = "min.replication.count";
-    public static final Object MIN_REPLICATION_COUNT_SCHEMA = Number.class;
+    public static final String NIMBUS_MIN_REPLICATION_COUNT = "nimbus.min.replication.count";
+    public static final Object NIMBUS_MIN_REPLICATION_COUNT_SCHEMA = Number.class;
 
     /**
-     * Maximum wait time for the nimbus host replication to achieve the min.replication.count.
+     * Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
      * Once this time is elapsed nimbus will go ahead and perform topology activation tasks even
-     * if required min.replication.count is not achieved. The default is 0 seconds, a value of
+     * if required nimbus.min.replication.count is not achieved. The default is 0 seconds, a value of
      * -1 indicates to wait for ever.
      */
-    public static final String MAX_REPLICATION_WAIT_TIME_SEC = "max.replication.wait.time.sec";
-    public static final Object MAX_REPLICATION_WAIT_TIME_SEC_SCHEMA = Number.class;
-
+    public static final String NIMBUS_MAX_REPLICATION_WAIT_TIME_SEC = "nimbus.max.replication.wait.time.sec";
+    public static final Object NIMBUS_MAX_REPLICATION_WAIT_TIME_SEC_SCHEMA = Number.class;
 
     /**
      * How often nimbus should wake the cleanup thread to clean the inbox.


[49/50] [abbrv] storm git commit: Merge branch 'STORM-166' of https://github.com/Parth-Brahmbhatt/incubator-storm into STORM-166

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

STORM-166: Nimbus HA


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

Branch: refs/heads/master
Commit: 6b3cb7595e04e175305380ba665428f5b9c245b3
Parents: cc27608 51fdc1a
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Mon Aug 24 08:43:16 2015 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Mon Aug 24 08:43:16 2015 -0500

----------------------------------------------------------------------
 STORM-UI-REST-API.md                            |  41 +-
 conf/defaults.yaml                              |   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 +++++
 .../ha/codedistributor/HDFSCodeDistributor.java | 101 +++
 pom.xml                                         |  16 +
 storm-core/pom.xml                              |  16 +
 storm-core/src/clj/backtype/storm/cluster.clj   |  94 ++-
 .../backtype/storm/command/shell_submission.clj |   9 +-
 storm-core/src/clj/backtype/storm/config.clj    |  15 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    | 252 ++++--
 .../clj/backtype/storm/daemon/supervisor.clj    |  53 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |  23 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  89 ++-
 storm-core/src/clj/backtype/storm/zookeeper.clj | 108 ++-
 storm-core/src/jvm/backtype/storm/Config.java   |  47 +-
 .../storm/codedistributor/ICodeDistributor.java |  56 ++
 .../LocalFileSystemCodeDistributor.java         | 106 +++
 .../storm/generated/ClusterSummary.java         | 292 ++++---
 .../backtype/storm/generated/NimbusSummary.java | 796 +++++++++++++++++++
 .../backtype/storm/generated/TopologyInfo.java  | 221 +++--
 .../storm/generated/TopologySummary.java        | 107 ++-
 .../backtype/storm/nimbus/ILeaderElector.java   |  60 ++
 .../jvm/backtype/storm/nimbus/NimbusInfo.java   |  93 +++
 .../jvm/backtype/storm/utils/NimbusClient.java  |  78 +-
 .../src/jvm/backtype/storm/utils/Utils.java     |   9 +
 storm-core/src/py/storm/ttypes.py               | 613 ++++++++------
 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       |   6 +
 .../test/clj/backtype/storm/cluster_test.clj    |  23 +-
 .../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 -
 39 files changed, 3224 insertions(+), 659 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/storm/blob/6b3cb759/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------


[19/50] [abbrv] storm git commit: STORM-166 making min.replication.count and max.wait.time topology configuration.

Posted by bo...@apache.org.
STORM-166 making min.replication.count and max.wait.time topology configuration.


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

Branch: refs/heads/master
Commit: 01eee7aa519c9d63afc38f2f61a53f48913c0a28
Parents: 27a381a
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Mon Feb 2 23:37:17 2015 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Mon Feb 2 23:37:17 2015 -0800

----------------------------------------------------------------------
 conf/defaults.yaml                                  |  4 ++--
 docs/documentation/nimbus-ha-design.md              | 10 +++++-----
 storm-core/src/clj/backtype/storm/daemon/nimbus.clj |  6 +++---
 storm-core/src/jvm/backtype/storm/Config.java       |  8 ++++----
 4 files changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/01eee7aa/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 7fea11c..fbea948 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -65,8 +65,8 @@ nimbus.task.launch.secs: 120
 nimbus.reassign: true
 nimbus.file.copy.expiration.secs: 600
 nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator"
-nimbus.min.replication.count: 0
-nimbus.max.replication.wait.time.sec: 0
+topology.min.replication.count: 1
+topology.max.replication.wait.time.sec: 60
 nimbus.credential.renewers.freq.secs: 600
 
 ### ui.* configs are for the master

http://git-wip-us.apache.org/repos/asf/storm/blob/01eee7aa/docs/documentation/nimbus-ha-design.md
----------------------------------------------------------------------
diff --git a/docs/documentation/nimbus-ha-design.md b/docs/documentation/nimbus-ha-design.md
index 9c38851..00fd115 100644
--- a/docs/documentation/nimbus-ha-design.md
+++ b/docs/documentation/nimbus-ha-design.md
@@ -209,12 +209,12 @@ downloading the code-distrbutor meta file it contacts zookeeper in order to figu
 actual code/config and to get the current replication count. An alternative is to use 
 "org.apache.storm.hdfs.ha.codedistributor.HDFSCodeDistributor" which relies on HDFS but does not add extra load on zookeeper and will 
 make topology submission faster.
-* nimbus.min.replication.count : Minimum number of nimbus hosts where the code must be replicated before leader nimbus
-can mark the topology as active and create assignments. Default is 0. in case of HDFSCodeDistributor this represents number
-of data nodes insted of nimbus hosts where code must be replicated before activating topology.
-* nimbus.max.replication.wait.time.sec: Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
+* topology.min.replication.count : Minimum number of nimbus hosts where the code must be replicated before leader nimbus
+can mark the topology as active and create assignments. Default is 1. in case of HDFSCodeDistributor this represents number
+of data nodes instead of nimbus hosts where code must be replicated before activating topology.
+* topology.max.replication.wait.time.sec: Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
 Once this time is elapsed nimbus will go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. 
-The default is 0 seconds, a value of -1 indicates to wait for ever.
+The default is 60 seconds, a value of -1 indicates to wait for ever.
 *nimbus.code.sync.freq.secs: frequency at which the background thread which syncs code for locally missing topologies will run. default is 5 minutes.
 
 Note: Even though all nimbus hosts have watchers on zookeeper to be notified immediately as soon as a new topology is available for code

http://git-wip-us.apache.org/repos/asf/storm/blob/01eee7aa/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 e07e6d1..2528825 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -340,8 +340,8 @@
    ))
 
 (defn- wait-for-desired-code-replication [nimbus conf storm-id]
-  (let [min-replication-count (conf NIMBUS-MIN-REPLICATION-COUNT)
-        max-replication-wait-time (conf NIMBUS-MAX-REPLICATION-WAIT-TIME-SEC)
+  (let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
+        max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
         total-wait-time (atom 0)
         current-replication-count (atom (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 0))]
   (if (:code-distributor nimbus)
@@ -1129,7 +1129,7 @@
               (.set-credentials! storm-cluster-state storm-id credentials storm-conf)
               (setup-storm-code nimbus conf storm-id uploadedJarLocation storm-conf topology)
               (.setup-code-distributor! storm-cluster-state storm-id (:nimbus-host-port-info nimbus))
-              (wait-for-desired-code-replication nimbus conf storm-id)
+              (wait-for-desired-code-replication nimbus total-storm-conf storm-id)
               (.setup-heartbeats! storm-cluster-state storm-id)
               (let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive
                                               TopologyInitialStatus/ACTIVE :active}]

http://git-wip-us.apache.org/repos/asf/storm/blob/01eee7aa/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 df330a0..c0668ee 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -1252,8 +1252,8 @@ public class Config extends HashMap<String, Object> {
      * is allowed to perform topology activation tasks like setting up heartbeats/assignments
      * and marking the topology as active. default is 0.
      */
-    public static final String NIMBUS_MIN_REPLICATION_COUNT = "nimbus.min.replication.count";
-    public static final Object NIMBUS_MIN_REPLICATION_COUNT_SCHEMA = Number.class;
+    public static final String TOPOLOGY_MIN_REPLICATION_COUNT = "topology.min.replication.count";
+    public static final Object TOPOLOGY_MIN_REPLICATION_COUNT_SCHEMA = Number.class;
 
     /**
      * Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
@@ -1261,8 +1261,8 @@ public class Config extends HashMap<String, Object> {
      * if required nimbus.min.replication.count is not achieved. The default is 0 seconds, a value of
      * -1 indicates to wait for ever.
      */
-    public static final String NIMBUS_MAX_REPLICATION_WAIT_TIME_SEC = "nimbus.max.replication.wait.time.sec";
-    public static final Object NIMBUS_MAX_REPLICATION_WAIT_TIME_SEC_SCHEMA = Number.class;
+    public static final String TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC = "nimbus.max.replication.wait.time.sec";
+    public static final Object TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC_SCHEMA = Number.class;
 
     /**
      * How often nimbus's background thread to sync code for missing topologies should run.


[03/50] [abbrv] storm git commit: STORM-166 Nimbus HA design documentation and sequence diagram.

Posted by bo...@apache.org.
STORM-166 Nimbus HA design documentation and sequence diagram.


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

Branch: refs/heads/master
Commit: 07b69b7b86828b4213873405323c0e9be03934f1
Parents: dd991e5
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Fri Dec 19 12:52:30 2014 -0800
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Fri Dec 19 12:52:30 2014 -0800

----------------------------------------------------------------------
 .../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          | 201 +++++++++++++++++++
 3 files changed, 201 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/07b69b7b/docs/documentation/images/nimbus_ha_leader_election_and_failover.png
----------------------------------------------------------------------
diff --git a/docs/documentation/images/nimbus_ha_leader_election_and_failover.png b/docs/documentation/images/nimbus_ha_leader_election_and_failover.png
new file mode 100644
index 0000000..60cc1b7
Binary files /dev/null and b/docs/documentation/images/nimbus_ha_leader_election_and_failover.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/07b69b7b/docs/documentation/images/nimbus_ha_topology_submission.png
----------------------------------------------------------------------
diff --git a/docs/documentation/images/nimbus_ha_topology_submission.png b/docs/documentation/images/nimbus_ha_topology_submission.png
new file mode 100644
index 0000000..7707e5a
Binary files /dev/null and b/docs/documentation/images/nimbus_ha_topology_submission.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/07b69b7b/docs/documentation/nimbus-ha-design.md
----------------------------------------------------------------------
diff --git a/docs/documentation/nimbus-ha-design.md b/docs/documentation/nimbus-ha-design.md
new file mode 100644
index 0000000..51b15f8
--- /dev/null
+++ b/docs/documentation/nimbus-ha-design.md
@@ -0,0 +1,201 @@
+#Highly Available Nimbus design proposal
+##Problem Statement:
+Currently the storm master aka nimbus, is a process that runs on a single machine under supervision. In most cases the 
+nimbus failure is transient and it is restarted by the supervisor. However sometimes when disks fail and networks 
+partitions occur, nimbus goes down. Under these circumstances the topologies run normally but no new topologies can be 
+submitted, no existing topologies can be killed/deactivated/activated and if a supervisor node fails then the 
+reassignments are not performed resulting in performance degradation or topology failures. With this project we intend 
+to resolve this problem by running nimbus in a primary backup mode to guarantee that even if a nimbus server fails one 
+of the backups will take over.
+##Requirements:
+* Increase overall availability of nimbus.
+* Allow nimbus hosts to leave and join the cluster at will any time. A newly joined host should auto catch up and join 
+the list of potential leaders automatically. 
+* No topology resubmissions required in case of nimbus fail overs.
+* No active topology should ever be lost. 
+##Components:
+Following are different components to achieve the above goals.
+###Leader Election:
+The nimbus server will use the following interface:
+
+```java
+public interface ILeaderElector {
+    /**
+     * queue up for leadership lock. The call returns immediately and the caller                     
+     * must check isLeader() to perform any leadership action.
+     */
+    void addToLeaderLockQueue();
+
+    /**
+     * Removes the caller from the leader lock queue. If the caller is leader
+     * also releases the lock.
+     */
+    void removeFromLeaderLockQueue();
+
+    /**
+     *
+     * @return true if the caller currently has the leader lock.
+     */
+    boolean isLeader();
+
+    /**
+     *
+     * @return the current leader's address , throws exception if noone has has    lock.
+     */
+    InetSocketAddress getLeaderAddress();
+
+    /**
+     * 
+     * @return list of current nimbus addresses, includes leader.
+     */
+    List<InetSocketAddress> getAllNimbusAddresses();
+}
+```
+On startup nimbus will check if it has code for all active topologies available locally. Once it gets to this state it 
+will call addToLeaderLockQueue() function. When a nimbus is notified to become a leader it will check if it has all the
+code locally before assuming the leadership role. If any active topology code is missing, the node will not accept the 
+leadership role instead it will release the lock and wait till it has all the code before requeueing for leader lock. 
+
+The first implementation will be Zookeeper based. If the zookeeper connection is lost/resetted resulting in loss of lock
+or the spot in queue the implementation will take care of updating the state such that isLeader() will reflect the 
+current status.The leader like actions must finish in less than minimumOf(connectionTimeout, SessionTimeout) to ensure
+the lock was held by nimbus for the entire duration of the action (Not sure if we want to just state this expectation 
+and ensure that zk configurations are set high enough which will result in higher failover time or we actually want to 
+create some sort of rollback mechanism for all actions, the second option needs a lot of code). If a nimbus that is not 
+leader receives a request that only a leader can perform it will throw a RunTimeException.
+
+Following steps describes a nimbus failover scenario:
+* Let’s say we have 4 topologies running with 3 nimbus nodes and code-replication-factor = 2. We assume that the 
+invariant “The leader nimbus has code for all topologies locally” holds true at the beginning. nonleader-1 has code for 
+the first 2 topologies and nonLeader-2 has code for the other 2 topologies.
+* Leader nimbus dies, hard disk failure so no recovery possible.
+* nonLeader-1 gets a zookeeper notification to indicate it is now the new leader. before accepting the leadership it 
+checks if it has code available for all 4 topologies(these are topologies under /storm/storms/). It realizes it only has
+code for 2 topologies so it relinquishes the lock and looks under  /storm/code-distributor/topologyId to find out from 
+where can it download the code/metafile for the missing topologies. it finds entries for the leader nimbus and 
+nonleader-2. It will try downloading from both as part of its retry mechanism.
+* nonLeader-2’s code sync thread also realizes that it is missing code for 2 topologies and follows the same process 
+described in step-3 to download code for missing topologies. 
+* eventually at least one of the nimbuses will have all the code locally and will accept leadership.
+This sequence diagram describes how leader election and failover would work with multiple components.
+
+![Nimbus Fail Over](images/nimbus_ha_leader_election_and_failover.png)
+
+###Nimbus state store:
+
+Currently the nimbus stores 2 kind of data
+* Meta information like supervisor info, assignment info which is stored in zookeeper
+* Actual topology configs and jars that is stored on nimbus host’s local disk.
+
+To achieve fail over from primary to backup servers nimbus state/data needs to be replicated across all nimbus hosts or 
+needs to be stored in a distributed storage. Replicating the data correctly involves state management, consistency checks
+and it is hard to test for correctness.However many storm users do not want to take extra dependency on another replicated
+storage system like HDFS and still need high availability.Eventually, we want to move to the bittorrent protocol for code 
+distribution given the size of the jars and to achieve better scaling when the total number of supervisors is very high. 
+The current file system based model for code distribution works fine with systems that have file system like structure
+but it fails to support a non file system based approach like bit torrent. To support bit torrent we can go with the 
+following interface instead of the storage interface described above. The interface described below can still be used with
+HDFS,S3 and local file system, so this is a more extensible interface. 
+```java
+/**
+ * Interface responsible to distribute code in the cluster.
+ */
+public interface ICodeDistributor {
+    /**
+     * Prepare this code distributor.
+     * @param conf
+     */
+    void prepare(Map conf);
+
+    /**
+     * This API will perform the actual upload of the code to the distributed implementation.
+     * The API should return a Meta file which should have enough information for downloader 
+     * so it can download the code e.g. for bittorrent it will be a torrent file, in case of something         
+     * like HDFS or s3  it might have the actual directory or paths for files to be downloaded.
+     * @param dirPath local directory where all the code to be distributed exists.
+     * @param topologyId the topologyId for which the meta file needs to be created.
+     * @return metaFile
+     */
+    File upload(Path dirPath, String topologyId);
+
+    /**
+     * Given the topologyId and metafile, download the actual code and return the downloaded file's list.
+     * @param topologyid
+     * @param metafile 
+     * @param destDirPath the folder where all the files will be downloaded.
+     * @return
+     */
+    List<File> download(Path destDirPath, String topologyid, File metafile);
+
+    /**
+      * Given the topologyId, returns number of hosts where the code has been replicated.
+      */
+    int getReplicationCount(String topologyId);
+    
+   /**
+     * Performs the cleanup.
+     * @param topologyid
+     */
+    void cleanup(String topologyid);
+
+    /**
+     * Close this distributor.
+     * @param conf
+     */
+    void close(Map conf);
+}
+```
+To support replication we will allow the user to define a code replication factor which would reflect number of nimbus 
+hosts to which the code must be replicated before starting the topology. With replication comes the issue of consistency. 
+We will treat zookeeper’s list of active topologies as our authority for topologies for which the code must exist on a 
+nimbus host. Any nimbus host that does not have all the code for all the topologies which are marked as active in zookeeper 
+will relinquish it’s lock so some other nimbus host could become leader. A background thread on all nimbus host will 
+continuously try to sync code from other hosts where the code was successfully replicated so eventually at least one nimbus 
+will accept leadership as long as at least one seed hosts exists for each active topology. 
+		
+Following steps describe code replication amongst nimbus hosts for a topology:
+* When client uploads jar, nothing changes.
+* When client submits a topology, leader nimbus calls code distributor’s upload function which will create a metafile stored 
+locally on leader nimbus. Leader nimbus will write new entries under /storm/code-distributor/topologyId to notify all 
+nonleader nimbuses that they should download this new code.
+* We wait on the leader nimbus to ensure at least N non leader nimbus has the code replicated, with a user configurable timeout.
+* When a non leader nimbus receives the notification about new code, it downloads the meta file from leader nimbus and then
+downloads the real code by calling code distributor’s download function with metafile as input.
+* Once non leader finishes downloading code, it will write an entry under /storm/code-distributor/topologyId to indicate 
+it is one of the possible places to download the code/metafile in case the leader nimbus dies.
+* leader nimbus goes ahead and does all the usual things it does as part of submit topologies.
+
+The following sequence diagram describes the communication between different components involved in code distribution.
+
+![Nimbus HA Topology Submission](images/nimbus_ha_topology_submission.png)
+
+##Thrift and Rest API 
+
+This section only exists to track and document how we can reduce the added load on zookeeper for nimbus discovery if the 
+performance numbers indicated any degradation. The actual implementation will not be part of nimbus HA unless we have 
+performance tests to indicate degradation.  
+
+In order to avoid workers/supervisors/ui talking to zookeeper for getting master nimbus address we can add following new API:
+
+```java
+/**
+* Returns list of all nimbus hosts that are either currently in queue or has
+* the leadership lock.
+*/
+List<NimbusInfo> getNimbusHosts();
+
+/**
+* NimbusInfo
+*/
+Class NimbusInfo {
+	String host;
+	short port;
+	boolean isLeader;
+}
+```
+
+These apis will be used by StormSubmitter, Nimbus clients,supervisors and ui to discover the current leaders and participating 
+nimbus hosts. Any nimbus host will be able to respond to these requests. The nimbus hosts can read this information once 
+from zookeeper and cache it and keep updating the cache when the watchers are fired to indicate any changes,which should be 
+rare in general case. In addition we should update all the existing thrift and rest apis’s to throw redirect 
+exceptions when a non leader receives a request that only a leader should serve.