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 2014/06/12 21:54:58 UTC

[01/13] move towards idiomatic Clojure style

Repository: incubator-storm
Updated Branches:
  refs/heads/master 2a60e9911 -> 0826b9332


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 6714fcd..6e78cca 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -13,6 +13,7 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.util
   (:import [java.net InetAddress])
   (:import [java.util Map Map$Entry List ArrayList Collection Iterator HashMap])
@@ -36,10 +37,10 @@
   (:require [clojure.java.io :as io])
   (:use [clojure walk])
   (:use [backtype.storm log])
-  )
+  (:refer-clojure :exclude [some?]))
 
 (defn wrap-in-runtime
-  "Wraps an exception in a RuntimeException if needed" 
+  "Wraps an exception in a RuntimeException if needed"
   [^Exception e]
   (if (instance? RuntimeException e)
     e
@@ -59,64 +60,63 @@
   any) and similar metadata. The metadata of the alias is its initial
   metadata (as provided by def) merged into the metadata of the original."
   ([name orig]
-     `(do
-        (alter-meta!
-         (if (.hasRoot (var ~orig))
-           (def ~name (.getRawRoot (var ~orig)))
-           (def ~name))
-         ;; When copying metadata, disregard {:macro false}.
-         ;; Workaround for http://www.assembla.com/spaces/clojure/tickets/273
-         #(conj (dissoc % :macro)
-                (apply dissoc (meta (var ~orig)) (remove #{:macro} (keys %)))))
-        (var ~name)))
+   `(do
+      (alter-meta!
+        (if (.hasRoot (var ~orig))
+          (def ~name (.getRawRoot (var ~orig)))
+          (def ~name))
+        ;; When copying metadata, disregard {:macro false}.
+        ;; Workaround for http://www.assembla.com/spaces/clojure/tickets/273
+        #(conj (dissoc % :macro)
+               (apply dissoc (meta (var ~orig)) (remove #{:macro} (keys %)))))
+      (var ~name)))
   ([name orig doc]
-     (list `defalias (with-meta name (assoc (meta name) :doc doc)) orig)))
+   (list `defalias (with-meta name (assoc (meta name) :doc doc)) orig)))
 
 ;; name-with-attributes by Konrad Hinsen:
 (defn name-with-attributes
   "To be used in macro definitions.
-   Handles optional docstrings and attribute maps for a name to be defined
-   in a list of macro arguments. If the first macro argument is a string,
-   it is added as a docstring to name and removed from the macro argument
-   list. If afterwards the first macro argument is a map, its entries are
-   added to the name's metadata map and the map is removed from the
-   macro argument list. The return value is a vector containing the name
-   with its extended metadata map and the list of unprocessed macro
-   arguments."
+  Handles optional docstrings and attribute maps for a name to be defined
+  in a list of macro arguments. If the first macro argument is a string,
+  it is added as a docstring to name and removed from the macro argument
+  list. If afterwards the first macro argument is a map, its entries are
+  added to the name's metadata map and the map is removed from the
+  macro argument list. The return value is a vector containing the name
+  with its extended metadata map and the list of unprocessed macro
+  arguments."
   [name macro-args]
   (let [[docstring macro-args] (if (string? (first macro-args))
                                  [(first macro-args) (next macro-args)]
                                  [nil macro-args])
-    [attr macro-args]          (if (map? (first macro-args))
-                                 [(first macro-args) (next macro-args)]
-                                 [{} macro-args])
-    attr                       (if docstring
-                                 (assoc attr :doc docstring)
-                                 attr)
-    attr                       (if (meta name)
-                                 (conj (meta name) attr)
-                                 attr)]
+        [attr macro-args] (if (map? (first macro-args))
+                            [(first macro-args) (next macro-args)]
+                            [{} macro-args])
+        attr (if docstring
+               (assoc attr :doc docstring)
+               attr)
+        attr (if (meta name)
+               (conj (meta name) attr)
+               attr)]
     [(with-meta name attr) macro-args]))
 
 (defmacro defnk
- "Define a function accepting keyword arguments. Symbols up to the first
- keyword in the parameter list are taken as positional arguments.  Then
- an alternating sequence of keywords and defaults values is expected. The
- values of the keyword arguments are available in the function body by
- virtue of the symbol corresponding to the keyword (cf. :keys destructuring).
- defnk accepts an optional docstring as well as an optional metadata map."
- [fn-name & fn-tail]
- (let [[fn-name [args & body]] (name-with-attributes fn-name fn-tail)
-       [pos kw-vals]           (split-with symbol? args)
-       syms                    (map #(-> % name symbol) (take-nth 2 kw-vals))
-       values                  (take-nth 2 (rest kw-vals))
-       sym-vals                (apply hash-map (interleave syms values))
-       de-map                  {:keys (vec syms)
-                                :or   sym-vals}]
-   `(defn ~fn-name
-      [~@pos & options#]
-      (let [~de-map (apply hash-map options#)]
-        ~@body))))
+  "Define a function accepting keyword arguments. Symbols up to the first
+  keyword in the parameter list are taken as positional arguments.  Then
+  an alternating sequence of keywords and defaults values is expected. The
+  values of the keyword arguments are available in the function body by
+  virtue of the symbol corresponding to the keyword (cf. :keys destructuring).
+  defnk accepts an optional docstring as well as an optional metadata map."
+  [fn-name & fn-tail]
+  (let [[fn-name [args & body]] (name-with-attributes fn-name fn-tail)
+        [pos kw-vals] (split-with symbol? args)
+        syms (map #(-> % name symbol) (take-nth 2 kw-vals))
+        values (take-nth 2 (rest kw-vals))
+        sym-vals (apply hash-map (interleave syms values))
+        de-map {:keys (vec syms) :or sym-vals}]
+    `(defn ~fn-name
+       [~@pos & options#]
+       (let [~de-map (apply hash-map options#)]
+         ~@body))))
 
 (defn find-first
   "Returns the first item of coll for which (pred item) returns logical true.
@@ -149,200 +149,203 @@
 
 (defn positions
   "Returns a lazy sequence containing the positions at which pred
-   is true for items in coll."
+  is true for items in coll."
   [pred coll]
   (for [[idx elt] (indexed coll) :when (pred elt)] idx))
 
-(defn exception-cause? [klass ^Throwable t]
+(defn exception-cause?
+  [klass ^Throwable t]
   (->> (iterate #(.getCause ^Throwable %) t)
        (take-while identity)
        (some (partial instance? klass))
        boolean))
 
-(defmacro thrown-cause? [klass & body]
+(defmacro thrown-cause?
+  [klass & body]
   `(try
-    ~@body
-    false
-    (catch Throwable t#
-      (exception-cause? ~klass t#))))
+     ~@body
+     false
+     (catch Throwable t#
+       (exception-cause? ~klass t#))))
 
-(defmacro thrown-cause-with-msg? [klass re & body]
+(defmacro thrown-cause-with-msg?
+  [klass re & body]
   `(try
-    ~@body
-    false
-    (catch Throwable t#
-      (and (re-matches ~re (.getMessage t#))
-        (exception-cause? ~klass t#)))))
+     ~@body
+     false
+     (catch Throwable t#
+       (and (re-matches ~re (.getMessage t#))
+            (exception-cause? ~klass t#)))))
 
-(defmacro forcat [[args aseq] & body]
+(defmacro forcat
+  [[args aseq] & body]
   `(mapcat (fn [~args]
              ~@body)
            ~aseq))
 
-(defmacro try-cause [& body]
+(defmacro try-cause
+  [& body]
   (let [checker (fn [form]
                   (or (not (sequential? form))
                       (not= 'catch (first form))))
         [code guards] (split-with checker body)
         error-local (gensym "t")
         guards (forcat [[_ klass local & guard-body] guards]
-                 `((exception-cause? ~klass ~error-local)
-                   (let [~local ~error-local]
-                     ~@guard-body
-                     )))
-        ]
+                       `((exception-cause? ~klass ~error-local)
+                         (let [~local ~error-local]
+                           ~@guard-body
+                           )))]
     `(try ~@code
-          (catch Throwable ~error-local
-            (cond ~@guards
-                  true (throw ~error-local)
-                  )))))
+       (catch Throwable ~error-local
+         (cond ~@guards
+               true (throw ~error-local)
+               )))))
 
-(defn local-hostname []
+(defn local-hostname
+  []
   (.getCanonicalHostName (InetAddress/getLocalHost)))
 
 (letfn [(try-port [port]
-          (with-open [socket (java.net.ServerSocket. port)]
-            (.getLocalPort socket)))]
+                  (with-open [socket (java.net.ServerSocket. port)]
+                    (.getLocalPort socket)))]
   (defn available-port
     ([] (try-port 0))
     ([preferred]
-      (try
-        (try-port preferred)
-        (catch java.io.IOException e
-          (available-port))))))
+     (try
+       (try-port preferred)
+       (catch java.io.IOException e
+         (available-port))))))
 
 (defn uuid []
   (str (UUID/randomUUID)))
 
-(defn current-time-secs []
+(defn current-time-secs
+  []
   (Time/currentTimeSecs))
 
-(defn current-time-millis []
+(defn current-time-millis
+  []
   (Time/currentTimeMillis))
 
-(defn secs-to-millis-long [secs]
+(defn secs-to-millis-long
+  [secs]
   (long (* (long 1000) secs)))
 
-(defn clojurify-structure [s]
+(defn clojurify-structure
+  [s]
   (prewalk (fn [x]
-              (cond (instance? Map x) (into {} x)
-                    (instance? List x) (vec x)
-                    true x))
+             (cond (instance? Map x) (into {} x)
+                   (instance? List x) (vec x)
+                   true x))
            s))
 
-(defmacro with-file-lock [path & body]
+(defmacro with-file-lock
+  [path & body]
   `(let [f# (File. ~path)
          _# (.createNewFile f#)
          rf# (RandomAccessFile. f# "rw")
          lock# (.. rf# (getChannel) (lock))]
-      (try
-        ~@body
-        (finally
-          (.release lock#)
-          (.close rf#))
-        )))
-
-(defn tokenize-path [^String path]
+     (try
+       ~@body
+       (finally
+         (.release lock#)
+         (.close rf#)))))
+
+(defn tokenize-path
+  [^String path]
   (let [toks (.split path "/")]
-    (vec (filter (complement empty?) toks))
-    ))
+    (vec (filter (complement empty?) toks))))
 
-(defn assoc-conj [m k v]
+(defn assoc-conj
+  [m k v]
   (merge-with concat m {k [v]}))
 
 ;; returns [ones in first set not in second, ones in second set not in first]
-(defn set-delta [old curr]
+(defn set-delta
+  [old curr]
   (let [s1 (set old)
         s2 (set curr)]
-    [(set/difference s1 s2) (set/difference s2 s1)]
-    ))
+    [(set/difference s1 s2) (set/difference s2 s1)]))
 
-(defn parent-path [path]
+(defn parent-path
+  [path]
   (let [toks (tokenize-path path)]
-    (str "/" (str/join "/" (butlast toks)))
-    ))
+    (str "/" (str/join "/" (butlast toks)))))
 
-(defn toks->path [toks]
-  (str "/" (str/join "/" toks))
-  )
+(defn toks->path
+  [toks]
+  (str "/" (str/join "/" toks)))
 
-(defn normalize-path [^String path]
+(defn normalize-path
+  [^String path]
   (toks->path (tokenize-path path)))
 
-(defn map-val [afn amap]
+(defn map-val
+  [afn amap]
   (into {}
-    (for [[k v] amap]
-      [k (afn v)]
-      )))
+        (for [[k v] amap]
+          [k (afn v)])))
 
-(defn filter-val [afn amap]
-  (into {}
-    (filter
-      (fn [[k v]]
-        (afn v))
-       amap
-       )))
+(defn filter-val
+  [afn amap]
+  (into {} (filter (fn [[k v]] (afn v)) amap)))
 
-(defn filter-key [afn amap]
-  (into {}
-    (filter
-      (fn [[k v]]
-        (afn k))
-       amap
-       )))
+(defn filter-key
+  [afn amap]
+  (into {} (filter (fn [[k v]] (afn k)) amap)))
 
-(defn map-key [afn amap]
-  (into {}
-    (for [[k v] amap]
-      [(afn k) v]
-      )))
+(defn map-key
+  [afn amap]
+  (into {} (for [[k v] amap] [(afn k) v])))
 
-(defn separate [pred aseq]
+(defn separate
+  [pred aseq]
   [(filter pred aseq) (filter (complement pred) aseq)])
 
-(defn full-path [parent name]
+(defn full-path
+  [parent name]
   (let [toks (tokenize-path parent)]
-    (toks->path (conj toks name))
-    ))
+    (toks->path (conj toks name))))
 
 (def not-nil? (complement nil?))
 
-(defn barr [& vals]
+(defn barr
+  [& vals]
   (byte-array (map byte vals)))
 
-(defn halt-process! [val & msg]
+(defn halt-process!
+  [val & msg]
   (log-message "Halting process: " msg)
-  (.halt (Runtime/getRuntime) val)
-  )
+  (.halt (Runtime/getRuntime) val))
 
-(defn sum [vals]
+(defn sum
+  [vals]
   (reduce + vals))
 
 (defn repeat-seq
   ([aseq]
-    (apply concat (repeat aseq)))
+   (apply concat (repeat aseq)))
   ([amt aseq]
-    (apply concat (repeat amt aseq))
-    ))
+   (apply concat (repeat amt aseq))))
 
 (defn div
   "Perform floating point division on the arguments."
-  [f & rest] (apply / (double f) rest))
+  [f & rest]
+  (apply / (double f) rest))
 
-(defn defaulted [val default]
+(defn defaulted
+  [val default]
   (if val val default))
 
 (defn mk-counter
   ([] (mk-counter 1))
   ([start-val]
-     (let [val (atom (dec start-val))]
-       (fn []
-         (swap! val inc)))))
+   (let [val (atom (dec start-val))]
+     (fn [] (swap! val inc)))))
 
 (defmacro for-times [times & body]
   `(for [i# (range ~times)]
-     ~@body
-     ))
+     ~@body))
 
 (defmacro dofor [& body]
   `(doall (for ~@body)))
@@ -351,9 +354,9 @@
   "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}"
   [amap]
   (reduce (fn [m [k v]]
-    (let [existing (get m v [])]
-      (assoc m v (conj existing k))))
-    {} amap))
+            (let [existing (get m v [])]
+              (assoc m v (conj existing k))))
+          {} amap))
 
 (defmacro print-vars [& vars]
   (let [prints (for [v vars] `(println ~(str v) ~v))]
@@ -366,16 +369,14 @@
         split (.split name "@")]
     (when-not (= 2 (count split))
       (throw (RuntimeException. (str "Got unexpected process name: " name))))
-    (first split)
-    ))
+    (first split)))
 
 (defn exec-command! [command]
   (let [[comm-str & args] (seq (.split command " "))
         command (CommandLine. comm-str)]
     (doseq [a args]
       (.addArgument command a))
-    (.execute (DefaultExecutor.) command)
-    ))
+    (.execute (DefaultExecutor.) command)))
 
 (defn extract-dir-from-jar [jarpath dir destdir]
   (try-cause
@@ -385,25 +386,22 @@
           (.mkdirs (.getParentFile (File. destdir (.getName file))))
           (with-open [out (FileOutputStream. (File. destdir (.getName file)))]
             (io/copy (.getInputStream jarpath file) out)))))
-  (catch IOException e
-    (log-message "Could not extract " dir " from " jarpath))
-  ))
+    (catch IOException e
+      (log-message "Could not extract " dir " from " jarpath))))
 
 (defn ensure-process-killed! [pid]
   ;; TODO: should probably do a ps ax of some sort to make sure it was killed
   (try-cause
     (exec-command! (str (if on-windows? "taskkill /f /pid " "kill -9 ") pid))
-  (catch ExecuteException e
-    (log-message "Error when trying to kill " pid ". Process is probably already dead."))
-    ))
+    (catch ExecuteException e
+      (log-message "Error when trying to kill " pid ". Process is probably already dead."))))
 
 (defnk launch-process [command :environment {}]
   (let [builder (ProcessBuilder. command)
         process-env (.environment builder)]
     (doseq [[k v] environment]
       (.put process-env k v))
-    (.start builder)
-    ))
+    (.start builder)))
 
 (defn sleep-secs [secs]
   (when (pos? secs)
@@ -427,23 +425,21 @@
                    :start true
                    :thread-name nil]
   (let [thread (Thread.
-                (fn []
-                  (try-cause
-                    (let [afn (if factory? (afn) afn)]
-                      (loop []
-                        (let [sleep-time (afn)]
-                          (when-not (nil? sleep-time)
-                            (sleep-secs sleep-time)
-                            (recur))
-                          )))
-                    (catch InterruptedException e
-                      (log-message "Async loop interrupted!")
-                      )
-                    (catch Throwable t
-                      (log-error t "Async loop died!")
-                      (kill-fn t)
-                      ))
-                  ))]
+                 (fn []
+                   (try-cause
+                     (let [afn (if factory? (afn) afn)]
+                       (loop []
+                         (let [sleep-time (afn)]
+                           (when-not (nil? sleep-time)
+                             (sleep-secs sleep-time)
+                             (recur))
+                           )))
+                     (catch InterruptedException e
+                       (log-message "Async loop interrupted!")
+                       )
+                     (catch Throwable t
+                       (log-error t "Async loop died!")
+                       (kill-fn t)))))]
     (.setDaemon thread daemon)
     (.setPriority thread priority)
     (when thread-name
@@ -452,26 +448,30 @@
       (.start thread))
     ;; should return object that supports stop, interrupt, join, and waiting?
     (reify SmartThread
-      (start [this]
+      (start
+        [this]
         (.start thread))
-      (join [this]
+      (join
+        [this]
         (.join thread))
-      (interrupt [this]
+      (interrupt
+        [this]
         (.interrupt thread))
-      (sleeping? [this]
-        (Time/isThreadWaiting thread)
-        ))
-      ))
+      (sleeping?
+        [this]
+        (Time/isThreadWaiting thread)))))
 
-(defn exists-file? [path]
+(defn exists-file?
+  [path]
   (.exists (File. path)))
 
-(defn rmr [path]
+(defn rmr
+  [path]
   (log-debug "Rmr path " path)
   (when (exists-file? path)
     (try
       (FileUtils/forceDelete (File. path))
-    (catch FileNotFoundException e))))
+      (catch FileNotFoundException e))))
 
 (defn rmpath
   "Removes file or directory at the path. Not recursive. Throws exception on failure"
@@ -480,107 +480,123 @@
   (when (exists-file? path)
     (let [deleted? (.delete (File. path))]
       (when-not deleted?
-        (throw (RuntimeException. (str "Failed to delete " path))))
-      )))
+        (throw (RuntimeException. (str "Failed to delete " path)))))))
 
 (defn local-mkdirs
   [path]
   (log-debug "Making dirs at " path)
   (FileUtils/forceMkdir (File. path)))
 
-(defn touch [path]
+(defn touch
+  [path]
   (log-debug "Touching file at " path)
   (let [success? (do (if on-windows? (.mkdirs (.getParentFile (File. path))))
-                     (.createNewFile (File. path)))]
+                   (.createNewFile (File. path)))]
     (when-not success?
-      (throw (RuntimeException. (str "Failed to touch " path))))
-    ))
+      (throw (RuntimeException. (str "Failed to touch " path))))))
 
-(defn read-dir-contents [dir]
+(defn read-dir-contents
+  [dir]
   (if (exists-file? dir)
     (let [content-files (.listFiles (File. dir))]
       (map #(.getName ^File %) content-files))
-    [] ))
+    []))
 
-(defn compact [aseq]
+(defn compact
+  [aseq]
   (filter (complement nil?) aseq))
 
-(defn current-classpath []
+(defn current-classpath
+  []
   (System/getProperty "java.class.path"))
 
-(defn add-to-classpath [classpath paths]
+(defn add-to-classpath
+  [classpath paths]
   (str/join class-path-separator (cons classpath paths)))
 
-(defn ^ReentrantReadWriteLock mk-rw-lock []
+(defn ^ReentrantReadWriteLock mk-rw-lock
+  []
   (ReentrantReadWriteLock.))
 
-(defmacro read-locked [rw-lock & body]
+(defmacro read-locked
+  [rw-lock & body]
   (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})]
     `(let [rlock# (.readLock ~lock)]
        (try (.lock rlock#)
-            ~@body
-            (finally (.unlock rlock#))))))
+         ~@body
+         (finally (.unlock rlock#))))))
 
-(defmacro write-locked [rw-lock & body]
+(defmacro write-locked
+  [rw-lock & body]
   (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})]
     `(let [wlock# (.writeLock ~lock)]
        (try (.lock wlock#)
-            ~@body
-            (finally (.unlock wlock#))))))
+         ~@body
+         (finally (.unlock wlock#))))))
 
-(defn wait-for-condition [apredicate]
+(defn wait-for-condition
+  [apredicate]
   (while (not (apredicate))
-    (Time/sleep 100)
-    ))
+    (Time/sleep 100)))
 
-(defn some? [pred aseq]
+(defn some?
+  [pred aseq]
   ((complement nil?) (some pred aseq)))
 
-(defn time-delta [time-secs]
+(defn time-delta
+  [time-secs]
   (- (current-time-secs) time-secs))
 
-(defn time-delta-ms [time-ms]
+(defn time-delta-ms
+  [time-ms]
   (- (System/currentTimeMillis) (long time-ms)))
 
-(defn parse-int [str]
+(defn parse-int
+  [str]
   (Integer/valueOf str))
 
-(defn integer-divided [sum num-pieces]
+(defn integer-divided
+  [sum num-pieces]
   (clojurify-structure (Utils/integerDivided sum num-pieces)))
 
-(defn collectify [obj]
-  (if (or (sequential? obj) (instance? Collection obj)) obj [obj]))
+(defn collectify
+  [obj]
+  (if (or (sequential? obj) (instance? Collection obj))
+    obj
+    [obj]))
 
-(defn to-json [obj]
+(defn to-json
+  [obj]
   (JSONValue/toJSONString obj))
 
-(defn from-json [^String str]
+(defn from-json
+  [^String str]
   (if str
     (clojurify-structure
-     (JSONValue/parse str))
-    nil
-    ))
-
-(defmacro letlocals [& body]
-   (let [[tobind lexpr] (split-at (dec (count body)) body)
-         binded (vec (mapcat (fn [e]
-                  (if (and (list? e) (= 'bind (first e)))
-                     [(second e) (last e)]
-                     ['_ e]
-                     ))
-                  tobind ))]
-     `(let ~binded
-         ~(first lexpr)
-      )))
-
-(defn remove-first [pred aseq]
+      (JSONValue/parse str))
+    nil))
+
+(defmacro letlocals
+  [& body]
+  (let [[tobind lexpr] (split-at (dec (count body)) body)
+        binded (vec (mapcat (fn [e]
+                              (if (and (list? e) (= 'bind (first e)))
+                                [(second e) (last e)]
+                                ['_ e]
+                                ))
+                            tobind))]
+    `(let ~binded
+       ~(first lexpr))))
+
+(defn remove-first
+  [pred aseq]
   (let [[b e] (split-with (complement pred) aseq)]
     (when (empty? e)
       (throw (IllegalArgumentException. "Nothing to remove")))
-    (concat b (rest e))
-    ))
+    (concat b (rest e))))
 
-(defn assoc-non-nil [m k v]
+(defn assoc-non-nil
+  [m k v]
   (if v (assoc m k v) m))
 
 (defn multi-set
@@ -589,76 +605,71 @@
   (apply merge-with +
          (map #(hash-map % 1) aseq)))
 
-(defn set-var-root* [avar val]
+(defn set-var-root*
+  [avar val]
   (alter-var-root avar (fn [avar] val)))
 
-(defmacro set-var-root [var-sym val]
+(defmacro set-var-root
+  [var-sym val]
   `(set-var-root* (var ~var-sym) ~val))
 
-(defmacro with-var-roots [bindings & body]
+(defmacro with-var-roots
+  [bindings & body]
   (let [settings (partition 2 bindings)
         tmpvars (repeatedly (count settings) (partial gensym "old"))
         vars (map first settings)
         savevals (vec (mapcat (fn [t v] [t v]) tmpvars vars))
         setters (for [[v s] settings] `(set-var-root ~v ~s))
-        restorers (map (fn [v s] `(set-var-root ~v ~s)) vars tmpvars)
-        ]
+        restorers (map (fn [v s] `(set-var-root ~v ~s)) vars tmpvars)]
     `(let ~savevals
-      ~@setters
-      (try
-        ~@body
-      (finally
-        ~@restorers))
-      )))
+       ~@setters
+       (try
+         ~@body
+         (finally
+           ~@restorers)))))
 
 (defn map-diff
   "Returns mappings in m2 that aren't in m1"
   [m1 m2]
-  (into {}
-    (filter
-      (fn [[k v]] (not= v (m1 k)))
-      m2
-      )))
-
-
-(defn select-keys-pred [pred amap]
-  (into {}
-        (filter
-         (fn [[k v]]
-           (pred k))
-         amap)))
+  (into {} (filter (fn [[k v]] (not= v (m1 k))) m2)))
 
+(defn select-keys-pred
+  [pred amap]
+  (into {} (filter (fn [[k v]] (pred k)) amap)))
 
-(defn rotating-random-range [choices]
+(defn rotating-random-range
+  [choices]
   (let [rand (Random.)
         choices (ArrayList. choices)]
     (Collections/shuffle choices rand)
     [(MutableInt. -1) choices rand]))
 
-(defn acquire-random-range-id [[^MutableInt curr ^List state ^Random rand]]
+(defn acquire-random-range-id
+  [[^MutableInt curr ^List state ^Random rand]]
   (when (>= (.increment curr) (.size state))
     (.set curr 0)
     (Collections/shuffle state rand))
   (.get state (.get curr)))
 
 ; this can be rewritten to be tail recursive
-(defn interleave-all [& colls]
+(defn interleave-all
+  [& colls]
   (if (empty? colls)
     []
     (let [colls (filter (complement empty?) colls)
           my-elems (map first colls)
           rest-elems (apply interleave-all (map rest colls))]
-      (concat my-elems rest-elems)
-      )))
+      (concat my-elems rest-elems))))
 
-(defn update [m k afn]
+(defn update
+  [m k afn]
   (assoc m k (afn (get m k))))
 
-(defn any-intersection [& sets]
+(defn any-intersection
+  [& sets]
   (let [elem->count (multi-set (apply concat sets))]
     (-> (filter-val #(> % 1) elem->count)
-        keys
-        )))
+        keys)))
 
 (defn between?
   "val >= lower and val <= upper"
@@ -666,19 +677,20 @@
   (and (>= val lower)
        (<= val upper)))
 
-(defmacro benchmark [& body]
+(defmacro benchmark
+  [& body]
   `(let [l# (doall (range 1000000))]
      (time
        (doseq [i# l#]
          ~@body))))
 
-(defn rand-sampler [freq]
+(defn rand-sampler
+  [freq]
   (let [r (java.util.Random.)]
-    (fn []
-      (= 0 (.nextInt r freq)))
-    ))
+    (fn [] (= 0 (.nextInt r freq)))))
 
-(defn even-sampler [freq]
+(defn even-sampler
+  [freq]
   (let [freq (int freq)
         start (int 0)
         r (java.util.Random.)
@@ -690,38 +702,42 @@
           (when (>= i freq)
             (.set curr start)
             (.set target (.nextInt r freq))))
-          (= (.get curr) (.get target)))
+        (= (.get curr) (.get target)))
       {:rate freq})))
 
-(defn sampler-rate [sampler]
+(defn sampler-rate
+  [sampler]
   (:rate (meta sampler)))
 
-(defn class-selector [obj & args] (class obj))
+(defn class-selector
+  [obj & args]
+  (class obj))
 
 (defn uptime-computer []
   (let [start-time (current-time-secs)]
-    (fn []
-      (time-delta start-time)
-      )))
+    (fn [] (time-delta start-time))))
 
 (defn stringify-error [error]
   (let [result (StringWriter.)
         printer (PrintWriter. result)]
     (.printStackTrace error printer)
-    (.toString result)
-    ))
+    (.toString result)))
 
-(defn nil-to-zero [v]
+(defn nil-to-zero
+  [v]
   (or v 0))
 
-(defn bit-xor-vals [vals]
+(defn bit-xor-vals
+  [vals]
   (reduce bit-xor 0 vals))
 
-(defmacro with-error-reaction [afn & body]
+(defmacro with-error-reaction
+  [afn & body]
   `(try ~@body
      (catch Throwable t# (~afn t#))))
 
-(defn container []
+(defn container
+  []
   (Container.))
 
 (defn container-set! [^Container container obj]
@@ -737,7 +753,8 @@
 (defn throw-runtime [& strs]
   (throw (RuntimeException. (apply str strs))))
 
-(defn redirect-stdio-to-slf4j! []
+(defn redirect-stdio-to-slf4j!
+  []
   ;; set-var-root doesn't work with *out* and *err*, so digging much deeper here
   ;; Unfortunately, this code seems to work at the REPL but not when spawned as worker processes
   ;; it might have something to do with being a child process
@@ -751,29 +768,32 @@
   ;;         true))
   (log-capture! "STDIO"))
 
-(defn spy [prefix val]
+(defn spy
+  [prefix val]
   (log-message prefix ": " val)
   val)
 
-(defn zip-contains-dir? [zipfile target]
+(defn zip-contains-dir?
+  [zipfile target]
   (let [entries (->> zipfile (ZipFile.) .entries enumeration-seq (map (memfn getName)))]
-    (some? #(.startsWith % (str target "/")) entries)
-    ))
+    (some? #(.startsWith % (str target "/")) entries)))
 
-(defn url-encode [s]
+(defn url-encode
+  [s]
   (java.net.URLEncoder/encode s "UTF-8"))
 
-(defn url-decode [s]
+(defn url-decode
+  [s]
   (java.net.URLDecoder/decode s "UTF-8"))
 
-(defn join-maps [& maps]
+(defn join-maps
+  [& maps]
   (let [all-keys (apply set/union (for [m maps] (-> m keys set)))]
-    (into {}
-      (for [k all-keys]
-        [k (for [m maps] (m k))]
-        ))))
+    (into {} (for [k all-keys]
+               [k (for [m maps] (m k))]))))
 
-(defn partition-fixed [max-num-chunks aseq]
+(defn partition-fixed
+  [max-num-chunks aseq]
   (if (zero? max-num-chunks)
     []
     (let [chunks (->> (integer-divided (count aseq) max-num-chunks)
@@ -792,99 +812,115 @@
                    rest-data)))))))
 
 
-(defn assoc-apply-self [curr key afn]
+(defn assoc-apply-self
+  [curr key afn]
   (assoc curr key (afn curr)))
 
-(defmacro recursive-map [& forms]
+(defmacro recursive-map
+  [& forms]
   (->> (partition 2 forms)
        (map (fn [[key form]] `(assoc-apply-self ~key (fn [~'<>] ~form))))
        (concat `(-> {}))))
 
-(defn current-stack-trace []
+(defn current-stack-trace
+  []
   (->> (Thread/currentThread)
        .getStackTrace
        (map str)
-       (str/join "\n")
-       ))
+       (str/join "\n")))
 
-(defn get-iterator [^Iterable alist]
+(defn get-iterator
+  [^Iterable alist]
   (if alist (.iterator alist)))
 
-(defn iter-has-next? [^Iterator iter]
+(defn iter-has-next?
+  [^Iterator iter]
   (if iter (.hasNext iter) false))
 
-(defn iter-next [^Iterator iter]
+(defn iter-next
+  [^Iterator iter]
   (.next iter))
 
-(defmacro fast-list-iter [pairs & body]
+(defmacro fast-list-iter
+  [pairs & body]
   (let [pairs (partition 2 pairs)
         lists (map second pairs)
         elems (map first pairs)
         iters (map (fn [_] (gensym)) lists)
-        bindings (->> (map (fn [i l] [i `(get-iterator ~l)]) iters lists) (apply concat))
+        bindings (->> (map (fn [i l] [i `(get-iterator ~l)]) iters lists)
+                      (apply concat))
         tests (map (fn [i] `(iter-has-next? ~i)) iters)
-        assignments (->> (map (fn [e i] [e `(iter-next ~i)]) elems iters) (apply concat))]
+        assignments (->> (map (fn [e i] [e `(iter-next ~i)]) elems iters)
+                         (apply concat))]
     `(let [~@bindings]
        (while (and ~@tests)
          (let [~@assignments]
-           ~@body
-           )))))
+           ~@body)))))
 
-(defn fast-list-map [afn alist]
+(defn fast-list-map
+  [afn alist]
   (let [ret (ArrayList.)]
     (fast-list-iter [e alist]
-      (.add ret (afn e)))
-    ret ))
+                    (.add ret (afn e)))
+    ret))
 
-(defmacro fast-list-for [[e alist] & body]
+(defmacro fast-list-for
+  [[e alist] & body]
   `(fast-list-map (fn [~e] ~@body) ~alist))
 
-(defn map-iter [^Map amap]
+(defn map-iter
+  [^Map amap]
   (if amap (-> amap .entrySet .iterator)))
 
-(defn convert-entry [^Map$Entry entry]
+(defn convert-entry
+  [^Map$Entry entry]
   [(.getKey entry) (.getValue entry)])
 
-(defmacro fast-map-iter [[bind amap] & body]
+(defmacro fast-map-iter
+  [[bind amap] & body]
   `(let [iter# (map-iter ~amap)]
-    (while (iter-has-next? iter#)
-      (let [entry# (iter-next iter#)
-            ~bind (convert-entry entry#)]
-        ~@body
-        ))))
+     (while (iter-has-next? iter#)
+       (let [entry# (iter-next iter#)
+             ~bind (convert-entry entry#)]
+         ~@body))))
 
-(defn fast-first [^List alist]
+(defn fast-first
+  [^List alist]
   (.get alist 0))
 
-(defmacro get-with-default [amap key default-val]
+(defmacro get-with-default
+  [amap key default-val]
   `(let [curr# (.get ~amap ~key)]
      (if curr#
        curr#
        (do
          (let [new# ~default-val]
            (.put ~amap ~key new#)
-           new#
-           )))))
+           new#)))))
 
-(defn fast-group-by [afn alist]
+(defn fast-group-by
+  [afn alist]
   (let [ret (HashMap.)]
-    (fast-list-iter [e alist]
+    (fast-list-iter
+      [e alist]
       (let [key (afn e)
             ^List curr (get-with-default ret key (ArrayList.))]
         (.add curr e)))
-    ret ))
+    ret))
 
-(defn new-instance [klass]
+(defn new-instance
+  [klass]
   (let [klass (if (string? klass) (Class/forName klass) klass)]
-    (.newInstance klass)
-    ))
+    (.newInstance klass)))
 
 (defmacro -<>
   ([x] x)
-  ([x form] (if (seq? form)
-              (with-meta
-                (let [[begin [_ & end]] (split-with #(not= % '<>) form)]
-                  (concat begin [x] end))
-                (meta form))
-              (list form x)))
-  ([x form & more] `(-<> (-<> ~x ~form) ~@more)))
+  ([x form]
+   (if (seq? form)
+     (with-meta
+       (let [[begin [_ & end]] (split-with #(not= % '<>) form)]
+         (concat begin [x] end))
+       (meta form))
+     (list form x)))
+  ([x form & more]
+   `(-<> (-<> ~x ~form) ~@more)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/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 ab3e0b2..46d1c69 100644
--- a/storm-core/src/clj/backtype/storm/zookeeper.clj
+++ b/storm-core/src/clj/backtype/storm/zookeeper.clj
@@ -13,6 +13,7 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.zookeeper
   (:import [org.apache.curator.retry RetryNTimes])
   (:import [org.apache.curator.framework.api CuratorEvent CuratorEventType CuratorListener UnhandledErrorListener])
@@ -31,47 +32,50 @@
   {Watcher$Event$KeeperState/Disconnected :disconnected
    Watcher$Event$KeeperState/SyncConnected :connected
    Watcher$Event$KeeperState/AuthFailed :auth-failed
-   Watcher$Event$KeeperState/Expired :expired
-  })
+   Watcher$Event$KeeperState/Expired :expired})
 
 (def zk-event-types
   {Watcher$Event$EventType/None :none
    Watcher$Event$EventType/NodeCreated :node-created
    Watcher$Event$EventType/NodeDeleted :node-deleted
    Watcher$Event$EventType/NodeDataChanged :node-data-changed
-   Watcher$Event$EventType/NodeChildrenChanged :node-children-changed
-  })
+   Watcher$Event$EventType/NodeChildrenChanged :node-children-changed})
 
-(defn- default-watcher [state type path]
+(defn- default-watcher
+  [state type path]
   (log-message "Zookeeper state update: " state type path))
 
-(defnk mk-client [conf servers port :root "" :watcher default-watcher :auth-conf nil]
+(defnk mk-client
+  [conf servers port
+   :root ""
+   :watcher default-watcher
+   :auth-conf nil]
   (let [fk (Utils/newCurator conf servers port root (when auth-conf (ZookeeperAuthInfo. auth-conf)))]
     (.. fk
         (getCuratorListenable)
         (addListener
-         (reify CuratorListener
-           (^void eventReceived [this ^CuratorFramework _fk ^CuratorEvent e]
-             (when (= (.getType e) CuratorEventType/WATCHED)                  
-               (let [^WatchedEvent event (.getWatchedEvent e)]
-                 (watcher (zk-keeper-states (.getState event))
-                          (zk-event-types (.getType event))
-                          (.getPath event))))))))
-;;    (.. fk
-;;        (getUnhandledErrorListenable)
-;;        (addListener
-;;         (reify UnhandledErrorListener
-;;           (unhandledError [this msg error]
-;;             (if (or (exception-cause? InterruptedException error)
-;;                     (exception-cause? java.nio.channels.ClosedByInterruptException error))
-;;               (do (log-warn-error error "Zookeeper exception " msg)
-;;                   (let [to-throw (InterruptedException.)]
-;;                     (.initCause to-throw error)
-;;                     (throw to-throw)
-;;                     ))
-;;               (do (log-error error "Unrecoverable Zookeeper error " msg)
-;;                   (halt-process! 1 "Unrecoverable Zookeeper error")))
-;;             ))))
+          (reify CuratorListener
+            (^void eventReceived [this ^CuratorFramework _fk ^CuratorEvent e]
+                   (when (= (.getType e) CuratorEventType/WATCHED)
+                     (let [^WatchedEvent event (.getWatchedEvent e)]
+                       (watcher (zk-keeper-states (.getState event))
+                                (zk-event-types (.getType event))
+                                (.getPath event))))))))
+    ;;    (.. fk
+    ;;        (getUnhandledErrorListenable)
+    ;;        (addListener
+    ;;         (reify UnhandledErrorListener
+    ;;           (unhandledError [this msg error]
+    ;;             (if (or (exception-cause? InterruptedException error)
+    ;;                     (exception-cause? java.nio.channels.ClosedByInterruptException error))
+    ;;               (do (log-warn-error error "Zookeeper exception " msg)
+    ;;                   (let [to-throw (InterruptedException.)]
+    ;;                     (.initCause to-throw error)
+    ;;                     (throw to-throw)
+    ;;                     ))
+    ;;               (do (log-error error "Unrecoverable Zookeeper error " msg)
+    ;;                   (halt-process! 1 "Unrecoverable Zookeeper error")))
+    ;;             ))))
     (.start fk)
     fk))
 
@@ -82,27 +86,30 @@
 
 (defn create-node
   ([^CuratorFramework zk ^String path ^bytes data mode]
-    (try
-      (.. zk (create) (withMode (zk-create-modes mode)) (withACL ZooDefs$Ids/OPEN_ACL_UNSAFE) (forPath (normalize-path path) data))
-      (catch Exception e (throw (wrap-in-runtime e)))))
+   (try
+     (.. zk (create) (withMode (zk-create-modes mode)) (withACL ZooDefs$Ids/OPEN_ACL_UNSAFE) (forPath (normalize-path path) data))
+     (catch Exception e (throw (wrap-in-runtime e)))))
   ([^CuratorFramework zk ^String path ^bytes data]
-    (create-node zk path data :persistent)))
+   (create-node zk path data :persistent)))
 
-(defn exists-node? [^CuratorFramework zk ^String path watch?]
+(defn exists-node?
+  [^CuratorFramework zk ^String path watch?]
   ((complement nil?)
-    (try
-      (if watch?
-         (.. zk (checkExists) (watched) (forPath (normalize-path path))) 
-         (.. zk (checkExists) (forPath (normalize-path path))))
-      (catch Exception e (throw (wrap-in-runtime e))))))
-
-(defnk delete-node [^CuratorFramework zk ^String path :force false]
+   (try
+     (if watch?
+       (.. zk (checkExists) (watched) (forPath (normalize-path path)))
+       (.. zk (checkExists) (forPath (normalize-path path))))
+     (catch Exception e (throw (wrap-in-runtime e))))))
+
+(defnk delete-node
+  [^CuratorFramework zk ^String path :force false]
   (try-cause  (.. zk (delete) (forPath (normalize-path path)))
-    (catch KeeperException$NoNodeException e
-      (when-not force (throw e)))
-    (catch Exception e (throw (wrap-in-runtime e)))))
+             (catch KeeperException$NoNodeException e
+               (when-not force (throw e)))
+             (catch Exception e (throw (wrap-in-runtime e)))))
 
-(defn mkdirs [^CuratorFramework zk ^String path]
+(defn mkdirs
+  [^CuratorFramework zk ^String path]
   (let [path (normalize-path path)]
     (when-not (or (= path "/") (exists-node? zk path false))
       (mkdirs zk (parent-path path))
@@ -113,59 +120,68 @@
           ))
       )))
 
-(defn get-data [^CuratorFramework zk ^String path watch?]
+(defn get-data
+  [^CuratorFramework zk ^String path watch?]
   (let [path (normalize-path path)]
     (try-cause
       (if (exists-node? zk path watch?)
         (if watch?
           (.. zk (getData) (watched) (forPath path))
           (.. zk (getData) (forPath path))))
-    (catch KeeperException$NoNodeException e
-      ;; this is fine b/c we still have a watch from the successful exists call
-      nil )
-    (catch Exception e (throw (wrap-in-runtime e))))))
+      (catch KeeperException$NoNodeException e
+        ;; this is fine b/c we still have a watch from the successful exists call
+        nil )
+      (catch Exception e (throw (wrap-in-runtime e))))))
 
-(defn get-children [^CuratorFramework zk ^String path watch?]
+(defn get-children
+  [^CuratorFramework zk ^String path watch?]
   (try
     (if watch?
       (.. zk (getChildren) (watched) (forPath (normalize-path path)))
       (.. zk (getChildren) (forPath (normalize-path path))))
     (catch Exception e (throw (wrap-in-runtime e)))))
 
-(defn set-data [^CuratorFramework zk ^String path ^bytes data]
+(defn set-data
+  [^CuratorFramework zk ^String path ^bytes data]
   (try
     (.. zk (setData) (forPath (normalize-path path) data))
     (catch Exception e (throw (wrap-in-runtime e)))))
 
-(defn exists [^CuratorFramework zk ^String path watch?]
+(defn exists
+  [^CuratorFramework zk ^String path watch?]
   (exists-node? zk path watch?))
 
-(defn delete-recursive [^CuratorFramework zk ^String path]
+(defn delete-recursive
+  [^CuratorFramework zk ^String path]
   (let [path (normalize-path path)]
     (when (exists-node? zk path false)
-      (let [children (try-cause (get-children zk path false)
-                                (catch KeeperException$NoNodeException e
-                                  []
-                                  ))]
+      (let [children (try-cause
+                       (get-children zk path false)
+                       (catch KeeperException$NoNodeException e []))]
         (doseq [c children]
           (delete-recursive zk (full-path path c)))
-        (delete-node zk path :force true)
-        ))))
+        (delete-node zk path :force true)))))
 
-(defnk mk-inprocess-zookeeper [localdir :port nil]
+(defnk mk-inprocess-zookeeper
+  [localdir :port nil]
   (let [localfile (File. localdir)
         zk (ZooKeeperServer. localfile localfile 2000)
-        [retport factory] (loop [retport (if port port 2000)]
-                            (if-let [factory-tmp (try-cause (doto (NIOServerCnxnFactory.) (.configure (InetSocketAddress. retport) 0))
-                                              (catch BindException e
-                                                (when (> (inc retport) (if port port 65535))
-                                                  (throw (RuntimeException. "No port is available to launch an inprocess zookeeper.")))))]
-                              [retport factory-tmp]
-                              (recur (inc retport))))]
-    (log-message "Starting inprocess zookeeper at port " retport " and dir " localdir)    
+        [retport factory]
+        (loop [retport (if port port 2000)]
+          (if-let [factory-tmp
+                   (try-cause
+                     (doto (NIOServerCnxnFactory.)
+                       (.configure (InetSocketAddress. retport) 0))
+                     (catch BindException e
+                       (when (> (inc retport) (if port port 65535))
+                         (throw (RuntimeException.
+                                  "No port is available to launch an inprocess zookeeper.")))))]
+            [retport factory-tmp]
+            (recur (inc retport))))]
+    (log-message "Starting inprocess zookeeper at port " retport " and dir " localdir)
     (.startup factory zk)
-    [retport factory]
-    ))
+    [retport factory]))
 
-(defn shutdown-inprocess-zookeeper [handle]
+(defn shutdown-inprocess-zookeeper
+  [handle]
   (.shutdown handle))


[04/13] git commit: move towards idiomatic Clojure style

Posted by bo...@apache.org.
move towards idiomatic Clojure style

Summary:
* When using defn, put function arguments on a separate line.
* Remove dangling ),],}
* Try to keep lines at 80 characters or less

Going forward, I’d like to see
* more docstrings
* most lines under 80 characters

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

Branch: refs/heads/master
Commit: 2278fc9623d5b71bca9ece3857e645e85ab1562f
Parents: eee8b24
Author: David James <da...@bluemontlabs.com>
Authored: Sat May 31 13:55:54 2014 -0400
Committer: David James <da...@bluemontlabs.com>
Committed: Sat May 31 13:55:54 2014 -0400

----------------------------------------------------------------------
 .../src/clj/backtype/storm/LocalCluster.clj     |   71 +-
 storm-core/src/clj/backtype/storm/LocalDRPC.clj |    5 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj |    9 +-
 storm-core/src/clj/backtype/storm/clojure.clj   |   10 +-
 storm-core/src/clj/backtype/storm/cluster.clj   |  427 +++---
 storm-core/src/clj/backtype/storm/config.clj    |  145 +-
 storm-core/src/clj/backtype/storm/disruptor.clj |   83 +-
 storm-core/src/clj/backtype/storm/event.clj     |   49 +-
 storm-core/src/clj/backtype/storm/log.clj       |   22 +-
 .../clj/backtype/storm/process_simulator.clj    |   27 +-
 storm-core/src/clj/backtype/storm/stats.clj     |  289 ++--
 storm-core/src/clj/backtype/storm/testing.clj   |  523 ++++----
 storm-core/src/clj/backtype/storm/testing4j.clj |   58 +-
 storm-core/src/clj/backtype/storm/thrift.clj    |  255 ++--
 storm-core/src/clj/backtype/storm/timer.clj     |   94 +-
 storm-core/src/clj/backtype/storm/tuple.clj     |    7 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   | 1262 +++++++++---------
 storm-core/src/clj/backtype/storm/util.clj      |  732 +++++-----
 storm-core/src/clj/backtype/storm/zookeeper.clj |  158 ++-
 19 files changed, 2205 insertions(+), 2021 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/LocalCluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/LocalCluster.clj b/storm-core/src/clj/backtype/storm/LocalCluster.clj
index 77f3b3f..dc8214d 100644
--- a/storm-core/src/clj/backtype/storm/LocalCluster.clj
+++ b/storm-core/src/clj/backtype/storm/LocalCluster.clj
@@ -13,66 +13,75 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.LocalCluster
   (:use [backtype.storm testing config])
   (:import [java.util Map])
   (:gen-class
-   :init init
-   :implements [backtype.storm.ILocalCluster]
-   :constructors {[] [] [java.util.Map] []}
-   :state state ))
+    :init init
+    :implements [backtype.storm.ILocalCluster]
+    :constructors {[] [] [java.util.Map] []}
+    :state state))
 
 (defn -init
   ([]
-     (let [ret (mk-local-storm-cluster :daemon-conf {TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true})]
-       [[] ret]
-       ))
+   (let [ret (mk-local-storm-cluster
+               :daemon-conf
+               {TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS true})]
+     [[] ret]))
   ([^Map stateMap]
-     [[] stateMap]))
+   [[] stateMap]))
 
-(defn -submitTopology [this name conf topology]
-  (submit-local-topology (:nimbus (. this state))
-                      name
-                      conf
-                      topology))
+(defn -submitTopology
+  [this name conf topology]
+  (submit-local-topology
+    (:nimbus (. this state)) name conf topology))
 
-(defn -submitTopologyWithOpts [this name conf topology submit-opts]
-  (submit-local-topology-with-opts (:nimbus (. this state))
-                      name
-                      conf
-                      topology
-                      submit-opts))
+(defn -submitTopologyWithOpts
+  [this name conf topology submit-opts]
+  (submit-local-topology-with-opts
+    (:nimbus (. this state)) name conf topology submit-opts))
 
-(defn -shutdown [this]
+(defn -shutdown
+  [this]
   (kill-local-storm-cluster (. this state)))
 
-(defn -killTopology [this name]
+(defn -killTopology
+  [this name]
   (.killTopology (:nimbus (. this state)) name))
 
-(defn -getTopologyConf [this id]
+(defn -getTopologyConf
+  [this id]
   (.getTopologyConf (:nimbus (. this state)) id))
 
-(defn -getTopology [this id]
+(defn -getTopology
+  [this id]
   (.getTopology (:nimbus (. this state)) id))
 
-(defn -getClusterInfo [this]
+(defn -getClusterInfo
+  [this]
   (.getClusterInfo (:nimbus (. this state))))
 
-(defn -getTopologyInfo [this id]
+(defn -getTopologyInfo
+  [this id]
   (.getTopologyInfo (:nimbus (. this state)) id))
 
-(defn -killTopologyWithOpts [this name opts]
+(defn -killTopologyWithOpts
+  [this name opts]
   (.killTopologyWithOpts (:nimbus (. this state)) name opts))
 
-(defn -activate [this name]
+(defn -activate
+  [this name]
   (.activate (:nimbus (. this state)) name))
 
-(defn -deactivate [this name]
+(defn -deactivate
+  [this name]
   (.deactivate (:nimbus (. this state)) name))
 
-(defn -rebalance [this name opts]
+(defn -rebalance
+  [this name opts]
   (.rebalance (:nimbus (. this state)) name opts))
 
-(defn -getState [this]
+(defn -getState
+  [this]
   (.state this))
-

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/LocalDRPC.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/LocalDRPC.clj b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
index a6dab95..daead2e 100644
--- a/storm-core/src/clj/backtype/storm/LocalDRPC.clj
+++ b/storm-core/src/clj/backtype/storm/LocalDRPC.clj
@@ -13,6 +13,7 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.LocalDRPC
   (:require [backtype.storm.daemon [drpc :as drpc]])
   (:use [backtype.storm util])
@@ -45,9 +46,9 @@
 (defn -failRequest [this id]
   (.failRequest (:handler (. this state)) id)
   )
-  
+
 (defn -getServiceId [this]
-  (:service-id (. this state)))  
+  (:service-id (. this state)))
 
 (defn -shutdown [this]
   (ServiceRegistry/unregisterService (:service-id (. this state)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/bootstrap.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/bootstrap.clj b/storm-core/src/clj/backtype/storm/bootstrap.clj
index 5f34ff1..3ee6f76 100644
--- a/storm-core/src/clj/backtype/storm/bootstrap.clj
+++ b/storm-core/src/clj/backtype/storm/bootstrap.clj
@@ -13,9 +13,11 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.bootstrap)
 
-(defmacro bootstrap []
+(defmacro bootstrap
+  []
   '(do
      (import (quote [backtype.storm Constants]))
      (import (quote [backtype.storm.testing FeederSpout TestPlannerBolt TestPlannerSpout
@@ -32,7 +34,7 @@
      (import (quote [backtype.storm.task IBolt IOutputCollector
                      OutputCollector TopologyContext ShellBolt
                      GeneralTopologyContext WorkerTopologyContext]))
-     (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs 
+     (import (quote [backtype.storm.coordination CoordinatedBolt CoordinatedBolt$SourceArgs
                      IBatchBolt BatchBoltExecutor]))
      (import (quote [backtype.storm.drpc KeyedFairBolt]))
      (import (quote [backtype.storm.daemon Shutdownable]))
@@ -58,5 +60,4 @@
      (import (quote [backtype.storm.grouping CustomStreamGrouping]))
      (import (quote [java.io File FileOutputStream FileInputStream]))
      (import (quote [java.util Collection List Random Map HashMap Collections ArrayList LinkedList]))
-     (import (quote [org.apache.commons.io FileUtils]))
-     ))
+     (import (quote [org.apache.commons.io FileUtils]))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/clojure.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/clojure.clj b/storm-core/src/clj/backtype/storm/clojure.clj
index 33d204b..fdae7cb 100644
--- a/storm-core/src/clj/backtype/storm/clojure.clj
+++ b/storm-core/src/clj/backtype/storm/clojure.clj
@@ -13,6 +13,7 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.clojure
   (:use [backtype.storm bootstrap util])
   (:import [backtype.storm StormSubmitter])
@@ -25,7 +26,6 @@
   (:import [java.util List])
   (:require [backtype.storm [thrift :as thrift]]))
 
-
 (defn direct-stream [fields]
   (StreamInfo. fields true))
 
@@ -145,9 +145,9 @@
   (tuple-values [this collector ^String stream]
     (let [^TopologyContext context (:context collector)
           fields (..  context (getThisOutputFields stream) toList) ]
-      (vec (map (into 
-                  (empty this) (for [[k v] this] 
-                                   [(if (keyword? k) (name k) k) v])) 
+      (vec (map (into
+                  (empty this) (for [[k v] this]
+                                   [(if (keyword? k) (name k) k) v]))
                 fields))))
   java.util.List
   (tuple-values [this collector stream]
@@ -195,7 +195,7 @@
 (defn submit-remote-topology [name conf topology]
   (StormSubmitter/submitTopology name conf topology))
 
-(defn local-cluster []  
+(defn local-cluster []
   ;; do this to avoid a cyclic dependency of
   ;; LocalCluster -> testing -> nimbus -> bootstrap -> clojure -> LocalCluster
   (eval '(new backtype.storm.LocalCluster)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/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 b5c1e3b..e370b97 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -13,30 +13,30 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.cluster
   (:import [org.apache.zookeeper.data Stat])
   (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException])
   (:import [backtype.storm.utils Utils])
   (:use [backtype.storm util log config])
   (:require [backtype.storm [zookeeper :as zk]])
-  (:require [backtype.storm.daemon [common :as common]])
-  
-  )
+  (:require [backtype.storm.daemon [common :as common]]))
 
 (defprotocol ClusterState
   (set-ephemeral-node [this path data])
   (delete-node [this path])
   (create-sequential [this path data])
-  (set-data [this path data])  ;; if node does not exist, create persistent with this data 
+  ;; if node does not exist, create persistent with this data
+  (set-data [this path data])
   (get-data [this path watch?])
   (get-children [this path watch?])
   (mkdirs [this path])
   (close [this])
   (register [this callback])
-  (unregister [this id])
-  )
+  (unregister [this id]))
 
-(defn mk-distributed-cluster-state [conf]
+(defn mk-distributed-cluster-state
+  [conf]
   (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf conf)]
     (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT))
     (.close zk))
@@ -48,83 +48,85 @@
                          :auth-conf conf
                          :root (conf STORM-ZOOKEEPER-ROOT)
                          :watcher (fn [state type path]
-                                     (when @active
-                                       (when-not (= :connected state)
-                                         (log-warn "Received event " state ":" type ":" path " with disconnected Zookeeper."))
-                                       (when-not (= :none type)
-                                         (doseq [callback (vals @callbacks)]
-                                           (callback type path))))
-                                       ))]
+                                    (when @active
+                                      (when-not (= :connected state)
+                                        (log-warn "Received event " state ":" type ":" path " with disconnected Zookeeper."))
+                                      (when-not (= :none type)
+                                        (doseq [callback (vals @callbacks)]
+                                          (callback type path))))))]
     (reify
      ClusterState
-     (register [this callback]
-               (let [id (uuid)]
-                 (swap! callbacks assoc id callback)
-                 id
-                 ))
-     (unregister [this id]
-                 (swap! callbacks dissoc id))
-
-     (set-ephemeral-node [this path data]
-                         (zk/mkdirs zk (parent-path path))
-                         (if (zk/exists zk path false)
-                           (try-cause
-                             (zk/set-data zk path data) ; should verify that it's ephemeral
-                             (catch KeeperException$NoNodeException e
-                               (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
-                               (zk/create-node zk path data :ephemeral)
-                               ))
-                           (zk/create-node zk path data :ephemeral)
-                           ))
-     
-     (create-sequential [this path data]
+
+     (register
+       [this callback]
+       (let [id (uuid)]
+         (swap! callbacks assoc id callback)
+         id))
+
+     (unregister
+       [this id]
+       (swap! callbacks dissoc id))
+
+     (set-ephemeral-node
+       [this path data]
+       (zk/mkdirs zk (parent-path path))
+       (if (zk/exists zk path false)
+         (try-cause
+           (zk/set-data zk path data) ; should verify that it's ephemeral
+           (catch KeeperException$NoNodeException e
+             (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
+             (zk/create-node zk path data :ephemeral)
+             ))
+         (zk/create-node zk path data :ephemeral)))
+
+     (create-sequential
+       [this path data]
        (zk/create-node zk path data :sequential))
-     
-     (set-data [this path data]
-               ;; note: this does not turn off any existing watches
-               (if (zk/exists zk path false)
-                 (zk/set-data zk path data)
-                 (do
-                   (zk/mkdirs zk (parent-path path))
-                   (zk/create-node zk path data :persistent)
-                   )))
-     
-     (delete-node [this path]
-                  (zk/delete-recursive zk path)
-                  )
-     
-     (get-data [this path watch?]
-               (zk/get-data zk path watch?)
-               )
-     
-     (get-children [this path watch?]
-                   (zk/get-children zk path watch?))
-     
-     (mkdirs [this path]
-             (zk/mkdirs zk path))
-     
-     (close [this]
-            (reset! active false)
-            (.close zk))
-     )))
+
+     (set-data
+       [this path data]
+       ;; note: this does not turn off any existing watches
+       (if (zk/exists zk path false)
+         (zk/set-data zk path data)
+         (do
+           (zk/mkdirs zk (parent-path path))
+           (zk/create-node zk path data :persistent))))
+
+     (delete-node
+       [this path]
+       (zk/delete-recursive zk path))
+
+     (get-data
+       [this path watch?]
+       (zk/get-data zk path watch?))
+
+     (get-children
+       [this path watch?]
+       (zk/get-children zk path watch?))
+
+     (mkdirs
+       [this path]
+       (zk/mkdirs zk path))
+
+     (close
+       [this]
+       (reset! active false)
+       (.close zk)))))
 
 (defprotocol StormClusterState
   (assignments [this callback])
   (assignment-info [this storm-id callback])
   (active-storms [this])
   (storm-base [this storm-id callback])
-
   (get-worker-heartbeat [this storm-id node port])
   (executor-beats [this storm-id executor->node+port])
   (supervisors [this callback])
-  (supervisor-info [this supervisor-id])  ;; returns nil if doesn't exist
-
+  (supervisor-info [this supervisor-id]) ;; returns nil if doesn't exist
   (setup-heartbeats! [this storm-id])
   (teardown-heartbeats! [this storm-id])
   (teardown-topology-errors! [this storm-id])
   (heartbeat-storms [this])
   (error-topologies [this])
-
   (worker-heartbeat! [this storm-id node port info])
   (remove-worker-heartbeat! [this storm-id node port])
   (supervisor-heartbeat! [this supervisor-id info])
@@ -135,10 +137,7 @@
   (remove-storm! [this storm-id])
   (report-error [this storm-id task-id error])
   (errors [this storm-id task-id])
-
-  (disconnect [this])
-  )
-
+  (disconnect [this]))
 
 (def ASSIGNMENTS-ROOT "assignments")
 (def CODE-ROOT "code")
@@ -153,64 +152,75 @@
 (def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
 (def ERRORS-SUBTREE (str "/" ERRORS-ROOT))
 
-(defn supervisor-path [id]
+(defn supervisor-path
+  [id]
   (str SUPERVISORS-SUBTREE "/" id))
 
-(defn assignment-path [id]
+(defn assignment-path
+  [id]
   (str ASSIGNMENTS-SUBTREE "/" id))
 
-(defn storm-path [id]
+(defn storm-path
+  [id]
   (str STORMS-SUBTREE "/" id))
 
-(defn workerbeat-storm-root [storm-id]
+(defn workerbeat-storm-root
+  [storm-id]
   (str WORKERBEATS-SUBTREE "/" storm-id))
 
-(defn workerbeat-path [storm-id node port]
+(defn workerbeat-path
+  [storm-id node port]
   (str (workerbeat-storm-root storm-id) "/" node "-" port))
 
-(defn error-storm-root [storm-id]
+(defn error-storm-root
+  [storm-id]
   (str ERRORS-SUBTREE "/" storm-id))
 
-(defn error-path [storm-id component-id]
+(defn error-path
+  [storm-id component-id]
   (str (error-storm-root storm-id) "/" (url-encode component-id)))
 
-(defn- issue-callback! [cb-atom]
+(defn- issue-callback!
+  [cb-atom]
   (let [cb @cb-atom]
     (reset! cb-atom nil)
     (when cb
-      (cb))
-    ))
+      (cb))))
 
-(defn- issue-map-callback! [cb-atom id]
+(defn- issue-map-callback!
+  [cb-atom id]
   (let [cb (@cb-atom id)]
     (swap! cb-atom dissoc id)
     (when cb
-      (cb id))
-    ))
+      (cb id))))
 
-(defn- maybe-deserialize [ser]
+(defn- maybe-deserialize
+  [ser]
   (when ser
     (Utils/deserialize ser)))
 
 (defstruct TaskError :error :time-secs)
 
-(defn- parse-error-path [^String p]
+(defn- parse-error-path
+  [^String p]
   (Long/parseLong (.substring p 1)))
 
-
-(defn convert-executor-beats [executors worker-hb]
-  ;; ensures that we only return heartbeats for executors assigned to this worker
+(defn convert-executor-beats
+  "Ensures that we only return heartbeats for executors assigned to
+  this worker."
+  [executors worker-hb]
   (let [executor-stats (:executor-stats worker-hb)]
     (->> executors
-      (map (fn [t] 
-             (if (contains? executor-stats t)
-               {t {:time-secs (:time-secs worker-hb)
-                    :uptime (:uptime worker-hb)
-                    :stats (get executor-stats t)}})))
-      (into {}))))
+         (map (fn [t]
+                (if (contains? executor-stats t)
+                  {t {:time-secs (:time-secs worker-hb)
+                      :uptime (:uptime worker-hb)
+                      :stats (get executor-stats t)}})))
+         (into {}))))
 
 ;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
-(defn mk-storm-cluster-state [cluster-state-spec]
+(defn mk-storm-cluster-state
+  [cluster-state-spec]
   (let [[solo? cluster-state] (if (satisfies? ClusterState cluster-state-spec)
                                 [false cluster-state-spec]
                                 [true (mk-distributed-cluster-state cluster-state-spec)])
@@ -219,164 +229,171 @@
         assignments-callback (atom nil)
         storm-base-callback (atom {})
         state-id (register
-                  cluster-state
-                  (fn [type path]
-                    (let [[subtree & args] (tokenize-path path)]
-                      (condp = subtree
-                          ASSIGNMENTS-ROOT (if (empty? args)
-                                             (issue-callback! assignments-callback)
-                                             (issue-map-callback! assignment-info-callback (first args)))
-                          SUPERVISORS-ROOT (issue-callback! supervisors-callback)
-                          STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
-                          ;; this should never happen
-                          (halt-process! 30 "Unknown callback for subtree " subtree args)
-                          )
-                      )))]
+                   cluster-state
+                   (fn [type path]
+                     (let [[subtree & args] (tokenize-path path)]
+                       (condp = subtree
+                         ASSIGNMENTS-ROOT (if (empty? args)
+                                            (issue-callback! assignments-callback)
+                                            (issue-map-callback! assignment-info-callback (first args)))
+                         SUPERVISORS-ROOT (issue-callback! supervisors-callback)
+                         STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
+                         ;; this should never happen
+                         (halt-process! 30 "Unknown callback for subtree " subtree args)))))]
     (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE]]
       (mkdirs cluster-state p))
     (reify
-     StormClusterState
-     
-     (assignments [this callback]
+      StormClusterState
+
+      (assignments
+        [this callback]
         (when callback
           (reset! assignments-callback callback))
         (get-children cluster-state ASSIGNMENTS-SUBTREE (not-nil? callback)))
-      
-      (assignment-info [this storm-id callback]
+
+      (assignment-info
+        [this storm-id callback]
         (when callback
           (swap! assignment-info-callback assoc storm-id callback))
-        (maybe-deserialize (get-data cluster-state (assignment-path storm-id) (not-nil? callback)))
-        )
+        (maybe-deserialize (get-data cluster-state (assignment-path storm-id) (not-nil? callback))))
 
-      (active-storms [this]
-        (get-children cluster-state STORMS-SUBTREE false)
-        )
+      (active-storms
+        [this]
+        (get-children cluster-state STORMS-SUBTREE false))
 
-      (heartbeat-storms [this]
-        (get-children cluster-state WORKERBEATS-SUBTREE false)
-        )
+      (heartbeat-storms
+        [this]
+        (get-children cluster-state WORKERBEATS-SUBTREE false))
 
-      (error-topologies [this]
-         (get-children cluster-state ERRORS-SUBTREE false)
-        )
+      (error-topologies
+        [this]
+        (get-children cluster-state ERRORS-SUBTREE false))
 
-      (get-worker-heartbeat [this storm-id node port]
+      (get-worker-heartbeat
+        [this storm-id node port]
         (-> cluster-state
             (get-data (workerbeat-path storm-id node port) false)
             maybe-deserialize))
 
-      (executor-beats [this storm-id executor->node+port]
-        ;; need to take executor->node+port in explicitly so that we don't run into a situation where a 
+      (executor-beats
+        [this storm-id executor->node+port]
+        ;; need to take executor->node+port in explicitly so that we don't run into a situation where a
         ;; long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
         ;; with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
         ;; we avoid situations like that
         (let [node+port->executors (reverse-map executor->node+port)
               all-heartbeats (for [[[node port] executors] node+port->executors]
-                                (->> (get-worker-heartbeat this storm-id node port)
-                                     (convert-executor-beats executors)
-                                     ))]
+                               (->> (get-worker-heartbeat this storm-id node port)
+                                    (convert-executor-beats executors)
+                                    ))]
           (apply merge all-heartbeats)))
 
-      (supervisors [this callback]
+      (supervisors
+        [this callback]
         (when callback
           (reset! supervisors-callback callback))
-        (get-children cluster-state SUPERVISORS-SUBTREE (not-nil? callback))
-        )
+        (get-children cluster-state SUPERVISORS-SUBTREE (not-nil? callback)))
 
-      (supervisor-info [this supervisor-id]
-        (maybe-deserialize (get-data cluster-state (supervisor-path supervisor-id) false))
-        )
+      (supervisor-info
+        [this supervisor-id]
+        (maybe-deserialize (get-data cluster-state (supervisor-path supervisor-id) false)))
 
-      (worker-heartbeat! [this storm-id node port info]
+      (worker-heartbeat!
+        [this storm-id node port info]
         (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize info)))
 
-      (remove-worker-heartbeat! [this storm-id node port]
-        (delete-node cluster-state (workerbeat-path storm-id node port))
-        )
+      (remove-worker-heartbeat!
+        [this storm-id node port]
+        (delete-node cluster-state (workerbeat-path storm-id node port)))
 
-      (setup-heartbeats! [this storm-id]
+      (setup-heartbeats!
+        [this storm-id]
         (mkdirs cluster-state (workerbeat-storm-root storm-id)))
 
-      (teardown-heartbeats! [this storm-id]
+      (teardown-heartbeats!
+        [this storm-id]
         (try-cause
-         (delete-node cluster-state (workerbeat-storm-root storm-id))
-         (catch KeeperException e
-           (log-warn-error e "Could not teardown heartbeats for " storm-id)
-           )))
+          (delete-node cluster-state (workerbeat-storm-root storm-id))
+          (catch KeeperException e
+            (log-warn-error e "Could not teardown heartbeats for " storm-id))))
 
-      (teardown-topology-errors! [this storm-id]
+      (teardown-topology-errors!
+        [this storm-id]
         (try-cause
-         (delete-node cluster-state (error-storm-root storm-id))         
-         (catch KeeperException e
-           (log-warn-error e "Could not teardown errors for " storm-id)
-           )))
+          (delete-node cluster-state (error-storm-root storm-id))
+          (catch KeeperException e
+            (log-warn-error e "Could not teardown errors for " storm-id))))
 
-      (supervisor-heartbeat! [this supervisor-id info]
-        (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info))
-        )
+      (supervisor-heartbeat!
+        [this supervisor-id info]
+        (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info)))
 
-      (activate-storm! [this storm-id storm-base]
-        (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base))
-        )
+      (activate-storm!
+        [this storm-id storm-base]
+        (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base)))
 
-      (update-storm! [this storm-id new-elems]
+      (update-storm!
+        [this storm-id new-elems]
         (let [base (storm-base this storm-id nil)
               executors (:component->executors base)
               new-elems (update new-elems :component->executors (partial merge executors))]
           (set-data cluster-state (storm-path storm-id)
-                                  (-> base
-                                      (merge new-elems)
-                                      Utils/serialize))))
+                    (-> base
+                        (merge new-elems)
+                        Utils/serialize))))
 
-      (storm-base [this storm-id callback]
+      (storm-base
+        [this storm-id callback]
         (when callback
           (swap! storm-base-callback assoc storm-id callback))
-        (maybe-deserialize (get-data cluster-state (storm-path storm-id) (not-nil? callback)))
-        )
+        (maybe-deserialize (get-data cluster-state (storm-path storm-id) (not-nil? callback))))
 
-      (remove-storm-base! [this storm-id]
-        (delete-node cluster-state (storm-path storm-id))
-        )
+      (remove-storm-base!
+        [this storm-id]
+        (delete-node cluster-state (storm-path storm-id)))
 
-      (set-assignment! [this storm-id info]
-        (set-data cluster-state (assignment-path storm-id) (Utils/serialize info))
-        )
+      (set-assignment!
+        [this storm-id info]
+        (set-data cluster-state (assignment-path storm-id) (Utils/serialize info)))
 
-      (remove-storm! [this storm-id]
+      (remove-storm!
+        [this storm-id]
         (delete-node cluster-state (assignment-path storm-id))
         (remove-storm-base! this storm-id))
 
-      (report-error [this storm-id component-id error]                
-         (let [path (error-path storm-id component-id)
-               data {:time-secs (current-time-secs) :error (stringify-error error)}
-               _ (mkdirs cluster-state path)
-               _ (create-sequential cluster-state (str path "/e") (Utils/serialize data))
-               to-kill (->> (get-children cluster-state path false)
-                            (sort-by parse-error-path)
-                            reverse
-                            (drop 10))]
-           (doseq [k to-kill]
-             (delete-node cluster-state (str path "/" k)))))
-
-      (errors [this storm-id component-id]
-         (let [path (error-path storm-id component-id)
-               _ (mkdirs cluster-state path)
-               children (get-children cluster-state path false)
-               errors (dofor [c children]
-                             (let [data (-> (get-data cluster-state (str path "/" c) false)
-                                            maybe-deserialize)]
-                               (when data
-                                 (struct TaskError (:error data) (:time-secs data))
-                                 )))
-               ]
-           (->> (filter not-nil? errors)
-                (sort-by (comp - :time-secs)))))
-      
-      (disconnect [this]
+      (report-error
+        [this storm-id component-id error]
+        (let [path (error-path storm-id component-id)
+              data {:time-secs (current-time-secs) :error (stringify-error error)}
+              _ (mkdirs cluster-state path)
+              _ (create-sequential cluster-state (str path "/e") (Utils/serialize data))
+              to-kill (->> (get-children cluster-state path false)
+                           (sort-by parse-error-path)
+                           reverse
+                           (drop 10))]
+          (doseq [k to-kill]
+            (delete-node cluster-state (str path "/" k)))))
+
+      (errors
+        [this storm-id component-id]
+        (let [path (error-path storm-id component-id)
+              _ (mkdirs cluster-state path)
+              children (get-children cluster-state path false)
+              errors (dofor [c children]
+                            (let [data (-> (get-data cluster-state (str path "/" c) false)
+                                           maybe-deserialize)]
+                              (when data
+                                (struct TaskError (:error data) (:time-secs data))
+                                )))
+              ]
+          (->> (filter not-nil? errors)
+               (sort-by (comp - :time-secs)))))
+
+      (disconnect
+        [this]
         (unregister cluster-state state-id)
         (when solo?
-          (close cluster-state)))
-      )))
+          (close cluster-state))))))
 
 ;; daemons have a single thread that will respond to events
 ;; start with initialize event
@@ -395,9 +412,8 @@
 ;; everyone reads this in full to understand structure
 ;; /tasks/{storm id}/{task id} ; just contains bolt id
 
-
 ;; supervisors send heartbeats here, master doesn't subscribe but checks asynchronously
-;; /supervisors/status/{ephemeral node ids}  ;; node metadata such as port ranges are kept here 
+;; /supervisors/status/{ephemeral node ids}  ;; node metadata such as port ranges are kept here
 
 ;; tasks send heartbeats here, master doesn't subscribe, just checks asynchronously
 ;; /taskbeats/{storm id}/{ephemeral task id}
@@ -406,8 +422,6 @@
 ;; master manipulates
 ;; /storms/{storm id}
 
-
-
 ;; Zookeeper flows:
 
 ;; Master:
@@ -423,7 +437,6 @@
 ;; 3. make new assignment to fix any problems
 ;; 4. if a storm exists but is not taken down fully, ensure that storm takedown is launched (step by step remove tasks and finally remove assignments)
 
-
 ;; masters only possible watches is on ephemeral nodes and tasks, and maybe not even
 
 ;; Supervisor:
@@ -439,8 +452,6 @@
 ;; 1. monitor assignments, reroute when assignments change
 ;; 2. monitor storm (when storm turns off, error if assignments change) - take down tasks as master turns them off
 
-
-
 ;; locally on supervisor: workers write pids locally on startup, supervisor deletes it on shutdown (associates pid with worker name)
 ;; supervisor periodically checks to make sure processes are alive
 ;; {rootdir}/workers/{storm id}/{worker id}   ;; contains pid inside

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/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 15be94d..14beb21 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -13,14 +13,14 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.config
   (:import [java.io FileReader File])
   (:import [backtype.storm Config ConfigValidation$FieldValidator])
   (:import [backtype.storm.utils Utils LocalState])
   (:import [org.apache.commons.io FileUtils])
   (:require [clojure [string :as str]])
-  (:use [backtype.storm util])
-  )
+  (:use [backtype.storm util]))
 
 (def RESOURCES-SUBDIR "resources")
 
@@ -32,13 +32,11 @@
   (let [name (.getName f)
         new-name (clojure-config-name name)]
     (eval
-      `(def ~(symbol new-name) (. Config ~(symbol name))))
-      ))
+      `(def ~(symbol new-name) (. Config ~(symbol name))))))
 
 (def ALL-CONFIGS
   (dofor [f (seq (.getFields Config))]
-         (.get f nil)
-         ))
+         (.get f nil)))
 
 (defmulti get-FieldValidator class-selector)
 
@@ -48,44 +46,49 @@
 (defmethod get-FieldValidator
   ConfigValidation$FieldValidator [validator] validator)
 
-(defmethod get-FieldValidator Object [klass]
+(defmethod get-FieldValidator Object
+  [klass]
   {:pre [(not (nil? klass))]}
   (reify ConfigValidation$FieldValidator
     (validateField [this name v]
-      (if (and (not (nil? v))
-               (not (instance? klass v)))
-        (throw (IllegalArgumentException.
-                 (str "field " name " '" v "' must be a '" (.getName klass) "'")))))))
+                   (if (and (not (nil? v))
+                            (not (instance? klass v)))
+                     (throw (IllegalArgumentException.
+                              (str "field " name " '" v "' must be a '" (.getName klass) "'")))))))
 
 ;; Create a mapping of config-string -> validator
 ;; Config fields must have a _SCHEMA field defined
 (def CONFIG-SCHEMA-MAP
   (->> (.getFields Config)
-          (filter #(not (re-matches #".*_SCHEMA$" (.getName %))))
-          (map (fn [f] [(.get f nil) (get-FieldValidator
-                                       (-> Config
-                                         (.getField (str (.getName f) "_SCHEMA"))
-                                         (.get nil)))]))
-          (into {})))
-
-(defn cluster-mode [conf & args]
+       (filter #(not (re-matches #".*_SCHEMA$" (.getName %))))
+       (map (fn [f] [(.get f nil)
+                     (get-FieldValidator
+                       (-> Config
+                           (.getField (str (.getName f) "_SCHEMA"))
+                           (.get nil)))]))
+       (into {})))
+
+(defn cluster-mode
+  [conf & args]
   (keyword (conf STORM-CLUSTER-MODE)))
 
-(defn local-mode? [conf]
+(defn local-mode?
+  [conf]
   (let [mode (conf STORM-CLUSTER-MODE)]
     (condp = mode
       "local" true
       "distributed" false
       (throw (IllegalArgumentException.
-                (str "Illegal cluster mode in conf: " mode)))
-      )))
+               (str "Illegal cluster mode in conf: " mode))))))
 
-(defn sampling-rate [conf]
+(defn sampling-rate
+  [conf]
   (->> (conf TOPOLOGY-STATS-SAMPLE-RATE)
        (/ 1)
        int))
 
-(defn mk-stats-sampler [conf]
+(defn mk-stats-sampler
+  [conf]
   (even-sampler (sampling-rate conf)))
 
 ; storm.zookeeper.servers:
@@ -93,106 +96,124 @@
 ;     - "server2"
 ;     - "server3"
 ; nimbus.host: "master"
-; 
+;
 ; ########### These all have default values as shown
-; 
+;
 ; ### storm.* configs are general configurations
 ; # the local dir is where jars are kept
 ; storm.local.dir: "/mnt/storm"
 ; storm.zookeeper.port: 2181
 ; storm.zookeeper.root: "/storm"
 
-(defn read-default-config []
+(defn read-default-config
+  []
   (clojurify-structure (Utils/readDefaultConfig)))
 
-(defn validate-configs-with-schemas [conf]
+(defn validate-configs-with-schemas
+  [conf]
   (doseq [[k v] conf
-         :let [schema (CONFIG-SCHEMA-MAP k)]]
+          :let [schema (CONFIG-SCHEMA-MAP k)]]
     (if (not (nil? schema))
       (.validateField schema k v))))
 
-(defn read-storm-config []
-  (let [
-        conf (clojurify-structure (Utils/readStormConfig))]
+(defn read-storm-config
+  []
+  (let [conf (clojurify-structure (Utils/readStormConfig))]
     (validate-configs-with-schemas conf)
     conf))
 
-(defn read-yaml-config [name]
+(defn read-yaml-config
+  [name]
   (let [conf (clojurify-structure (Utils/findAndReadConfigFile name true))]
     (validate-configs-with-schemas conf)
     conf))
 
-(defn master-local-dir [conf]
+(defn master-local-dir
+  [conf]
   (let [ret (str (conf STORM-LOCAL-DIR) file-path-separator "nimbus")]
     (FileUtils/forceMkdir (File. ret))
-    ret
-    ))
+    ret))
 
 (defn master-stormdist-root
   ([conf]
-     (str (master-local-dir conf) file-path-separator "stormdist"))
+   (str (master-local-dir conf) file-path-separator "stormdist"))
   ([conf storm-id]
-     (str (master-stormdist-root conf) file-path-separator storm-id)))
+   (str (master-stormdist-root conf) file-path-separator storm-id)))
 
-(defn master-stormjar-path [stormroot]
+(defn master-stormjar-path
+  [stormroot]
   (str stormroot file-path-separator "stormjar.jar"))
 
-(defn master-stormcode-path [stormroot]
+(defn master-stormcode-path
+  [stormroot]
   (str stormroot file-path-separator "stormcode.ser"))
 
-(defn master-stormconf-path [stormroot]
+(defn master-stormconf-path
+  [stormroot]
   (str stormroot file-path-separator "stormconf.ser"))
 
-(defn master-inbox [conf]
+(defn master-inbox
+  [conf]
   (let [ret (str (master-local-dir conf) file-path-separator "inbox")]
     (FileUtils/forceMkdir (File. ret))
     ret ))
 
-(defn master-inimbus-dir [conf]
+(defn master-inimbus-dir
+  [conf]
   (str (master-local-dir conf) file-path-separator "inimbus"))
 
-(defn supervisor-local-dir [conf]
+(defn supervisor-local-dir
+  [conf]
   (let [ret (str (conf STORM-LOCAL-DIR) file-path-separator "supervisor")]
     (FileUtils/forceMkdir (File. ret))
-    ret
-    ))
+    ret))
 
-(defn supervisor-isupervisor-dir [conf]
+(defn supervisor-isupervisor-dir
+  [conf]
   (str (supervisor-local-dir conf) file-path-separator "isupervisor"))
 
 (defn supervisor-stormdist-root
-  ([conf] (str (supervisor-local-dir conf) file-path-separator "stormdist"))
+  ([conf]
+   (str (supervisor-local-dir conf) file-path-separator "stormdist"))
   ([conf storm-id]
-      (str (supervisor-stormdist-root conf) file-path-separator (url-encode storm-id))))
+   (str (supervisor-stormdist-root conf) file-path-separator (url-encode storm-id))))
 
-(defn supervisor-stormjar-path [stormroot]
+(defn supervisor-stormjar-path
+  [stormroot]
   (str stormroot file-path-separator "stormjar.jar"))
 
-(defn supervisor-stormcode-path [stormroot]
+(defn supervisor-stormcode-path
+  [stormroot]
   (str stormroot file-path-separator "stormcode.ser"))
 
-(defn supervisor-stormconf-path [stormroot]
+(defn supervisor-stormconf-path
+  [stormroot]
   (str stormroot file-path-separator "stormconf.ser"))
 
-(defn supervisor-tmp-dir [conf]
+(defn supervisor-tmp-dir
+  [conf]
   (let [ret (str (supervisor-local-dir conf) file-path-separator "tmp")]
     (FileUtils/forceMkdir (File. ret))
     ret ))
 
-(defn supervisor-storm-resources-path [stormroot]
+(defn supervisor-storm-resources-path
+  [stormroot]
   (str stormroot file-path-separator RESOURCES-SUBDIR))
 
-(defn ^LocalState supervisor-state [conf]
+(defn ^LocalState supervisor-state
+  [conf]
   (LocalState. (str (supervisor-local-dir conf) file-path-separator "localstate")))
 
-(defn read-supervisor-storm-conf [conf storm-id]
+(defn read-supervisor-storm-conf
+  [conf storm-id]
   (let [stormroot (supervisor-stormdist-root conf storm-id)
         conf-path (supervisor-stormconf-path stormroot)
         topology-path (supervisor-stormcode-path stormroot)]
     (merge conf (Utils/deserialize (FileUtils/readFileToByteArray (File. conf-path))))
     ))
 
-(defn read-supervisor-topology [conf storm-id]
+(defn read-supervisor-topology
+  [conf storm-id]
   (let [stormroot (supervisor-stormdist-root conf storm-id)
         topology-path (supervisor-stormcode-path stormroot)]
     (Utils/deserialize (FileUtils/readFileToByteArray (File. topology-path)))
@@ -200,15 +221,16 @@
 
 (defn worker-root
   ([conf]
-     (str (conf STORM-LOCAL-DIR) file-path-separator "workers"))
+   (str (conf STORM-LOCAL-DIR) file-path-separator "workers"))
   ([conf id]
-     (str (worker-root conf) file-path-separator id)))
+   (str (worker-root conf) file-path-separator id)))
 
 (defn worker-pids-root
   [conf id]
   (str (worker-root conf id) file-path-separator "pids"))
 
-(defn worker-pid-path [conf id pid]
+(defn worker-pid-path
+  [conf id pid]
   (str (worker-pids-root conf id) file-path-separator pid))
 
 (defn worker-heartbeats-root
@@ -218,5 +240,6 @@
 ;; workers heartbeat here with pid and timestamp
 ;; if supervisor stops receiving heartbeat, it kills and restarts the process
 ;; in local mode, keep a global map of ids to threads for simulating process management
-(defn ^LocalState worker-state  [conf id]
+(defn ^LocalState worker-state
+  [conf id]
   (LocalState. (worker-heartbeats-root conf id)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/disruptor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/disruptor.clj b/storm-core/src/clj/backtype/storm/disruptor.clj
index 9456d1a..d8c5c91 100644
--- a/storm-core/src/clj/backtype/storm/disruptor.clj
+++ b/storm-core/src/clj/backtype/storm/disruptor.clj
@@ -13,89 +13,90 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.disruptor
   (:import [backtype.storm.utils DisruptorQueue])
   (:import [com.lmax.disruptor MultiThreadedClaimStrategy SingleThreadedClaimStrategy
-              BlockingWaitStrategy SleepingWaitStrategy YieldingWaitStrategy
-              BusySpinWaitStrategy])
+            BlockingWaitStrategy SleepingWaitStrategy YieldingWaitStrategy
+            BusySpinWaitStrategy])
   (:require [clojure [string :as str]])
   (:require [clojure [set :as set]])
   (:use [clojure walk])
-  (:use [backtype.storm util log])
-  )
+  (:use [backtype.storm util log]))
 
 (def CLAIM-STRATEGY
   {:multi-threaded (fn [size] (MultiThreadedClaimStrategy. (int size)))
-   :single-threaded (fn [size] (SingleThreadedClaimStrategy. (int size)))
-    })
-    
+   :single-threaded (fn [size] (SingleThreadedClaimStrategy. (int size)))})
+
 (def WAIT-STRATEGY
   {:block (fn [] (BlockingWaitStrategy.))
    :yield (fn [] (YieldingWaitStrategy.))
    :sleep (fn [] (SleepingWaitStrategy.))
-   :spin (fn [] (BusySpinWaitStrategy.))
-    })
-
+   :spin (fn [] (BusySpinWaitStrategy.))})
 
-(defn- mk-wait-strategy [spec]
+(defn- mk-wait-strategy
+  [spec]
   (if (keyword? spec)
     ((WAIT-STRATEGY spec))
-    (-> (str spec) new-instance)
-    ))
+    (-> (str spec) new-instance)))
 
 ;; :block strategy requires using a timeout on waitFor (implemented in DisruptorQueue), as sometimes the consumer stays blocked even when there's an item on the queue.
 ;; This would manifest itself in Trident when doing 1 batch at a time processing, and the ack_init message
-;; wouldn't make it to the acker until the batch timed out and another tuple was played into the queue, 
+;; wouldn't make it to the acker until the batch timed out and another tuple was played into the queue,
 ;; unblocking the consumer
-(defnk disruptor-queue [buffer-size :claim-strategy :multi-threaded :wait-strategy :block]
+(defnk disruptor-queue
+  [buffer-size :claim-strategy :multi-threaded :wait-strategy :block]
   (DisruptorQueue. ((CLAIM-STRATEGY claim-strategy) buffer-size)
-                   (mk-wait-strategy wait-strategy)
-                   ))
+                   (mk-wait-strategy wait-strategy)))
 
-(defn clojure-handler [afn]
+(defn clojure-handler
+  [afn]
   (reify com.lmax.disruptor.EventHandler
-    (onEvent [this o seq-id batchEnd?]
-      (afn o seq-id batchEnd?)
-      )))
+    (onEvent
+      [this o seq-id batchEnd?]
+      (afn o seq-id batchEnd?))))
 
-(defmacro handler [& args]
+(defmacro handler
+  [& args]
   `(clojure-handler (fn ~@args)))
 
 (defn publish
   ([^DisruptorQueue q o block?]
-    (.publish q o block?))
+   (.publish q o block?))
   ([q o]
-    (publish q o true)))
+   (publish q o true)))
 
-(defn try-publish [^DisruptorQueue q o]
+(defn try-publish
+  [^DisruptorQueue q o]
   (.tryPublish q o))
 
-(defn consume-batch [^DisruptorQueue queue handler]
+(defn consume-batch
+  [^DisruptorQueue queue handler]
   (.consumeBatch queue handler))
 
-(defn consume-batch-when-available [^DisruptorQueue queue handler]
+(defn consume-batch-when-available
+  [^DisruptorQueue queue handler]
   (.consumeBatchWhenAvailable queue handler))
 
-(defn consumer-started! [^DisruptorQueue queue]
+(defn consumer-started!
+  [^DisruptorQueue queue]
   (.consumerStarted queue))
 
-(defn halt-with-interrupt! [^DisruptorQueue queue]
+(defn halt-with-interrupt!
+  [^DisruptorQueue queue]
   (.haltWithInterrupt queue))
 
-(defnk consume-loop* [^DisruptorQueue queue handler :kill-fn (fn [error] (halt-process! 1 "Async loop died!"))
-                      :thread-name nil]
+(defnk consume-loop*
+  [^DisruptorQueue queue handler
+   :kill-fn (fn [error] (halt-process! 1 "Async loop died!"))
+   :thread-name nil]
   (let [ret (async-loop
-              (fn []
-                (consume-batch-when-available queue handler)
-                0 )
+              (fn [] (consume-batch-when-available queue handler) 0)
               :kill-fn kill-fn
-              :thread-name thread-name
-              )]
-     (consumer-started! queue)
-     ret
-     ))
+              :thread-name thread-name)]
+    (consumer-started! queue)
+    ret))
 
 (defmacro consume-loop [queue & handler-args]
   `(let [handler# (handler ~@handler-args)]
-     (consume-loop* ~queue handler#)
-     ))
+     (consume-loop* ~queue handler#)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/event.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/event.clj b/storm-core/src/clj/backtype/storm/event.clj
index 70ba8a6..f92c9bb 100644
--- a/storm-core/src/clj/backtype/storm/event.clj
+++ b/storm-core/src/clj/backtype/storm/event.clj
@@ -13,11 +13,11 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.event
   (:use [backtype.storm log util])
   (:import [backtype.storm.utils Time Utils])
-  (:import [java.util.concurrent LinkedBlockingQueue TimeUnit])
-  )
+  (:import [java.util.concurrent LinkedBlockingQueue TimeUnit]))
 
 (defprotocol EventManager
   (add [this event-fn])
@@ -32,36 +32,37 @@
         ^LinkedBlockingQueue queue (LinkedBlockingQueue.)
         running (atom true)
         runner (Thread.
-                  (fn []
-                    (try-cause
-                      (while @running
-                        (let [r (.take queue)]
-                          (r)
-                          (swap! processed inc)))
-                    (catch InterruptedException t
-                      (log-message "Event manager interrupted"))
-                    (catch Throwable t
-                      (log-error t "Error when processing event")
-                      (halt-process! 20 "Error when processing an event"))
-                      )))]
+                 (fn []
+                   (try-cause
+                     (while @running
+                       (let [r (.take queue)]
+                         (r)
+                         (swap! processed inc)))
+                     (catch InterruptedException t
+                       (log-message "Event manager interrupted"))
+                     (catch Throwable t
+                       (log-error t "Error when processing event")
+                       (halt-process! 20 "Error when processing an event")))))]
     (.setDaemon runner daemon?)
     (.start runner)
     (reify
       EventManager
-      (add [this event-fn]
+
+      (add
+        [this event-fn]
         ;; should keep track of total added and processed to know if this is finished yet
         (when-not @running
           (throw (RuntimeException. "Cannot add events to a shutdown event manager")))
         (swap! added inc)
-        (.put queue event-fn)
-        )
-      (waiting? [this]
+        (.put queue event-fn))
+
+      (waiting?
+        [this]
         (or (Time/isThreadWaiting runner)
-            (= @processed @added)
-            ))
-      (shutdown [this]
+            (= @processed @added)))
+
+      (shutdown
+        [this]
         (reset! running false)
         (.interrupt runner)
-        (.join runner)
-        )
-        )))
+        (.join runner)))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/log.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/log.clj b/storm-core/src/clj/backtype/storm/log.clj
index adb2774..0fcf822 100644
--- a/storm-core/src/clj/backtype/storm/log.clj
+++ b/storm-core/src/clj/backtype/storm/log.clj
@@ -13,26 +13,34 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.log
   (:require [clojure.tools [logging :as log]]))
 
-(defmacro log-message [& args]
+(defmacro log-message
+  [& args]
   `(log/info (str ~@args)))
 
-(defmacro log-error [e & args]
+(defmacro log-error
+  [e & args]
   `(log/log :error ~e (str ~@args)))
 
-(defmacro log-debug [& args]
+(defmacro log-debug
+  [& args]
   `(log/debug (str ~@args)))
 
-(defmacro log-warn-error [e & args]
+(defmacro log-warn-error
+  [e & args]
   `(log/warn (str ~@args) ~e))
 
-(defmacro log-warn [& args]
+(defmacro log-warn
+  [& args]
   `(log/warn (str ~@args)))
 
-(defn log-capture! [& args]
+(defn log-capture!
+  [& args]
   (apply log/log-capture! args))
 
-(defn log-stream [& args]
+(defn log-stream
+  [& args]
   (apply log/log-stream args))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/process_simulator.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/process_simulator.clj b/storm-core/src/clj/backtype/storm/process_simulator.clj
index 0446a98..e0cf6ed 100644
--- a/storm-core/src/clj/backtype/storm/process_simulator.clj
+++ b/storm-core/src/clj/backtype/storm/process_simulator.clj
@@ -13,9 +13,9 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.process-simulator
-  (:use [backtype.storm log util])
-  )
+  (:use [backtype.storm log util]))
 
 (def pid-counter (mk-counter))
 
@@ -26,23 +26,26 @@
 (defn register-process [pid shutdownable]
   (swap! process-map assoc pid shutdownable))
 
-(defn process-handle [pid]
+(defn process-handle
+  [pid]
   (@process-map pid))
 
-(defn all-processes []
+(defn all-processes
+  []
   (vals @process-map))
 
-(defn kill-process [pid]
-  (locking kill-lock ; in case cluster shuts down while supervisor is
-                     ; killing a task
+(defn kill-process
+  "Uses `locking` in case cluster shuts down while supervisor is
+  killing a task"
+  [pid]
+  (locking kill-lock
     (log-message "Killing process " pid)
     (let [shutdownable (process-handle pid)]
       (swap! process-map dissoc pid)
       (when shutdownable
-        (.shutdown shutdownable))
-      )))
+        (.shutdown shutdownable)))))
 
-(defn kill-all-processes []
+(defn kill-all-processes
+  []
   (doseq [pid (keys @process-map)]
-    (kill-process pid)
-    ))
+    (kill-process pid)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/stats.clj b/storm-core/src/clj/backtype/storm/stats.clj
index 944d2b6..b872c6f 100644
--- a/storm-core/src/clj/backtype/storm/stats.clj
+++ b/storm-core/src/clj/backtype/storm/stats.clj
@@ -13,6 +13,7 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.stats
   (:import [backtype.storm.generated Nimbus Nimbus$Processor Nimbus$Iface StormTopology ShellComponent
             NotAliveException AlreadyAliveException InvalidTopologyException GlobalStreamId
@@ -23,43 +24,44 @@
 
 ;;TODO: consider replacing this with some sort of RRD
 
-(defn curr-time-bucket [^Integer time-secs ^Integer bucket-size-secs]
-  (* bucket-size-secs (unchecked-divide-int time-secs bucket-size-secs))
-  )
+(defn curr-time-bucket
+  [^Integer time-secs ^Integer bucket-size-secs]
+  (* bucket-size-secs (unchecked-divide-int time-secs bucket-size-secs)))
 
-(defrecord RollingWindow [updater merger extractor bucket-size-secs num-buckets buckets])
+(defrecord RollingWindow
+  [updater merger extractor bucket-size-secs num-buckets buckets])
 
-(defn rolling-window [updater merger extractor bucket-size-secs num-buckets]
+(defn rolling-window
+  [updater merger extractor bucket-size-secs num-buckets]
   (RollingWindow. updater merger extractor bucket-size-secs num-buckets {}))
 
 (defn update-rolling-window
   ([^RollingWindow rw time-secs & args]
-     ;; this is 2.5x faster than using update-in...
-     (let [time-bucket (curr-time-bucket time-secs (:bucket-size-secs rw))
-           buckets (:buckets rw)
-           curr (get buckets time-bucket)           
-           curr (apply (:updater rw) curr args)
-           ]
-       (assoc rw :buckets (assoc buckets time-bucket curr))
-       )))
-
-(defn value-rolling-window [^RollingWindow rw]
+   ;; this is 2.5x faster than using update-in...
+   (let [time-bucket (curr-time-bucket time-secs (:bucket-size-secs rw))
+         buckets (:buckets rw)
+         curr (get buckets time-bucket)
+         curr (apply (:updater rw) curr args)]
+     (assoc rw :buckets (assoc buckets time-bucket curr)))))
+
+(defn value-rolling-window
+  [^RollingWindow rw]
   ((:extractor rw)
    (let [values (vals (:buckets rw))]
-     (apply (:merger rw) values)
-     )))
+     (apply (:merger rw) values))))
 
-(defn cleanup-rolling-window [^RollingWindow rw]
+(defn cleanup-rolling-window
+  [^RollingWindow rw]
   (let [buckets (:buckets rw)
         cutoff (- (current-time-secs)
                   (* (:num-buckets rw)
                      (:bucket-size-secs rw)))
         to-remove (filter #(< % cutoff) (keys buckets))
         buckets (apply dissoc buckets to-remove)]
-    (assoc rw :buckets buckets)
-    ))
+    (assoc rw :buckets buckets)))
 
-(defn rolling-window-size [^RollingWindow rw]
+(defn rolling-window-size
+  [^RollingWindow rw]
   (* (:bucket-size-secs rw) (:num-buckets rw)))
 
 (defrecord RollingWindowSet [updater extractor windows all-time])
@@ -70,49 +72,52 @@
 
 (defn update-rolling-window-set
   ([^RollingWindowSet rws & args]
-     (let [now (current-time-secs)
-           new-windows (dofor [w (:windows rws)]
-                         (apply update-rolling-window w now args))]
-       (assoc rws :windows new-windows :all-time (apply (:updater rws) (:all-time rws) args))
-       )))
+   (let [now (current-time-secs)
+         new-windows (dofor [w (:windows rws)]
+                            (apply update-rolling-window w now args))]
+     (assoc rws
+       :windows new-windows
+       :all-time (apply (:updater rws) (:all-time rws) args)))))
 
 (defn cleanup-rolling-window-set
   ([^RollingWindowSet rws]
-     (let [windows (:windows rws)]
-       (assoc rws :windows (map cleanup-rolling-window windows))
-       )))
+   (let [windows (:windows rws)]
+     (assoc rws :windows (map cleanup-rolling-window windows)))))
 
-(defn value-rolling-window-set [^RollingWindowSet rws]
+(defn value-rolling-window-set
+  [^RollingWindowSet rws]
   (merge
-   (into {}
-         (for [w (:windows rws)]
-           {(rolling-window-size w) (value-rolling-window w)}
-           ))
-   {:all-time ((:extractor rws) (:all-time rws))}))
+    (into {}
+          (for [w (:windows rws)]
+            {(rolling-window-size w) (value-rolling-window w)}
+            ))
+    {:all-time ((:extractor rws) (:all-time rws))}))
 
 (defn- incr-val
   ([amap key]
-     (incr-val amap key 1))
+   (incr-val amap key 1))
   ([amap key amt]
-     (let [val (get amap key (long 0))]
-       (assoc amap key (+ val amt))
-       )))
+   (let [val (get amap key (long 0))]
+     (assoc amap key (+ val amt)))))
 
-(defn- update-avg [curr val]
+(defn- update-avg
+  [curr val]
   (if curr
     [(+ (first curr) val) (inc (second curr))]
-    [val (long 1)]
-    ))
+    [val (long 1)]))
 
-(defn- merge-avg [& avg]
+(defn- merge-avg
+  [& avg]
   [(apply + (map first avg))
    (apply + (map second avg))
    ])
 
-(defn- extract-avg [pair]
+(defn- extract-avg
+  [pair]
   (double (/ (first pair) (second pair))))
 
-(defn- update-keyed-avg [amap key val]
+(defn- update-keyed-avg
+  [amap key val]
   (assoc amap key (update-avg (get amap key) val)))
 
 (defn- merge-keyed-avg [& vals]
@@ -124,14 +129,16 @@
 (defn- counter-extract [v]
   (if v v {}))
 
-(defn keyed-counter-rolling-window-set [num-buckets & bucket-sizes]
+(defn keyed-counter-rolling-window-set
+  [num-buckets & bucket-sizes]
   (apply rolling-window-set incr-val (partial merge-with +) counter-extract num-buckets bucket-sizes))
 
-(defn avg-rolling-window-set [num-buckets & bucket-sizes]
-  (apply rolling-window-set update-avg merge-avg extract-avg num-buckets bucket-sizes)
-  )
+(defn avg-rolling-window-set
+  [num-buckets & bucket-sizes]
+  (apply rolling-window-set update-avg merge-avg extract-avg num-buckets bucket-sizes))
 
-(defn keyed-avg-rolling-window-set [num-buckets & bucket-sizes]
+(defn keyed-avg-rolling-window-set
+  [num-buckets & bucket-sizes]
   (apply rolling-window-set update-keyed-avg merge-keyed-avg extract-keyed-avg num-buckets bucket-sizes))
 
 ;; (defn choose-bucket [val buckets]
@@ -169,160 +176,166 @@
 ;; 10 minutes, 3 hours, 1 day
 (def STAT-BUCKETS [30 540 4320])
 
-(defn- mk-common-stats [rate]
-  (CommonStats. (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                rate
-                ))
-
-(defn mk-bolt-stats [rate]
-  (BoltExecutorStats. (mk-common-stats rate)
-                  (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                  (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                  (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                  (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                  (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                  ))
-
-(defn mk-spout-stats [rate]
-  (SpoutExecutorStats. (mk-common-stats rate)
-                   (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                   (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                   (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-                   ))
-
-(defmacro update-executor-stat! [stats path & args]
+(defn- mk-common-stats
+  [rate]
+  (CommonStats.
+    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    rate))
+
+(defn mk-bolt-stats
+  [rate]
+  (BoltExecutorStats.
+    (mk-common-stats rate)
+    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))))
+
+(defn mk-spout-stats
+  [rate]
+  (SpoutExecutorStats.
+    (mk-common-stats rate)
+    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))))
+
+(defmacro update-executor-stat!
+  [stats path & args]
   (let [path (collectify path)]
-    `(swap! (-> ~stats ~@path) update-rolling-window-set ~@args)
-    ))
+    `(swap! (-> ~stats ~@path) update-rolling-window-set ~@args)))
 
-(defmacro stats-rate [stats]
+(defmacro stats-rate
+  [stats]
   `(-> ~stats :common :rate))
 
-(defn emitted-tuple! [stats stream]
+(defn emitted-tuple!
+  [stats stream]
   (update-executor-stat! stats [:common :emitted] stream (stats-rate stats)))
 
-(defn transferred-tuples! [stats stream amt]
+(defn transferred-tuples!
+  [stats stream amt]
   (update-executor-stat! stats [:common :transferred] stream (* (stats-rate stats) amt)))
 
-(defn bolt-execute-tuple! [^BoltExecutorStats stats component stream latency-ms]
+(defn bolt-execute-tuple!
+  [^BoltExecutorStats stats component stream latency-ms]
   (let [key [component stream]]
     (update-executor-stat! stats :executed key (stats-rate stats))
-    (update-executor-stat! stats :execute-latencies key latency-ms)
-    ))
+    (update-executor-stat! stats :execute-latencies key latency-ms)))
 
-(defn bolt-acked-tuple! [^BoltExecutorStats stats component stream latency-ms]
+(defn bolt-acked-tuple!
+  [^BoltExecutorStats stats component stream latency-ms]
   (let [key [component stream]]
     (update-executor-stat! stats :acked key (stats-rate stats))
-    (update-executor-stat! stats :process-latencies key latency-ms)
-    ))
+    (update-executor-stat! stats :process-latencies key latency-ms)))
 
-(defn bolt-failed-tuple! [^BoltExecutorStats stats component stream latency-ms]
+(defn bolt-failed-tuple!
+  [^BoltExecutorStats stats component stream latency-ms]
   (let [key [component stream]]
-    (update-executor-stat! stats :failed key (stats-rate stats))
-    ))
+    (update-executor-stat! stats :failed key (stats-rate stats))))
 
-(defn spout-acked-tuple! [^SpoutExecutorStats stats stream latency-ms]
+(defn spout-acked-tuple!
+  [^SpoutExecutorStats stats stream latency-ms]
   (update-executor-stat! stats :acked stream (stats-rate stats))
-  (update-executor-stat! stats :complete-latencies stream latency-ms)
-  )
+  (update-executor-stat! stats :complete-latencies stream latency-ms))
 
-(defn spout-failed-tuple! [^SpoutExecutorStats stats stream latency-ms]
+(defn spout-failed-tuple!
+  [^SpoutExecutorStats stats stream latency-ms]
   (update-executor-stat! stats :failed stream (stats-rate stats))
   )
 
 (defn- cleanup-stat! [stat]
   (swap! stat cleanup-rolling-window-set))
 
-(defn- cleanup-common-stats! [^CommonStats stats]
+(defn- cleanup-common-stats!
+  [^CommonStats stats]
   (doseq [f COMMON-FIELDS]
-    (cleanup-stat! (f stats))
-    ))
+    (cleanup-stat! (f stats))))
 
-(defn cleanup-bolt-stats! [^BoltExecutorStats stats]
+(defn cleanup-bolt-stats!
+  [^BoltExecutorStats stats]
   (cleanup-common-stats! (:common stats))
   (doseq [f BOLT-FIELDS]
-    (cleanup-stat! (f stats))
-    ))
+    (cleanup-stat! (f stats))))
 
-(defn cleanup-spout-stats! [^SpoutExecutorStats stats]
+(defn cleanup-spout-stats!
+  [^SpoutExecutorStats stats]
   (cleanup-common-stats! (:common stats))
   (doseq [f SPOUT-FIELDS]
-    (cleanup-stat! (f stats))
-    ))
+    (cleanup-stat! (f stats))))
 
-(defn- value-stats [stats fields]
-  (into
-   {}
-   (dofor [f fields]
-          [f (value-rolling-window-set @(f stats))]
-          )))
+(defn- value-stats
+  [stats fields]
+  (into {} (dofor [f fields]
+                  [f (value-rolling-window-set @(f stats))])))
 
-(defn- value-common-stats [^CommonStats stats]
+(defn- value-common-stats
+  [^CommonStats stats]
   (merge
-   (value-stats stats COMMON-FIELDS)
-   {:rate (:rate stats)}))
+    (value-stats stats COMMON-FIELDS)
+    {:rate (:rate stats)}))
 
-(defn value-bolt-stats! [^BoltExecutorStats stats]
+(defn value-bolt-stats!
+  [^BoltExecutorStats stats]
   (cleanup-bolt-stats! stats)
   (merge (value-common-stats (:common stats))
          (value-stats stats BOLT-FIELDS)
          {:type :bolt}))
 
-(defn value-spout-stats! [^SpoutExecutorStats stats]
+(defn value-spout-stats!
+  [^SpoutExecutorStats stats]
   (cleanup-spout-stats! stats)
   (merge (value-common-stats (:common stats))
          (value-stats stats SPOUT-FIELDS)
          {:type :spout}))
 
-
 (defmulti render-stats! class-selector)
 
-(defmethod render-stats! SpoutExecutorStats [stats]
+(defmethod render-stats! SpoutExecutorStats
+  [stats]
   (value-spout-stats! stats))
 
-(defmethod render-stats! BoltExecutorStats [stats]
+(defmethod render-stats! BoltExecutorStats
+  [stats]
   (value-bolt-stats! stats))
 
 (defmulti thriftify-specific-stats :type)
 
 (defn window-set-converter
   ([stats key-fn]
-     ;; make the first key a string,
-     (into {}
-           (for [[k v] stats]
-             [(str k)
-              (into {}
-                    (for [[k2 v2] v]
-                      [(key-fn k2) v2]))]
-             )
-           ))
+   ;; make the first key a string,
+   (into {}
+         (for [[k v] stats]
+           [(str k)
+            (into {} (for [[k2 v2] v]
+                       [(key-fn k2) v2]))])))
   ([stats]
-     (window-set-converter stats identity)))
+   (window-set-converter stats identity)))
 
-(defn to-global-stream-id [[component stream]]
-  (GlobalStreamId. component stream)
-  )
+(defn to-global-stream-id
+  [[component stream]]
+  (GlobalStreamId. component stream))
 
 (defmethod thriftify-specific-stats :bolt
   [stats]
   (ExecutorSpecificStats/bolt
-   (BoltStats. (window-set-converter (:acked stats) to-global-stream-id)
-               (window-set-converter (:failed stats) to-global-stream-id)
-               (window-set-converter (:process-latencies stats) to-global-stream-id)
-               (window-set-converter (:executed stats) to-global-stream-id)
-               (window-set-converter (:execute-latencies stats) to-global-stream-id)
-               )))
+    (BoltStats.
+      (window-set-converter (:acked stats) to-global-stream-id)
+      (window-set-converter (:failed stats) to-global-stream-id)
+      (window-set-converter (:process-latencies stats) to-global-stream-id)
+      (window-set-converter (:executed stats) to-global-stream-id)
+      (window-set-converter (:execute-latencies stats) to-global-stream-id))))
 
 (defmethod thriftify-specific-stats :spout
   [stats]
   (ExecutorSpecificStats/spout
-   (SpoutStats. (window-set-converter (:acked stats))
-                (window-set-converter (:failed stats))
-                (window-set-converter (:complete-latencies stats)))
-   ))
+    (SpoutStats. (window-set-converter (:acked stats))
+                 (window-set-converter (:failed stats))
+                 (window-set-converter (:complete-latencies stats)))))
 
-(defn thriftify-executor-stats [stats]
+(defn thriftify-executor-stats
+  [stats]
   (let [specific-stats (thriftify-specific-stats stats)]
     (ExecutorStats. (window-set-converter (:emitted stats))
                     (window-set-converter (:transferred stats))


[02/13] move towards idiomatic Clojure style

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/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 ad1a038..6da383e 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -13,6 +13,7 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.ui.core
   (:use compojure.core)
   (:use ring.middleware.reload)
@@ -38,12 +39,14 @@
 
 (def ^:dynamic *STORM-CONF* (read-storm-config))
 
-(defmacro with-nimbus [nimbus-sym & body]
-  `(thrift/with-nimbus-connection [~nimbus-sym (*STORM-CONF* NIMBUS-HOST) (*STORM-CONF* NIMBUS-THRIFT-PORT)]
-     ~@body
-     ))
+(defmacro with-nimbus
+  [nimbus-sym & body]
+  `(thrift/with-nimbus-connection
+     [~nimbus-sym (*STORM-CONF* NIMBUS-HOST) (*STORM-CONF* NIMBUS-THRIFT-PORT)]
+     ~@body))
 
-(defn get-filled-stats [summs]
+(defn get-filled-stats
+  [summs]
   (->> summs
        (map #(.get_stats ^ExecutorSummary %))
        (filter not-nil?)))
@@ -110,31 +113,32 @@
               "assigned. Click on the port number to open the logviewer page "
               "for this Worker.")})
 
-(defn mk-system-toggle-button [include-sys?]
+(defn mk-system-toggle-button
+  [include-sys?]
   [:p {:class "js-only"}
    [:span.tip.right {:title (:sys-stats tips)}
     [:input {:type "button"
              :value (str (if include-sys? "Hide" "Show") " System Stats")
              :onclick "toggleSys()"}]]])
 
-(defn ui-template [body]
+(defn ui-template
+  [body]
   (html4
-   [:head
-    [:title "Storm UI"]
-    (include-css "/css/bootstrap-1.4.0.css")
-    (include-css "/css/style.css")
-    (include-js "/js/jquery-1.6.2.min.js")
-    (include-js "/js/jquery.tablesorter.min.js")
-    (include-js "/js/jquery.cookies.2.2.0.min.js")
-    (include-js "/js/bootstrap-twipsy.js")
-    (include-js "/js/script.js")
-    ]
-   [:body
-    [:h1 (link-to "/" "Storm UI")]
-    (seq body)
-    ]))
-
-(defn read-storm-version []
+    [:head
+     [:title "Storm UI"]
+     (include-css "/css/bootstrap-1.4.0.css")
+     (include-css "/css/style.css")
+     (include-js "/js/jquery-1.6.2.min.js")
+     (include-js "/js/jquery.tablesorter.min.js")
+     (include-js "/js/jquery.cookies.2.2.0.min.js")
+     (include-js "/js/bootstrap-twipsy.js")
+     (include-js "/js/script.js")]
+    [:body
+     [:h1 (link-to "/" "Storm UI")]
+     (seq body)]))
+
+(defn read-storm-version
+  []
   (let [storm-home (System/getProperty "storm.home")
         release-path (format "%s/RELEASE" storm-home)
         release-file (File. release-path)]
@@ -142,7 +146,8 @@
       (trim (slurp release-path))
       "Unknown")))
 
-(defn cluster-summary-table [^ClusterSummary summ]
+(defn cluster-summary-table
+  [^ClusterSummary summ]
   (let [sups (.get_supervisors summ)
         used-slots (reduce + (map #(.get_num_used_workers ^SupervisorSummary %) sups))
         total-slots (reduce + (map #(.get_num_workers ^SupervisorSummary %) sups))
@@ -164,7 +169,7 @@
             {:text "Free slots" :attr {:class "tip above"
                                        :title (:num-slots tips)}}
             {:text "Total slots" :attr {:class "tip above"
-                                       :title (:num-slots tips)}}
+                                        :title (:num-slots tips)}}
             {:text  "Executors" :attr {:class "tip above"
                                        :title (:num-execs tips)}}
             {:text "Tasks" :attr {:class "tip left"
@@ -176,98 +181,100 @@
              free-slots
              total-slots
              total-executors
-             total-tasks]])
-    ))
+             total-tasks]])))
 
 (defn topology-link
   ([id] (topology-link id id))
   ([id content]
-     (link-to (url-format "/topology/%s" id) (escape-html content))))
+   (link-to (url-format "/topology/%s" id) (escape-html content))))
 
-(defn main-topology-summary-table [summs]
+(defn main-topology-summary-table
+  [summs]
   (sorted-table
-   [{:text "Name" :attr {:class "tip right"
-                         :title (str (:name tips) " " (:name-link tips))}}
-    {:text "Id" :attr {:class "tip right"
-                       :title (:topo-id tips)}}
-    {:text "Status" :attr {:class "tip above"
-                           :title (:status tips)}}
-    {:text "Uptime" :attr {:class "tip above"
-                           :title (:topo-uptime tips)}}
-    {:text "Num workers" :attr {:class "tip above"
-                                :title (:num-workers tips)}}
-    {:text "Num executors" :attr {:class "tip above"
-                                  :title (:num-execs tips)}}
-    {:text "Num tasks" :attr {:class "tip above"
-                              :title (:num-tasks tips)}}]
-   (for [^TopologySummary t summs]
-     [(topology-link (.get_id t) (.get_name t))
-      (escape-html (.get_id t))
-      (.get_status t)
-      (pretty-uptime-sec (.get_uptime_secs t))
-      (.get_num_workers t)
-      (.get_num_executors t)
-      (.get_num_tasks t)
-      ])
-   :time-cols [3]
-   :sort-list "[[0,0]]"
-   ))
-
-(defn supervisor-summary-table [summs]
+    [{:text "Name" :attr {:class "tip right"
+                          :title (str (:name tips) " " (:name-link tips))}}
+     {:text "Id" :attr {:class "tip right"
+                        :title (:topo-id tips)}}
+     {:text "Status" :attr {:class "tip above"
+                            :title (:status tips)}}
+     {:text "Uptime" :attr {:class "tip above"
+                            :title (:topo-uptime tips)}}
+     {:text "Num workers" :attr {:class "tip above"
+                                 :title (:num-workers tips)}}
+     {:text "Num executors" :attr {:class "tip above"
+                                   :title (:num-execs tips)}}
+     {:text "Num tasks" :attr {:class "tip above"
+                               :title (:num-tasks tips)}}]
+    (for [^TopologySummary t summs]
+      [(topology-link (.get_id t) (.get_name t))
+       (escape-html (.get_id t))
+       (.get_status t)
+       (pretty-uptime-sec (.get_uptime_secs t))
+       (.get_num_workers t)
+       (.get_num_executors t)
+       (.get_num_tasks t)])
+    :time-cols [3]
+    :sort-list "[[0,0]]"))
+
+(defn supervisor-summary-table
+  [summs]
   (sorted-table
-   [{:text "Id" :attr {:class "tip right"
-                       :title (:sup-id tips)}}
-    {:text "Host" :attr {:class "tip above"
-                         :title (:sup-host tips)}}
-    {:text "Uptime" :attr {:class "tip above"
-                         :title (:sup-uptime tips)}}
-    {:text "Slots" :attr {:class "tip above"
-                          :title (:num-slots tips)}}
-    {:text "Used slots" :attr {:class "tip left"
-                               :title (:num-slots tips)}}]
-   (for [^SupervisorSummary s summs]
-     [(.get_supervisor_id s)
-      (.get_host s)
-      (pretty-uptime-sec (.get_uptime_secs s))
-      (.get_num_workers s)
-      (.get_num_used_workers s)])
-   :time-cols [2]))
-
-(defn configuration-table [conf]
+    [{:text "Id" :attr {:class "tip right"
+                        :title (:sup-id tips)}}
+     {:text "Host" :attr {:class "tip above"
+                          :title (:sup-host tips)}}
+     {:text "Uptime" :attr {:class "tip above"
+                            :title (:sup-uptime tips)}}
+     {:text "Slots" :attr {:class "tip above"
+                           :title (:num-slots tips)}}
+     {:text "Used slots" :attr {:class "tip left"
+                                :title (:num-slots tips)}}]
+    (for [^SupervisorSummary s summs]
+      [(.get_supervisor_id s)
+       (.get_host s)
+       (pretty-uptime-sec (.get_uptime_secs s))
+       (.get_num_workers s)
+       (.get_num_used_workers s)])
+    :time-cols [2]))
+
+(defn configuration-table
+  [conf]
   (sorted-table ["Key" "Value"]
-    (map #(vector (key %) (str (val %))) conf)))
+                (map #(vector (key %) (str (val %))) conf)))
 
-(defn main-page []
+(defn main-page
+  []
   (with-nimbus nimbus
-    (let [summ (.getClusterInfo ^Nimbus$Client nimbus)]
-      (concat
-       [[:h2 "Cluster Summary"]]
-       [(cluster-summary-table summ)]
-       [[:h2 "Topology summary"]]
-       (main-topology-summary-table (.get_topologies summ))
-       [[:h2 "Supervisor summary"]]
-       (supervisor-summary-table (.get_supervisors summ))
-       [[:h2 "Nimbus Configuration"]]
-       (configuration-table (from-json (.getNimbusConf ^Nimbus$Client nimbus)))
-       ))))
-
-(defn component-type [^StormTopology topology id]
+               (let [summ (.getClusterInfo ^Nimbus$Client nimbus)]
+                 (concat
+                   [[:h2 "Cluster Summary"]]
+                   [(cluster-summary-table summ)]
+                   [[:h2 "Topology summary"]]
+                   (main-topology-summary-table (.get_topologies summ))
+                   [[:h2 "Supervisor summary"]]
+                   (supervisor-summary-table (.get_supervisors summ))
+                   [[:h2 "Nimbus Configuration"]]
+                   (configuration-table (from-json (.getNimbusConf ^Nimbus$Client nimbus)))))))
+
+(defn component-type
+  [^StormTopology topology id]
   (let [bolts (.get_bolts topology)
         spouts (.get_spouts topology)]
     (cond
-     (.containsKey bolts id) :bolt
-     (.containsKey spouts id) :spout
-     )))
+      (.containsKey bolts id) :bolt
+      (.containsKey spouts id) :spout)))
 
-(defn executor-summary-type [topology ^ExecutorSummary s]
+(defn executor-summary-type
+  [topology ^ExecutorSummary s]
   (component-type topology (.get_component_id s)))
 
 (defn add-pairs
   ([] [0 0])
   ([[a1 a2] [b1 b2]]
-      [(+ a1 b1) (+ a2 b2)]))
+   [(+ a1 b1) (+ a2 b2)]))
 
-(defn expand-averages [avg counts]
+(defn expand-averages
+  [avg counts]
   (let [avg (clojurify-structure avg)
         counts (clojurify-structure counts)]
     (into {}
@@ -278,63 +285,64 @@
                      [stream
                       [(* c (get-in avg [slice stream]))
                        c]]
-                     ))]
-            ))))
+                     ))]))))
 
 
-(defn expand-averages-seq [average-seq counts-seq]
+(defn expand-averages-seq
+  [average-seq counts-seq]
   (->> (map vector average-seq counts-seq)
        (map #(apply expand-averages %))
        (apply merge-with
               (fn [s1 s2]
                 (merge-with
-                 add-pairs
-                 s1
-                 s2)))
-       ))
+                  add-pairs
+                  s1
+                  s2)))))
 
 (defn- val-avg [[t c]]
   (if (= t 0) 0
-      (double (/ t c))))
+    (double (/ t c))))
 
-(defn aggregate-averages [average-seq counts-seq]
+(defn aggregate-averages
+  [average-seq counts-seq]
   (->> (expand-averages-seq average-seq counts-seq)
        (map-val
-        (fn [s]
-          (map-val val-avg s)
-          ))
-       ))
+         (fn [s]
+           (map-val val-avg s)))))
 
-(defn aggregate-counts [counts-seq]
+(defn aggregate-counts
+  [counts-seq]
   (->> counts-seq
        (map clojurify-structure)
        (apply merge-with
               (fn [s1 s2]
-                (merge-with + s1 s2))
-              )))
+                (merge-with + s1 s2)))))
 
-(defn aggregate-avg-streams [avg counts]
+(defn aggregate-avg-streams
+  [avg counts]
   (let [expanded (expand-averages avg counts)]
     (->> expanded
          (map-val #(reduce add-pairs (vals %)))
-         (map-val val-avg)
-         )))
+         (map-val val-avg))))
 
-(defn aggregate-count-streams [stats]
+(defn aggregate-count-streams
+  [stats]
   (->> stats
        (map-val #(reduce + (vals %)))))
 
-(defn aggregate-common-stats [stats-seq]
+(defn aggregate-common-stats
+  [stats-seq]
   {:emitted (aggregate-counts (map #(.get_emitted ^ExecutorStats %) stats-seq))
-   :transferred (aggregate-counts (map #(.get_transferred ^ExecutorStats %) stats-seq))}
-  )
+   :transferred (aggregate-counts (map #(.get_transferred ^ExecutorStats %) stats-seq))})
 
-(defn mk-include-sys-fn [include-sys?]
+(defn mk-include-sys-fn
+  [include-sys?]
   (if include-sys?
     (fn [_] true)
     (fn [stream] (and (string? stream) (not (system-id? stream))))))
 
-(defn pre-process [stream-summary include-sys?]
+(defn pre-process
+  [stream-summary include-sys?]
   (let [filter-fn (mk-include-sys-fn include-sys?)
         emitted (:emitted stream-summary)
         emitted (into {} (for [[window stat] emitted]
@@ -346,7 +354,8 @@
         stream-summary (-> stream-summary (dissoc :transferred) (assoc :transferred transferred))]
     stream-summary))
 
-(defn aggregate-bolt-stats [stats-seq include-sys?]
+(defn aggregate-bolt-stats
+  [stats-seq include-sys?]
   (let [stats-seq (collectify stats-seq)]
     (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
            {:acked
@@ -367,10 +376,10 @@
             (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_execute_ms_avg)
                                      stats-seq)
                                 (map #(.. ^ExecutorStats % get_specific get_bolt get_executed)
-                                     stats-seq))
-            })))
+                                     stats-seq))})))
 
-(defn aggregate-spout-stats [stats-seq include-sys?]
+(defn aggregate-spout-stats
+  [stats-seq include-sys?]
   (let [stats-seq (collectify stats-seq)]
     (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
            {:acked
@@ -383,11 +392,10 @@
             (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_spout get_complete_ms_avg)
                                      stats-seq)
                                 (map #(.. ^ExecutorStats % get_specific get_spout get_acked)
-                                     stats-seq))
-            }
-           )))
+                                     stats-seq))})))
 
-(defn aggregate-bolt-streams [stats]
+(defn aggregate-bolt-streams
+  [stats]
   {:acked (aggregate-count-streams (:acked stats))
    :failed (aggregate-count-streams (:failed stats))
    :emitted (aggregate-count-streams (:emitted stats))
@@ -396,25 +404,27 @@
                                              (:acked stats))
    :executed (aggregate-count-streams (:executed stats))
    :execute-latencies (aggregate-avg-streams (:execute-latencies stats)
-                                             (:executed stats))
-   })
+                                             (:executed stats))})
 
-(defn aggregate-spout-streams [stats]
+(defn aggregate-spout-streams
+  [stats]
   {:acked (aggregate-count-streams (:acked stats))
    :failed (aggregate-count-streams (:failed stats))
    :emitted (aggregate-count-streams (:emitted stats))
    :transferred (aggregate-count-streams (:transferred stats))
    :complete-latencies (aggregate-avg-streams (:complete-latencies stats)
-                                              (:acked stats))
-   })
+                                              (:acked stats))})
 
-(defn spout-summary? [topology s]
+(defn spout-summary?
+  [topology s]
   (= :spout (executor-summary-type topology s)))
 
-(defn bolt-summary? [topology s]
+(defn bolt-summary?
+  [topology s]
   (= :bolt (executor-summary-type topology s)))
 
-(defn topology-summary-table [^TopologyInfo summ]
+(defn topology-summary-table
+  [^TopologyInfo summ]
   (let [executors (.get_executors summ)
         workers (set (for [^ExecutorSummary e executors] [(.get_host e) (.get_port e)]))]
     (table [{:text "Name" :attr {:class "tip right"
@@ -437,11 +447,10 @@
              (pretty-uptime-sec (.get_uptime_secs summ))
              (count workers)
              (count executors)
-             (sum-tasks executors)
-             ]]
-           )))
+             (sum-tasks executors)]])))
 
-(defn total-aggregate-stats [spout-summs bolt-summs include-sys?]
+(defn total-aggregate-stats
+  [spout-summs bolt-summs include-sys?]
   (let [spout-stats (get-filled-stats spout-summs)
         bolt-stats (get-filled-stats bolt-summs)
         agg-spout-stats (-> spout-stats
@@ -451,59 +460,60 @@
                            (aggregate-bolt-stats include-sys?)
                            aggregate-bolt-streams)]
     (merge-with
-     (fn [s1 s2]
-       (merge-with + s1 s2))
-     (select-keys agg-bolt-stats [:emitted :transferred])
-     agg-spout-stats
-     )))
+      (fn [s1 s2]
+        (merge-with + s1 s2))
+      (select-keys agg-bolt-stats [:emitted :transferred])
+      agg-spout-stats)))
 
-(defn stats-times [stats-map]
+(defn stats-times
+  [stats-map]
   (sort-by #(Integer/parseInt %)
            (-> stats-map
                clojurify-structure
                (dissoc ":all-time")
                keys)))
 
-(defn topology-stats-table [id window stats]
+(defn topology-stats-table
+  [id window stats]
   (let [times (stats-times (:emitted stats))
         display-map (into {} (for [t times] [t pretty-uptime-sec]))
         display-map (assoc display-map ":all-time" (fn [_] "All time"))]
     (sorted-table
-     [{:text "Window" :attr {:class "tip right"
-                             :title (:window tips)}}
-      {:text "Emitted" :attr {:class "tip above"
-                              :title (:emitted tips)}}
-      {:text "Transferred" :attr {:class "tip above"
-                                  :title (:transferred tips)}}
-      {:text "Complete latency (ms)" :attr {:class "tip above"
-                                            :title (:complete-lat tips)}}
-      {:text "Acked" :attr {:class "tip above"
-                            :title (:spout-acked tips)}}
-      {:text "Failed" :attr {:class "tip left"
-                            :title (:spout-failed tips)}}]
-     (for [k (concat times [":all-time"])
-           :let [disp ((display-map k) k)]]
-       [(link-to (if (= k window) {:class "red"} {})
-                 (url-format "/topology/%s?window=%s" id k)
-                 (escape-html disp))
-        (get-in stats [:emitted k])
-        (get-in stats [:transferred k])
-        (float-str (get-in stats [:complete-latencies k]))
-        (get-in stats [:acked k])
-        (get-in stats [:failed k])
-        ]
-       )
-     :time-cols [0]
-     )))
-
-(defn group-by-comp [summs]
+      [{:text "Window" :attr {:class "tip right"
+                              :title (:window tips)}}
+       {:text "Emitted" :attr {:class "tip above"
+                               :title (:emitted tips)}}
+       {:text "Transferred" :attr {:class "tip above"
+                                   :title (:transferred tips)}}
+       {:text "Complete latency (ms)" :attr {:class "tip above"
+                                             :title (:complete-lat tips)}}
+       {:text "Acked" :attr {:class "tip above"
+                             :title (:spout-acked tips)}}
+       {:text "Failed" :attr {:class "tip left"
+                              :title (:spout-failed tips)}}]
+      (for [k (concat times [":all-time"])
+            :let [disp ((display-map k) k)]]
+        [(link-to (if (= k window) {:class "red"} {})
+                  (url-format "/topology/%s?window=%s" id k)
+                  (escape-html disp))
+         (get-in stats [:emitted k])
+         (get-in stats [:transferred k])
+         (float-str (get-in stats [:complete-latencies k]))
+         (get-in stats [:acked k])
+         (get-in stats [:failed k])])
+      :time-cols [0])))
+
+(defn group-by-comp
+  [summs]
   (let [ret (group-by #(.get_component_id ^ExecutorSummary %) summs)]
     (into (sorted-map) ret )))
 
-(defn error-subset [error-str]
+(defn error-subset
+  [error-str]
   (apply str (take 200 error-str)))
 
-(defn most-recent-error [errors-list]
+(defn most-recent-error
+  [errors-list]
   (let [error (->> errors-list
                    (sort-by #(.get_error_time_secs ^ErrorInfo %))
                    reverse
@@ -513,24 +523,27 @@
                     (* 60 30))
                {:class "red"}
                {})
-       (error-subset (.get_error ^ErrorInfo error))]
-      )))
+       (error-subset (.get_error ^ErrorInfo error))])))
 
-(defn component-link [storm-id id]
+(defn component-link
+  [storm-id id]
   (link-to (url-format "/topology/%s/component/%s" storm-id id) (escape-html id)))
 
-(defn worker-log-link [host port]
+(defn worker-log-link
+  [host port]
   (link-to (url-format "http://%s:%s/log?file=worker-%s.log"
-              host (*STORM-CONF* LOGVIEWER-PORT) port) (str port)))
+                       host (*STORM-CONF* LOGVIEWER-PORT) port) (str port)))
 
-(defn render-capacity [capacity]
+(defn render-capacity
+  [capacity]
   (let [capacity (nil-to-zero capacity)]
     [:span (if (> capacity 0.9)
-                 {:class "red"}
-                 {})
-           (float-str capacity)]))
+             {:class "red"}
+             {})
+     (float-str capacity)]))
 
-(defn compute-executor-capacity [^ExecutorSummary e]
+(defn compute-executor-capacity
+  [^ExecutorSummary e]
   (let [stats (.get_stats e)
         stats (if stats
                 (-> stats
@@ -541,99 +554,95 @@
         uptime (nil-to-zero (.get_uptime_secs e))
         window (if (< uptime 600) uptime 600)
         executed (-> stats :executed nil-to-zero)
-        latency (-> stats :execute-latencies nil-to-zero)
-        ]
-   (if (> window 0)
-     (div (* executed latency) (* 1000 window))
-     )))
+        latency (-> stats :execute-latencies nil-to-zero)]
+    (if (> window 0)
+      (div (* executed latency) (* 1000 window)))))
 
-(defn compute-bolt-capacity [executors]
+(defn compute-bolt-capacity
+  [executors]
   (->> executors
        (map compute-executor-capacity)
        (map nil-to-zero)
        (apply max)))
 
-(defn spout-comp-table [top-id summ-map errors window include-sys?]
+(defn spout-comp-table
+  [top-id summ-map errors window include-sys?]
   (sorted-table
-   [{:text "Id" :attr {:class "tip right"
-                       :title (str (:comp-id tips) " " (:comp-id-link tips))}}
-    {:text "Executors" :attr {:class "tip right"
-                       :title (:num-execs tips)}}
-    {:text "Tasks" :attr {:class "tip above"
-                   :title (:num-tasks tips)}}
-    {:text "Emitted" :attr {:class "tip above"
-                     :title (:emitted tips)}}
-    {:text "Transferred" :attr {:class "tip above"
-                         :title (:transferred tips)}}
-    {:text "Complete latency (ms)" :attr {:class "tip above"
-                                   :title (:complete-lat tips)}}
-    {:text "Acked" :attr {:class "tip above"
-                          :title (:spout-acked tips)}}
-    {:text "Failed" :attr {:class "tip above"
-                           :title (:spout-failed tips)}}
-    "Last error"]
-   (for [[id summs] summ-map
-         :let [stats-seq (get-filled-stats summs)
-               stats (aggregate-spout-streams
-                      (aggregate-spout-stats
-                       stats-seq include-sys?))]]
-     [(component-link top-id id)
-      (count summs)
-      (sum-tasks summs)
-      (get-in stats [:emitted window])
-      (get-in stats [:transferred window])
-      (float-str (get-in stats [:complete-latencies window]))
-      (get-in stats [:acked window])
-      (get-in stats [:failed window])
-      (most-recent-error (get errors id))
-      ]
-     )))
-
-(defn bolt-comp-table [top-id summ-map errors window include-sys?]
+    [{:text "Id" :attr {:class "tip right"
+                        :title (str (:comp-id tips) " " (:comp-id-link tips))}}
+     {:text "Executors" :attr {:class "tip right"
+                               :title (:num-execs tips)}}
+     {:text "Tasks" :attr {:class "tip above"
+                           :title (:num-tasks tips)}}
+     {:text "Emitted" :attr {:class "tip above"
+                             :title (:emitted tips)}}
+     {:text "Transferred" :attr {:class "tip above"
+                                 :title (:transferred tips)}}
+     {:text "Complete latency (ms)" :attr {:class "tip above"
+                                           :title (:complete-lat tips)}}
+     {:text "Acked" :attr {:class "tip above"
+                           :title (:spout-acked tips)}}
+     {:text "Failed" :attr {:class "tip above"
+                            :title (:spout-failed tips)}}
+     "Last error"]
+    (for [[id summs] summ-map
+          :let [stats-seq (get-filled-stats summs)
+                stats (aggregate-spout-streams
+                        (aggregate-spout-stats
+                          stats-seq include-sys?))]]
+      [(component-link top-id id)
+       (count summs)
+       (sum-tasks summs)
+       (get-in stats [:emitted window])
+       (get-in stats [:transferred window])
+       (float-str (get-in stats [:complete-latencies window]))
+       (get-in stats [:acked window])
+       (get-in stats [:failed window])
+       (most-recent-error (get errors id))])))
+
+(defn bolt-comp-table
+  [top-id summ-map errors window include-sys?]
   (sorted-table
-   [{:text "Id" :attr {:class "tip right"
-                       :title (str (:comp-id tips) " " (:comp-id-link tips))}}
-    {:text "Executors" :attr {:class "tip right"
-                              :title (:num-execs tips)}}
-    {:text "Tasks" :attr {:class "tip above"
-                          :title (:num-tasks tips)}}
-    {:text "Emitted" :attr {:class "tip above"
-                            :title (:emitted tips)}}
-    {:text "Transferred" :attr {:class "tip above"
-                                :title (:transferred tips)}}
-    {:text "Capacity (last 10m)" :attr {:class "tip above"
-                                        :title (:capacity tips)}}
-    {:text "Execute latency (ms)" :attr {:class "tip above"
-                                         :title (:exec-lat tips)}}
-    {:text "Executed" :attr {:class "tip above"
-                             :title (:num-executed tips)}}
-    {:text "Process latency (ms)":attr {:class "tip above"
-                                        :title (:proc-lat tips)}}
-    {:text "Acked" :attr {:class "tip above"
-                          :title (:bolt-acked tips)}}
-    {:text "Failed" :attr {:class "tip left"
-                           :title (:bolt-failed tips)}}
-    "Last error"]
-   (for [[id summs] summ-map
-         :let [stats-seq (get-filled-stats summs)
-               stats (aggregate-bolt-streams
-                      (aggregate-bolt-stats
-                       stats-seq include-sys?))
-               ]]
-     [(component-link top-id id)
-      (count summs)
-      (sum-tasks summs)
-      (get-in stats [:emitted window])
-      (get-in stats [:transferred window])
-      (render-capacity (compute-bolt-capacity summs))
-      (float-str (get-in stats [:execute-latencies window]))
-      (get-in stats [:executed window])
-      (float-str (get-in stats [:process-latencies window]))
-      (get-in stats [:acked window])
-      (get-in stats [:failed window])
-      (most-recent-error (get errors id))
-      ]
-     )))
+    [{:text "Id" :attr {:class "tip right"
+                        :title (str (:comp-id tips) " " (:comp-id-link tips))}}
+     {:text "Executors" :attr {:class "tip right"
+                               :title (:num-execs tips)}}
+     {:text "Tasks" :attr {:class "tip above"
+                           :title (:num-tasks tips)}}
+     {:text "Emitted" :attr {:class "tip above"
+                             :title (:emitted tips)}}
+     {:text "Transferred" :attr {:class "tip above"
+                                 :title (:transferred tips)}}
+     {:text "Capacity (last 10m)" :attr {:class "tip above"
+                                         :title (:capacity tips)}}
+     {:text "Execute latency (ms)" :attr {:class "tip above"
+                                          :title (:exec-lat tips)}}
+     {:text "Executed" :attr {:class "tip above"
+                              :title (:num-executed tips)}}
+     {:text "Process latency (ms)":attr {:class "tip above"
+                                         :title (:proc-lat tips)}}
+     {:text "Acked" :attr {:class "tip above"
+                           :title (:bolt-acked tips)}}
+     {:text "Failed" :attr {:class "tip left"
+                            :title (:bolt-failed tips)}}
+     "Last error"]
+    (for [[id summs] summ-map
+          :let [stats-seq (get-filled-stats summs)
+                stats (aggregate-bolt-streams
+                        (aggregate-bolt-stats
+                          stats-seq include-sys?))]]
+      [(component-link top-id id)
+       (count summs)
+       (sum-tasks summs)
+       (get-in stats [:emitted window])
+       (get-in stats [:transferred window])
+       (render-capacity (compute-bolt-capacity summs))
+       (float-str (get-in stats [:execute-latencies window]))
+       (get-in stats [:executed window])
+       (float-str (get-in stats [:process-latencies window]))
+       (get-in stats [:acked window])
+       (get-in stats [:failed window])
+       (most-recent-error (get errors id))])))
 
 (defn window-hint [window]
   (if (= window ":all-time")
@@ -644,48 +653,48 @@
   [:input {:type "button"
            :value action
            (if enabled :enabled :disabled) ""
-           :onclick (str "confirmAction('" 
-                         (StringEscapeUtils/escapeJavaScript id) "', '" 
+           :onclick (str "confirmAction('"
+                         (StringEscapeUtils/escapeJavaScript id) "', '"
                          (StringEscapeUtils/escapeJavaScript name) "', '"
                          command "', " is-wait ", " default-wait ")")}])
 
-(defn topology-page [id window include-sys?]
+(defn topology-page
+  [id window include-sys?]
   (with-nimbus nimbus
-    (let [window (if window window ":all-time")
-          window-hint (window-hint window)
-          summ (.getTopologyInfo ^Nimbus$Client nimbus id)
-          topology (.getTopology ^Nimbus$Client nimbus id)
-          topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id))
-          spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
-          bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
-          spout-comp-summs (group-by-comp spout-summs)
-          bolt-comp-summs (group-by-comp bolt-summs)
-          bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs)
-          name (.get_name summ)
-          status (.get_status summ)
-          msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)
-          ]
-      (concat
-       [[:h2 "Topology summary"]]
-       [(topology-summary-table summ)]
-       [[:h2 {:class "js-only"} "Topology actions"]]
-       [[:p {:class "js-only"} (concat
-         [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))]
-         [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))]
-         [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))]
-         [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))]
-       )]]
-       [[:h2 "Topology stats"]]
-       (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
-       [[:h2 "Spouts (" window-hint ")"]]
-       (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?)
-       [[:h2 "Bolts (" window-hint ")"]]
-       (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?)
-       [[:h2 "Topology Configuration"]]
-       (configuration-table topology-conf)
-       ))))
-
-(defn component-task-summs [^TopologyInfo summ topology id]
+               (let [window (if window window ":all-time")
+                     window-hint (window-hint window)
+                     summ (.getTopologyInfo ^Nimbus$Client nimbus id)
+                     topology (.getTopology ^Nimbus$Client nimbus id)
+                     topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id))
+                     spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
+                     bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
+                     spout-comp-summs (group-by-comp spout-summs)
+                     bolt-comp-summs (group-by-comp bolt-summs)
+                     bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs)
+                     name (.get_name summ)
+                     status (.get_status summ)
+                     msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)]
+                 (concat
+                   [[:h2 "Topology summary"]]
+                   [(topology-summary-table summ)]
+                   [[:h2 {:class "js-only"} "Topology actions"]]
+                   [[:p {:class "js-only"}
+                     (concat
+                       [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))]
+                       [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))]
+                       [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))]
+                       [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))])]]
+                   [[:h2 "Topology stats"]]
+                   (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
+                   [[:h2 "Spouts (" window-hint ")"]]
+                   (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?)
+                   [[:h2 "Bolts (" window-hint ")"]]
+                   (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?)
+                   [[:h2 "Topology Configuration"]]
+                   (configuration-table topology-conf)))))
+
+(defn component-task-summs
+  [^TopologyInfo summ topology id]
   (let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
         bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
         spout-comp-summs (group-by-comp spout-summs)
@@ -693,320 +702,314 @@
         ret (if (contains? spout-comp-summs id)
               (spout-comp-summs id)
               (bolt-comp-summs id))]
-    (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret)
-    ))
+    (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret)))
 
-(defn spout-summary-table [topology-id id stats window]
+(defn spout-summary-table
+  [topology-id id stats window]
   (let [times (stats-times (:emitted stats))
         display-map (into {} (for [t times] [t pretty-uptime-sec]))
         display-map (assoc display-map ":all-time" (fn [_] "All time"))]
     (sorted-table
-     [{:text "Window" :attr {:class "tip right"
-                             :title (:window tips)}}
-      {:text "Emitted" :attr {:class "tip above"
-                              :title (:emitted tips)}}
-      {:text "Transferred" :attr {:class "tip above"
-                                  :title (:transferred tips)}}
-      {:text "Complete latency (ms)" :attr {:class "tip above"
-                                            :title (:complete-lat tips)}}
-      {:text "Acked" :attr {:class "tip above"
-                            :title (:spout-acked tips)}}
-      {:text "Failed" :attr {:class "tip left"
-                            :title (:spout-failed tips)}}]
-     (for [k (concat times [":all-time"])
-           :let [disp ((display-map k) k)]]
-       [(link-to (if (= k window) {:class "red"} {})
-                 (url-format "/topology/%s/component/%s?window=%s" topology-id id k)
-                 (escape-html disp))
-        (get-in stats [:emitted k])
-        (get-in stats [:transferred k])
-        (float-str (get-in stats [:complete-latencies k]))
-        (get-in stats [:acked k])
-        (get-in stats [:failed k])
-        ])
-     :time-cols [0])))
-
-(defn spout-output-summary-table [stream-summary window]
+      [{:text "Window" :attr {:class "tip right"
+                              :title (:window tips)}}
+       {:text "Emitted" :attr {:class "tip above"
+                               :title (:emitted tips)}}
+       {:text "Transferred" :attr {:class "tip above"
+                                   :title (:transferred tips)}}
+       {:text "Complete latency (ms)" :attr {:class "tip above"
+                                             :title (:complete-lat tips)}}
+       {:text "Acked" :attr {:class "tip above"
+                             :title (:spout-acked tips)}}
+       {:text "Failed" :attr {:class "tip left"
+                              :title (:spout-failed tips)}}]
+      (for [k (concat times [":all-time"])
+            :let [disp ((display-map k) k)]]
+        [(link-to (if (= k window) {:class "red"} {})
+                  (url-format "/topology/%s/component/%s?window=%s" topology-id id k)
+                  (escape-html disp))
+         (get-in stats [:emitted k])
+         (get-in stats [:transferred k])
+         (float-str (get-in stats [:complete-latencies k]))
+         (get-in stats [:acked k])
+         (get-in stats [:failed k])])
+      :time-cols [0])))
+
+(defn spout-output-summary-table
+  [stream-summary window]
   (let [stream-summary (map-val swap-map-order (swap-map-order stream-summary))]
     (sorted-table
-     [{:text "Stream" :attr {:class "tip right"
-                             :title (:stream tips)}}
-      {:text "Emitted" :attr {:class "tip above"
-                              :title (:emitted tips)}}
-      {:text "Transferred" :attr {:class "tip above"
-                                  :title (:transferred tips)}}
-      {:text "Complete latency (ms)" :attr {:class "tip above"
-                                            :title (:complete-lat tips)}}
-      {:text "Acked" :attr {:class "tip above"
-                            :title (:spout-acked tips)}}
-      {:text "Failed" :attr {:class "tip left"
-                            :title (:spout-failed tips)}}]
-     (for [[s stats] (stream-summary window)]
-       [s
-        (nil-to-zero (:emitted stats))
-        (nil-to-zero (:transferred stats))
-        (float-str (:complete-latencies stats))
-        (nil-to-zero (:acked stats))
-        (nil-to-zero (:failed stats))])
-     )))
-
-(defn spout-executor-table [topology-id executors window include-sys?]
+      [{:text "Stream" :attr {:class "tip right"
+                              :title (:stream tips)}}
+       {:text "Emitted" :attr {:class "tip above"
+                               :title (:emitted tips)}}
+       {:text "Transferred" :attr {:class "tip above"
+                                   :title (:transferred tips)}}
+       {:text "Complete latency (ms)" :attr {:class "tip above"
+                                             :title (:complete-lat tips)}}
+       {:text "Acked" :attr {:class "tip above"
+                             :title (:spout-acked tips)}}
+       {:text "Failed" :attr {:class "tip left"
+                              :title (:spout-failed tips)}}]
+      (for [[s stats] (stream-summary window)]
+        [s
+         (nil-to-zero (:emitted stats))
+         (nil-to-zero (:transferred stats))
+         (float-str (:complete-latencies stats))
+         (nil-to-zero (:acked stats))
+         (nil-to-zero (:failed stats))]))))
+
+(defn spout-executor-table
+  [topology-id executors window include-sys?]
   (sorted-table
-   [{:text "Id" :attr {:class "tip right"
-                       :title (:exec-id tips)}}
-    {:text "Uptime" :attr {:class "tip right"
-                           :title (:exec-uptime tips)}}
-    {:text "Host" :attr {:class "tip above"
-                         :title (:sup-host tips)}}
-    {:text "Port" :attr {:class "tip above"
-                         :title (:port tips)}}
-    {:text "Emitted" :attr {:class "tip above"
-                            :title (:emitted tips)}}
-    {:text "Transferred" :attr {:class "tip above"
-                                :title (:transferred tips)}}
-    {:text "Complete latency (ms)" :attr {:class "tip above"
-                                          :title (:complete-lat tips)}}
-    {:text "Acked" :attr {:class "tip above"
-                          :title (:spout-acked tips)}}
-    {:text "Failed" :attr {:class "tip left"
-                          :title (:spout-failed tips)}}]
-   (for [^ExecutorSummary e executors
-         :let [stats (.get_stats e)
-               stats (if stats
-                       (-> stats
-                           (aggregate-spout-stats include-sys?)
-                           aggregate-spout-streams
-                           swap-map-order
-                           (get window)))]]
-     [(pretty-executor-info (.get_executor_info e))
-      (pretty-uptime-sec (.get_uptime_secs e))
-      (.get_host e)
-      (worker-log-link (.get_host e) (.get_port e))
-      (nil-to-zero (:emitted stats))
-      (nil-to-zero (:transferred stats))
-      (float-str (:complete-latencies stats))
-      (nil-to-zero (:acked stats))
-      (nil-to-zero (:failed stats))
-      ]
-     )
-   :time-cols [1]
-   ))
-
-(defn spout-page [window ^TopologyInfo topology-info component executors include-sys?]
+    [{:text "Id" :attr {:class "tip right"
+                        :title (:exec-id tips)}}
+     {:text "Uptime" :attr {:class "tip right"
+                            :title (:exec-uptime tips)}}
+     {:text "Host" :attr {:class "tip above"
+                          :title (:sup-host tips)}}
+     {:text "Port" :attr {:class "tip above"
+                          :title (:port tips)}}
+     {:text "Emitted" :attr {:class "tip above"
+                             :title (:emitted tips)}}
+     {:text "Transferred" :attr {:class "tip above"
+                                 :title (:transferred tips)}}
+     {:text "Complete latency (ms)" :attr {:class "tip above"
+                                           :title (:complete-lat tips)}}
+     {:text "Acked" :attr {:class "tip above"
+                           :title (:spout-acked tips)}}
+     {:text "Failed" :attr {:class "tip left"
+                            :title (:spout-failed tips)}}]
+    (for [^ExecutorSummary e executors
+          :let [stats (.get_stats e)
+                stats (if stats
+                        (-> stats
+                            (aggregate-spout-stats include-sys?)
+                            aggregate-spout-streams
+                            swap-map-order
+                            (get window)))]]
+      [(pretty-executor-info (.get_executor_info e))
+       (pretty-uptime-sec (.get_uptime_secs e))
+       (.get_host e)
+       (worker-log-link (.get_host e) (.get_port e))
+       (nil-to-zero (:emitted stats))
+       (nil-to-zero (:transferred stats))
+       (float-str (:complete-latencies stats))
+       (nil-to-zero (:acked stats))
+       (nil-to-zero (:failed stats))])
+    :time-cols [1]))
+
+(defn spout-page
+  [window ^TopologyInfo topology-info component executors include-sys?]
   (let [window-hint (str " (" (window-hint window) ")")
         stats (get-filled-stats executors)
         stream-summary (-> stats (aggregate-spout-stats include-sys?))
         summary (-> stream-summary aggregate-spout-streams)]
     (concat
-     [[:h2 "Spout stats"]]
-     (spout-summary-table (.get_id topology-info) component summary window)
-     [[:h2 "Output stats" window-hint]]
-     (spout-output-summary-table stream-summary window)
-     [[:h2 "Executors" window-hint]]
-     (spout-executor-table (.get_id topology-info) executors window include-sys?)
-     ;; task id, task uptime, stream aggregated stats, last error
-     )))
-
-(defn bolt-output-summary-table [stream-summary window]
+      [[:h2 "Spout stats"]]
+      (spout-summary-table (.get_id topology-info) component summary window)
+      [[:h2 "Output stats" window-hint]]
+      (spout-output-summary-table stream-summary window)
+      [[:h2 "Executors" window-hint]]
+      ;; task id, task uptime, stream aggregated stats, last error
+      (spout-executor-table (.get_id topology-info) executors window include-sys?))))
+
+(defn bolt-output-summary-table
+  [stream-summary window]
   (let [stream-summary (-> stream-summary
                            swap-map-order
                            (get window)
                            (select-keys [:emitted :transferred])
                            swap-map-order)]
     (sorted-table
-     [{:text "Stream" :attr {:class "tip right"
-                             :title (:stream tips)}}
-      {:text "Emitted" :attr {:class "tip above"
-                              :title (:emitted tips)}}
-      {:text "Transferred" :attr {:class "tip above"
-                                  :title (:transferred tips)}}]
-     (for [[s stats] stream-summary]
-       [s
-        (nil-to-zero (:emitted stats))
-        (nil-to-zero (:transferred stats))
-        ])
-     )))
-
-(defn bolt-input-summary-table [stream-summary window]
+      [{:text "Stream" :attr {:class "tip right"
+                              :title (:stream tips)}}
+       {:text "Emitted" :attr {:class "tip above"
+                               :title (:emitted tips)}}
+       {:text "Transferred" :attr {:class "tip above"
+                                   :title (:transferred tips)}}]
+      (for [[s stats] stream-summary]
+        [s
+         (nil-to-zero (:emitted stats))
+         (nil-to-zero (:transferred stats))]))))
+
+(defn bolt-input-summary-table
+  [stream-summary window]
   (let [stream-summary (-> stream-summary
                            swap-map-order
                            (get window)
                            (select-keys [:acked :failed :process-latencies :executed :execute-latencies])
                            swap-map-order)]
     (sorted-table
-     [{:text "Component" :attr {:class "tip right"
-                         :title (:comp-id tips)}}
-      {:text "Stream" :attr {:class "tip right"
-                             :title (:stream tips)}}
-      {:text "Execute latency (ms)" :attr {:class "tip above"
-                                           :title (:exec-lat tips)}}
-      {:text "Executed" :attr {:class "tip above"
-                               :title (:num-executed tips)}}
-      {:text "Process latency (ms)":attr {:class "tip above"
-                                          :title (:proc-lat tips)}}
-      {:text "Acked" :attr {:class "tip above"
-                            :title (:bolt-acked tips)}}
-      {:text "Failed" :attr {:class "tip left"
-                             :title (:bolt-failed tips)}}]
-     (for [[^GlobalStreamId s stats] stream-summary]
-       [(escape-html (.get_componentId s))
-        (.get_streamId s)
-        (float-str (:execute-latencies stats))
-        (nil-to-zero (:executed stats))
-        (float-str (:process-latencies stats))
-        (nil-to-zero (:acked stats))
-        (nil-to-zero (:failed stats))
-        ])
-     )))
-
-(defn bolt-executor-table [topology-id executors window include-sys?]
+      [{:text "Component" :attr {:class "tip right"
+                                 :title (:comp-id tips)}}
+       {:text "Stream" :attr {:class "tip right"
+                              :title (:stream tips)}}
+       {:text "Execute latency (ms)" :attr {:class "tip above"
+                                            :title (:exec-lat tips)}}
+       {:text "Executed" :attr {:class "tip above"
+                                :title (:num-executed tips)}}
+       {:text "Process latency (ms)":attr {:class "tip above"
+                                           :title (:proc-lat tips)}}
+       {:text "Acked" :attr {:class "tip above"
+                             :title (:bolt-acked tips)}}
+       {:text "Failed" :attr {:class "tip left"
+                              :title (:bolt-failed tips)}}]
+      (for [[^GlobalStreamId s stats] stream-summary]
+        [(escape-html (.get_componentId s))
+         (.get_streamId s)
+         (float-str (:execute-latencies stats))
+         (nil-to-zero (:executed stats))
+         (float-str (:process-latencies stats))
+         (nil-to-zero (:acked stats))
+         (nil-to-zero (:failed stats))]))))
+
+(defn bolt-executor-table
+  [topology-id executors window include-sys?]
   (sorted-table
-   [{:text "Id" :attr {:class "tip right"
-                       :title (:exec-id tips)}}
-    {:text "Uptime" :attr {:class "tip right"
-                           :title (:exec-uptime tips)}}
-    {:text "Host" :attr {:class "tip above"
-                         :title (:sup-host tips)}}
-    {:text "Port" :attr {:class "tip above"
-                         :title (:port tips)}}
-    {:text "Emitted" :attr {:class "tip above"
-                            :title (:emitted tips)}}
-    {:text "Transferred" :attr {:class "tip above"
-                                :title (:transferred tips)}}
-    {:text "Capacity (last 10m)" :attr {:class "tip above"
-                                        :title (:capacity tips)}}
-    {:text "Execute latency (ms)" :attr {:class "tip above"
-                                         :title (:exec-lat tips)}}
-    {:text "Executed" :attr {:class "tip above"
-                             :title (:num-executed tips)}}
-    {:text "Process latency (ms)":attr {:class "tip above"
-                                        :title (:proc-lat tips)}}
-    {:text "Acked" :attr {:class "tip above"
-                          :title (:bolt-acked tips)}}
-    {:text "Failed" :attr {:class "tip left"
-                           :title (:bolt-failed tips)}}]
-   (for [^ExecutorSummary e executors
-         :let [stats (.get_stats e)
-               stats (if stats
-                       (-> stats
-                           (aggregate-bolt-stats include-sys?)
-                           (aggregate-bolt-streams)
-                           swap-map-order
-                           (get window)))]]
-     [(pretty-executor-info (.get_executor_info e))
-      (pretty-uptime-sec (.get_uptime_secs e))
-      (.get_host e)
-      (worker-log-link (.get_host e) (.get_port e))
-      (nil-to-zero (:emitted stats))
-      (nil-to-zero (:transferred stats))
-      (render-capacity (compute-executor-capacity e))
-      (float-str (:execute-latencies stats))
-      (nil-to-zero (:executed stats))
-      (float-str (:process-latencies stats))
-      (nil-to-zero (:acked stats))
-      (nil-to-zero (:failed stats))
-      ]
-     )
-   :time-cols [1]
-   ))
-
-(defn bolt-summary-table [topology-id id stats window]
+    [{:text "Id" :attr {:class "tip right"
+                        :title (:exec-id tips)}}
+     {:text "Uptime" :attr {:class "tip right"
+                            :title (:exec-uptime tips)}}
+     {:text "Host" :attr {:class "tip above"
+                          :title (:sup-host tips)}}
+     {:text "Port" :attr {:class "tip above"
+                          :title (:port tips)}}
+     {:text "Emitted" :attr {:class "tip above"
+                             :title (:emitted tips)}}
+     {:text "Transferred" :attr {:class "tip above"
+                                 :title (:transferred tips)}}
+     {:text "Capacity (last 10m)" :attr {:class "tip above"
+                                         :title (:capacity tips)}}
+     {:text "Execute latency (ms)" :attr {:class "tip above"
+                                          :title (:exec-lat tips)}}
+     {:text "Executed" :attr {:class "tip above"
+                              :title (:num-executed tips)}}
+     {:text "Process latency (ms)":attr {:class "tip above"
+                                         :title (:proc-lat tips)}}
+     {:text "Acked" :attr {:class "tip above"
+                           :title (:bolt-acked tips)}}
+     {:text "Failed" :attr {:class "tip left"
+                            :title (:bolt-failed tips)}}]
+    (for [^ExecutorSummary e executors
+          :let [stats (.get_stats e)
+                stats (if stats
+                        (-> stats
+                            (aggregate-bolt-stats include-sys?)
+                            (aggregate-bolt-streams)
+                            swap-map-order
+                            (get window)))]]
+      [(pretty-executor-info (.get_executor_info e))
+       (pretty-uptime-sec (.get_uptime_secs e))
+       (.get_host e)
+       (worker-log-link (.get_host e) (.get_port e))
+       (nil-to-zero (:emitted stats))
+       (nil-to-zero (:transferred stats))
+       (render-capacity (compute-executor-capacity e))
+       (float-str (:execute-latencies stats))
+       (nil-to-zero (:executed stats))
+       (float-str (:process-latencies stats))
+       (nil-to-zero (:acked stats))
+       (nil-to-zero (:failed stats))])
+    :time-cols [1]))
+
+(defn bolt-summary-table
+  [topology-id id stats window]
   (let [times (stats-times (:emitted stats))
         display-map (into {} (for [t times] [t pretty-uptime-sec]))
         display-map (assoc display-map ":all-time" (fn [_] "All time"))]
     (sorted-table
-     [{:text "Window" :attr {:class "tip right"
-                             :title (:window tips)}}
-      {:text "Emitted" :attr {:class "tip above"
-                              :title (:emitted tips)}}
-      {:text "Transferred" :attr {:class "tip above"
-                                  :title (:transferred tips)}}
-      {:text "Execute latency (ms)" :attr {:class "tip above"
-                                           :title (:exec-lat tips)}}
-      {:text "Executed" :attr {:class "tip above"
-                               :title (:num-executed tips)}}
-      {:text "Process latency (ms)":attr {:class "tip above"
-                                          :title (:proc-lat tips)}}
-      {:text "Acked" :attr {:class "tip above"
-                            :title (:bolt-acked tips)}}
-      {:text "Failed" :attr {:class "tip left"
-                             :title (:bolt-failed tips)}}]
-     (for [k (concat times [":all-time"])
-           :let [disp ((display-map k) k)]]
-       [(link-to (if (= k window) {:class "red"} {})
-                 (url-format "/topology/%s/component/%s?window=%s" topology-id id k)
-                 (escape-html disp))
-        (get-in stats [:emitted k])
-        (get-in stats [:transferred k])
-        (float-str (get-in stats [:execute-latencies k]))
-        (get-in stats [:executed k])
-        (float-str (get-in stats [:process-latencies k]))
-        (get-in stats [:acked k])
-        (get-in stats [:failed k])
-        ])
-     :time-cols [0])))
-
-(defn bolt-page [window ^TopologyInfo topology-info component executors include-sys?]
+      [{:text "Window" :attr {:class "tip right"
+                              :title (:window tips)}}
+       {:text "Emitted" :attr {:class "tip above"
+                               :title (:emitted tips)}}
+       {:text "Transferred" :attr {:class "tip above"
+                                   :title (:transferred tips)}}
+       {:text "Execute latency (ms)" :attr {:class "tip above"
+                                            :title (:exec-lat tips)}}
+       {:text "Executed" :attr {:class "tip above"
+                                :title (:num-executed tips)}}
+       {:text "Process latency (ms)":attr {:class "tip above"
+                                           :title (:proc-lat tips)}}
+       {:text "Acked" :attr {:class "tip above"
+                             :title (:bolt-acked tips)}}
+       {:text "Failed" :attr {:class "tip left"
+                              :title (:bolt-failed tips)}}]
+      (for [k (concat times [":all-time"])
+            :let [disp ((display-map k) k)]]
+        [(link-to (if (= k window) {:class "red"} {})
+                  (url-format "/topology/%s/component/%s?window=%s" topology-id id k)
+                  (escape-html disp))
+         (get-in stats [:emitted k])
+         (get-in stats [:transferred k])
+         (float-str (get-in stats [:execute-latencies k]))
+         (get-in stats [:executed k])
+         (float-str (get-in stats [:process-latencies k]))
+         (get-in stats [:acked k])
+         (get-in stats [:failed k])])
+      :time-cols [0])))
+
+(defn bolt-page
+  [window ^TopologyInfo topology-info component executors include-sys?]
   (let [window-hint (str " (" (window-hint window) ")")
         stats (get-filled-stats executors)
         stream-summary (-> stats (aggregate-bolt-stats include-sys?))
         summary (-> stream-summary aggregate-bolt-streams)]
     (concat
-     [[:h2 "Bolt stats"]]
-     (bolt-summary-table (.get_id topology-info) component summary window)
+      [[:h2 "Bolt stats"]]
+      (bolt-summary-table (.get_id topology-info) component summary window)
 
-     [[:h2 "Input stats" window-hint]]
-     (bolt-input-summary-table stream-summary window)
+      [[:h2 "Input stats" window-hint]]
+      (bolt-input-summary-table stream-summary window)
 
-     [[:h2 "Output stats" window-hint]]
-     (bolt-output-summary-table stream-summary window)
+      [[:h2 "Output stats" window-hint]]
+      (bolt-output-summary-table stream-summary window)
 
-     [[:h2 "Executors"]]
-     (bolt-executor-table (.get_id topology-info) executors window include-sys?)
-     )))
+      [[:h2 "Executors"]]
+      (bolt-executor-table (.get_id topology-info) executors window include-sys?))))
 
-(defn errors-table [errors-list]
+(defn errors-table
+  [errors-list]
   (let [errors (->> errors-list
                     (sort-by #(.get_error_time_secs ^ErrorInfo %))
                     reverse)]
     (sorted-table
-     ["Time" "Error"]
-     (for [^ErrorInfo e errors]
-       [(date-str (.get_error_time_secs e))
-        [:pre (.get_error e)]])
-     :sort-list "[[0,1]]"
-     )))
-
-(defn component-page [topology-id component window include-sys?]
+      ["Time" "Error"]
+      (for [^ErrorInfo e errors]
+        [(date-str (.get_error_time_secs e))
+         [:pre (.get_error e)]])
+      :sort-list "[[0,1]]")))
+
+(defn component-page
+  [topology-id component window include-sys?]
   (with-nimbus nimbus
-    (let [window (if window window ":all-time")
-          summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
-          topology (.getTopology ^Nimbus$Client nimbus topology-id)
-          type (component-type topology component)
-          summs (component-task-summs summ topology component)
-          spec (cond (= type :spout) (spout-page window summ component summs include-sys?)
-                     (= type :bolt) (bolt-page window summ component summs include-sys?))]
-      (concat
-       [[:h2 "Component summary"]
-        (table [{:text "Id" :attr {:class "tip right"
-                                   :title (:comp-id tips)}}
-                {:text "Topology" :attr {:class "tip above"
-                                   :title (str (:name tips) " " (:name-link tips))}}
-                {:text "Executors" :attr {:class "tip above"
-                                   :title (:num-execs tips)}}
-                {:text "Tasks" :attr {:class "tip above"
-                               :title (:num-tasks tips)}}]
-               [[(escape-html component)
-                 (topology-link (.get_id summ) (.get_name summ))
-                 (count summs)
-                 (sum-tasks summs)
-                 ]])]
-       spec
-       [[:h2 "Errors"]
-        (errors-table (get (.get_errors summ) component))]
-       ))))
-
-(defn get-include-sys? [cookies]
+               (let [window (if window window ":all-time")
+                     summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
+                     topology (.getTopology ^Nimbus$Client nimbus topology-id)
+                     type (component-type topology component)
+                     summs (component-task-summs summ topology component)
+                     spec (cond (= type :spout) (spout-page window summ component summs include-sys?)
+                                (= type :bolt) (bolt-page window summ component summs include-sys?))]
+                 (concat
+                   [[:h2 "Component summary"]
+                    (table [{:text "Id" :attr {:class "tip right"
+                                               :title (:comp-id tips)}}
+                            {:text "Topology" :attr {:class "tip above"
+                                                     :title (str (:name tips) " " (:name-link tips))}}
+                            {:text "Executors" :attr {:class "tip above"
+                                                      :title (:num-execs tips)}}
+                            {:text "Tasks" :attr {:class "tip above"
+                                                  :title (:num-tasks tips)}}]
+                           [[(escape-html component)
+                             (topology-link (.get_id summ) (.get_name summ))
+                             (count summs)
+                             (sum-tasks summs)
+                             ]])]
+                   spec
+                   [[:h2 "Errors"]
+                    (errors-table (get (.get_errors summ) component))]))))
+
+(defn get-include-sys?
+  [cookies]
   (let [sys? (get cookies "sys")
         sys? (if (or (nil? sys?) (= "false" (:value sys?))) false true)]
     sys?))
@@ -1017,81 +1020,84 @@
            ui-template))
   (GET "/topology/:id" [:as {cookies :cookies} id & m]
        (let [include-sys? (get-include-sys? cookies)
-            id (url-decode id)]
+             id (url-decode id)]
          (try
            (-> (topology-page (url-decode id) (:window m) include-sys?)
-             (concat [(mk-system-toggle-button include-sys?)])
-             ui-template)
+               (concat [(mk-system-toggle-button include-sys?)])
+               ui-template)
            (catch Exception e (resp/redirect "/")))))
   (GET "/topology/:id/component/:component" [:as {cookies :cookies} id component & m]
        (let [include-sys? (get-include-sys? cookies)
-            id (url-decode id)
-            component (url-decode component)]
+             id (url-decode id)
+             component (url-decode component)]
          (-> (component-page id component (:window m) include-sys?)
              (concat [(mk-system-toggle-button include-sys?)])
              ui-template)))
   (POST "/topology/:id/activate" [id]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)]
-        (.activate nimbus name)
-        (log-message "Activating topology '" name "'")))
-    (resp/redirect (str "/topology/" id)))
+        (with-nimbus nimbus
+                     (let [id (url-decode id)
+                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+                           name (.get_name tplg)]
+                       (.activate nimbus name)
+                       (log-message "Activating topology '" name "'")))
+        (resp/redirect (str "/topology/" id)))
   (POST "/topology/:id/deactivate" [id]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)]
-        (.deactivate nimbus name)
-        (log-message "Deactivating topology '" name "'")))
-    (resp/redirect (str "/topology/" id)))
+        (with-nimbus nimbus
+                     (let [id (url-decode id)
+                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+                           name (.get_name tplg)]
+                       (.deactivate nimbus name)
+                       (log-message "Deactivating topology '" name "'")))
+        (resp/redirect (str "/topology/" id)))
   (POST "/topology/:id/rebalance/:wait-time" [id wait-time]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)
-            options (RebalanceOptions.)]
-        (.set_wait_secs options (Integer/parseInt wait-time))
-        (.rebalance nimbus name options)
-        (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
-    (resp/redirect (str "/topology/" id)))
+        (with-nimbus nimbus
+                     (let [id (url-decode id)
+                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+                           name (.get_name tplg)
+                           options (RebalanceOptions.)]
+                       (.set_wait_secs options (Integer/parseInt wait-time))
+                       (.rebalance nimbus name options)
+                       (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
+        (resp/redirect (str "/topology/" id)))
   (POST "/topology/:id/kill/:wait-time" [id wait-time]
-    (with-nimbus nimbus
-      (let [id (url-decode id)
-            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-            name (.get_name tplg)
-            options (KillOptions.)]
-        (.set_wait_secs options (Integer/parseInt wait-time))
-        (.killTopologyWithOpts nimbus name options)
-        (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
-    (resp/redirect (str "/topology/" id)))
+        (with-nimbus nimbus
+                     (let [id (url-decode id)
+                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+                           name (.get_name tplg)
+                           options (KillOptions.)]
+                       (.set_wait_secs options (Integer/parseInt wait-time))
+                       (.killTopologyWithOpts nimbus name options)
+                       (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
+        (resp/redirect (str "/topology/" id)))
   (route/resources "/")
   (route/not-found "Page not found"))
 
-(defn exception->html [ex]
+(defn exception->html
+  [ex]
   (concat
     [[:h2 "Internal Server Error"]]
     [[:pre (let [sw (java.io.StringWriter.)]
-      (.printStackTrace ex (java.io.PrintWriter. sw))
-      (.toString sw))]]))
+             (.printStackTrace ex (java.io.PrintWriter. sw))
+             (.toString sw))]]))
 
-(defn catch-errors [handler]
+(defn catch-errors
+  [handler]
   (fn [request]
     (try
       (handler request)
       (catch Exception ex
         (-> (resp/response (ui-template (exception->html ex)))
-          (resp/status 500)
-          (resp/content-type "text/html"))
-        ))))
+            (resp/status 500)
+            (resp/content-type "text/html"))))))
 
 (def app
   (handler/site (-> main-routes
                     (wrap-reload '[backtype.storm.ui.core])
                     catch-errors)))
 
-(defn start-server! [] (run-jetty app {:port (Integer. (*STORM-CONF* UI-PORT))
-                                       :join? false}))
+(defn start-server!
+  []
+  (run-jetty app {:port (Integer. (*STORM-CONF* UI-PORT))
+                  :join? false}))
 
 (defn -main [] (start-server!))


[10/13] git commit: Merge branch 'master' into idiomatic-clojure-01

Posted by bo...@apache.org.
Merge branch 'master' into idiomatic-clojure-01

Conflicts:
	storm-core/src/clj/backtype/storm/disruptor.clj
	storm-core/src/clj/backtype/storm/timer.clj
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/src/clj/backtype/storm/util.clj

Results :

Tests run: 35, Failures: 0, Errors: 0, Skipped: 0

[INFO] ------------------------------------------------------------------------
[INFO] Reactor Summary:
[INFO] 
[INFO] Storm ............................................. SUCCESS [0.612s]
[INFO] maven-shade-clojure-transformer ................... SUCCESS [1.538s]
[INFO] Storm Core ........................................ SUCCESS [2:31.494s]
[INFO] storm-starter ..................................... SUCCESS [6.167s]
[INFO] storm-kafka ....................................... SUCCESS [28.252s]
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time: 3:08.652s
[INFO] Finished at: Mon Jun 09 20:53:23 PDT 2014
[INFO] Final Memory: 41M/694M
[INFO] ------------------------------------------------------------------------

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

Branch: refs/heads/master
Commit: 55e1664d40c87047dc1e9f322726aeb1a77ea344
Parents: 3352dc2
Author: David James <da...@bluemontlabs.com>
Authored: Mon Jun 9 20:47:54 2014 -0700
Committer: David James <da...@bluemontlabs.com>
Committed: Mon Jun 9 20:54:24 2014 -0700

----------------------------------------------------------------------
 .gitignore                                      |    4 +-
 CHANGELOG.md                                    |    7 +
 conf/defaults.yaml                              |    9 +
 .../src/clj/backtype/storm/daemon/drpc.clj      |  140 +-
 .../src/clj/backtype/storm/daemon/executor.clj  |    1 +
 .../src/clj/backtype/storm/daemon/logviewer.clj |   12 -
 .../src/clj/backtype/storm/daemon/worker.clj    |   67 +-
 storm-core/src/clj/backtype/storm/disruptor.clj |   13 +-
 .../src/clj/backtype/storm/messaging/loader.clj |   81 +-
 .../src/clj/backtype/storm/messaging/local.clj  |   20 +-
 storm-core/src/clj/backtype/storm/timer.clj     |   49 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   | 1324 ++++++++----------
 .../src/clj/backtype/storm/ui/helpers.clj       |    1 -
 storm-core/src/clj/backtype/storm/util.clj      |   19 +-
 storm-core/src/jvm/backtype/storm/Config.java   |   19 +
 .../backtype/storm/messaging/IConnection.java   |   15 +-
 .../backtype/storm/messaging/netty/Client.java  |  388 ++---
 .../backtype/storm/messaging/netty/Context.java |   41 +-
 .../storm/messaging/netty/ControlMessage.java   |    6 +-
 .../storm/messaging/netty/MessageBatch.java     |    5 +
 .../storm/messaging/netty/MessageDecoder.java   |  108 +-
 .../netty/NettyRenameThreadFactory.java         |   35 +
 .../backtype/storm/messaging/netty/Server.java  |  145 +-
 .../netty/StormClientErrorHandler.java          |   41 +
 .../messaging/netty/StormClientHandler.java     |   87 --
 .../netty/StormClientPipelineFactory.java       |    2 +-
 .../messaging/netty/StormServerHandler.java     |   40 +-
 .../storm/testing/TestEventLogSpout.java        |  139 ++
 .../storm/testing/TestEventOrderCheckBolt.java  |   76 +
 .../backtype/storm/utils/DisruptorQueue.java    |    9 +-
 .../backtype/storm/utils/TransferDrainer.java   |  113 ++
 .../src/jvm/backtype/storm/utils/Utils.java     |   46 +-
 storm-core/src/ui/public/component.html         |   88 ++
 storm-core/src/ui/public/index.html             |   73 +
 storm-core/src/ui/public/js/arbor-graphics.js   |   51 +
 storm-core/src/ui/public/js/arbor-tween.js      |   86 ++
 storm-core/src/ui/public/js/arbor.js            |   67 +
 storm-core/src/ui/public/js/jquery.mustache.js  |  592 ++++++++
 storm-core/src/ui/public/js/purl.js             |  267 ++++
 storm-core/src/ui/public/js/script.js           |   51 +-
 storm-core/src/ui/public/js/visualization.js    |  403 ++++++
 .../templates/component-page-template.html      |  152 ++
 .../public/templates/index-page-template.html   |   62 +
 .../public/templates/json-error-template.html   |    4 +
 .../templates/topology-page-template.html       |  128 ++
 storm-core/src/ui/public/topology.html          |   90 ++
 .../test/clj/backtype/storm/drpc_test.clj       |   27 +-
 .../storm/messaging/netty_unit_test.clj         |   46 +-
 .../test/clj/backtype/storm/messaging_test.clj  |   35 +-
 49 files changed, 3956 insertions(+), 1328 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index b2a37f9..b575a02 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,4 +27,6 @@ target
 /.lein-plugins/
 *.ipr
 *.iws
-.idea
\ No newline at end of file
+.idea
+.*
+!/.gitignore

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 134d009..0fa35e7 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,11 @@
 ## 0.9.2-incubating (unreleased)
+ * STORM-335: add drpc test for removing timed out requests from queue
+ * STORM-69: Storm UI Visualizations for Topologies
+ * STORM-297: Performance scaling with CPU
+ * STORM-244: DRPC timeout can return null instead of throwing an exception
+ * STORM-63: remove timeout drpc request from its function's request queue
+ * STORM-313: Remove log-level-page from logviewer
+ * STORM-205: Add REST API To Storm UI
  * STORM-326: tasks send duplicate metrics
  * STORM-331: Update the Kafka dependency of storm-kafka to 0.8.1.1
  * STORM-308: Add support for config_value to {supervisor,nimbus,ui,drpc,logviewer} childopts

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index bb1cb04..39c4b92 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -94,6 +94,9 @@ supervisor.enable: true
 worker.childopts: "-Xmx768m"
 worker.heartbeat.frequency.secs: 1
 
+# control how many worker receiver threads we need per worker 
+topology.worker.receiver.thread.count: 1
+
 task.heartbeat.frequency.secs: 3
 task.refresh.poll.secs: 10
 
@@ -109,6 +112,12 @@ storm.messaging.netty.max_retries: 30
 storm.messaging.netty.max_wait_ms: 1000
 storm.messaging.netty.min_wait_ms: 100
 
+# If the Netty messaging layer is busy(netty internal buffer not writable), the Netty client will try to batch message as more as possible up to the size of storm.messaging.netty.transfer.batch.size bytes, otherwise it will try to flush message as soon as possible to reduce latency.
+storm.messaging.netty.transfer.batch.size: 262144
+
+# We check with this interval that whether the Netty channel is writable and try to write pending messages if it is.
+storm.messaging.netty.flush.check.interval.ms: 10
+
 ### topology.* configs are for specific executing storms
 topology.enable.message.timeouts: true
 topology.debug: false

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/daemon/drpc.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
index df07343..eb4d6d3 100644
--- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
@@ -13,15 +13,20 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.daemon.drpc
   (: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 [backtype.storm.generated DistributedRPC DistributedRPC$Iface DistributedRPC$Processor
-            DRPCRequest DRPCExecutionException DistributedRPCInvocations DistributedRPCInvocations$Iface
+  (:import [org.apache.thrift.transport
+            TNonblockingServerTransport TNonblockingServerSocket])
+  (:import [backtype.storm.generated
+            DistributedRPC DistributedRPC$Iface DistributedRPC$Processor
+            DRPCRequest DRPCExecutionException
+            DistributedRPCInvocations DistributedRPCInvocations$Iface
             DistributedRPCInvocations$Processor])
-  (:import [java.util.concurrent Semaphore ConcurrentLinkedQueue ThreadPoolExecutor ArrayBlockingQueue TimeUnit])
+  (:import [java.util.concurrent Semaphore ConcurrentLinkedQueue
+            ThreadPoolExecutor ArrayBlockingQueue TimeUnit])
   (:import [backtype.storm.daemon Shutdownable])
   (:import [java.net InetAddress])
   (:use [backtype.storm bootstrap config log])
@@ -29,15 +34,14 @@
 
 (bootstrap)
 
-(def TIMEOUT-CHECK-SECS 5)
+(defn timeout-check-secs [] 5)
 
 (defn acquire-queue [queues-atom function]
   (swap! queues-atom
     (fn [amap]
       (if-not (amap function)
         (assoc amap function (ConcurrentLinkedQueue.))
-        amap)
-        ))
+        amap)))
   (@queues-atom function))
 
 ;; TODO: change this to use TimeCacheMap
@@ -47,33 +51,38 @@
         id->sem (atom {})
         id->result (atom {})
         id->start (atom {})
+        id->function (atom {})
+        id->request (atom {})
         request-queues (atom {})
         cleanup (fn [id] (swap! id->sem dissoc id)
-                         (swap! id->result dissoc id)
-                         (swap! id->start dissoc id))
+                  (swap! id->result dissoc id)
+                  (swap! id->function dissoc id)
+                  (swap! id->request dissoc id)
+                  (swap! id->start dissoc id))
         my-ip (.getHostAddress (InetAddress/getLocalHost))
         clear-thread (async-loop
-                      (fn []
-                        (doseq [[id start] @id->start]
-                          (when (> (time-delta start) (conf DRPC-REQUEST-TIMEOUT-SECS))
-                            (when-let [sem (@id->sem id)]
-                              (swap! id->result assoc id (DRPCExecutionException. "Request timed out"))
-                              (.release sem))
-                            (cleanup id)
-                            ))
-                        TIMEOUT-CHECK-SECS
-                        ))
-        ]
+                       (fn []
+                         (doseq [[id start] @id->start]
+                           (when (> (time-delta start) (conf DRPC-REQUEST-TIMEOUT-SECS))
+                             (when-let [sem (@id->sem id)]
+                               (.remove (acquire-queue request-queues (@id->function id)) (@id->request id))
+                               (log-warn "Timeout DRPC request id: " id " start at " start)
+                               (.release sem))
+                             (cleanup id)))
+                         (timeout-check-secs)))]
     (reify DistributedRPC$Iface
-      (^String execute [this ^String function ^String args]
+
+      (^String execute
+        [this ^String function ^String args]
         (log-debug "Received DRPC request for " function " " args " at " (System/currentTimeMillis))
         (let [id (str (swap! ctr (fn [v] (mod (inc v) 1000000000))))
               ^Semaphore sem (Semaphore. 0)
               req (DRPCRequest. args id)
-              ^ConcurrentLinkedQueue queue (acquire-queue request-queues function)
-              ]
+              ^ConcurrentLinkedQueue queue (acquire-queue request-queues function)]
           (swap! id->start assoc id (current-time-secs))
           (swap! id->sem assoc id sem)
+          (swap! id->function assoc id function)
+          (swap! id->request assoc id req)
           (.add queue req)
           (log-debug "Waiting for DRPC result for " function " " args " at " (System/currentTimeMillis))
           (.acquire sem)
@@ -83,34 +92,41 @@
             (log-debug "Returning DRPC result for " function " " args " at " (System/currentTimeMillis))
             (if (instance? DRPCExecutionException result)
               (throw result)
-              result
-              ))))
+              (if (nil? result)
+                (throw (DRPCExecutionException. "Request timed out"))
+                result)))))
+
       DistributedRPCInvocations$Iface
-      (^void result [this ^String id ^String result]
+
+      (^void result
+        [this ^String id ^String result]
         (let [^Semaphore sem (@id->sem id)]
           (log-debug "Received result " result " for " id " at " (System/currentTimeMillis))
           (when sem
             (swap! id->result assoc id result)
-            (.release sem)
-            )))
-      (^void failRequest [this ^String id]
+            (.release sem))))
+
+      (^void failRequest
+        [this ^String id]
         (let [^Semaphore sem (@id->sem id)]
           (when sem
             (swap! id->result assoc id (DRPCExecutionException. "Request failed"))
-            (.release sem)
-            )))
-      (^DRPCRequest fetchRequest [this ^String func]
+            (.release sem))))
+
+      (^DRPCRequest fetchRequest
+        [this ^String func]
         (let [^ConcurrentLinkedQueue queue (acquire-queue request-queues func)
               ret (.poll queue)]
           (if ret
             (do (log-debug "Fetched request for " func " at " (System/currentTimeMillis))
-                ret)
-            (DRPCRequest. "" ""))
-          ))
+              ret)
+            (DRPCRequest. "" ""))))
+
       Shutdownable
-      (shutdown [this]
-        (.interrupt clear-thread))
-      )))
+
+      (shutdown
+        [this]
+        (.interrupt clear-thread)))))
 
 (defn launch-server!
   ([]
@@ -118,26 +134,34 @@
           worker-threads (int (conf DRPC-WORKER-THREADS))
           queue-size (int (conf DRPC-QUEUE-SIZE))
           service-handler (service-handler)
-          ;; requests and returns need to be on separate thread pools, since calls to
-          ;; "execute" don't unblock until other thrift methods are called. So if 
-          ;; 64 threads are calling execute, the server won't accept the result
-          ;; invocations that will unblock those threads
-          handler-server (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-PORT)))
-                                             (THsHaServer$Args.)
-                                             (.workerThreads 64)
-                                             (.executorService (ThreadPoolExecutor. worker-threads worker-threads 
-                                                                 60 TimeUnit/SECONDS (ArrayBlockingQueue. queue-size)))
-                                             (.protocolFactory (TBinaryProtocol$Factory.))
-                                             (.processor (DistributedRPC$Processor. service-handler))
-                                             ))
-          invoke-server (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-INVOCATIONS-PORT)))
-                                             (THsHaServer$Args.)
-                                             (.workerThreads 64)
-                                             (.protocolFactory (TBinaryProtocol$Factory.))
-                                             (.processor (DistributedRPCInvocations$Processor. service-handler))
-                                             ))]
-      
-      (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop handler-server) (.stop invoke-server))))
+
+          ;; Requests and returns need to be on separate thread pools, since
+          ;; calls to "execute" don't unblock until other thrift methods are
+          ;; called. So if 64 threads are calling execute, the server won't
+          ;; accept the result invocations that will unblock those threads.
+
+          handler-server
+          (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-PORT)))
+                            (THsHaServer$Args.)
+                            (.workerThreads 64)
+                            (.executorService
+                              (ThreadPoolExecutor.
+                                worker-threads worker-threads 60 TimeUnit/SECONDS
+                                (ArrayBlockingQueue. queue-size)))
+                            (.protocolFactory (TBinaryProtocol$Factory.))
+                            (.processor (DistributedRPC$Processor. service-handler))))
+
+          invoke-server
+          (THsHaServer. (-> (TNonblockingServerSocket. (int (conf DRPC-INVOCATIONS-PORT)))
+                            (THsHaServer$Args.)
+                            (.workerThreads 64)
+                            (.protocolFactory (TBinaryProtocol$Factory.))
+                            (.processor
+                              (DistributedRPCInvocations$Processor. service-handler))))]
+
+      (.addShutdownHook
+        (Runtime/getRuntime)
+        (Thread. (fn [] (.stop handler-server) (.stop invoke-server))))
       (log-message "Starting Distributed RPC servers...")
       (future (.serve invoke-server))
       (.serve handler-server))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index 15c375f..1bbe53d 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -204,6 +204,7 @@
         storm-conf (normalized-component-conf (:storm-conf worker) worker-context component-id)
         executor-type (executor-type worker-context component-id)
         batch-transfer->worker (disruptor/disruptor-queue
+                                  (str "executor"  executor-id "-send-queue")
                                   (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
                                   :claim-strategy :single-threaded
                                   :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
index 2a48f55..4903b4d 100644
--- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
@@ -20,10 +20,7 @@
   (:use [ring.adapter.jetty :only [run-jetty]])
   (:import [org.slf4j LoggerFactory])
   (:import [ch.qos.logback.classic Logger])
-  (:import [org.apache.commons.logging LogFactory])
-  (:import [org.apache.commons.logging.impl Log4JLogger])
   (:import [ch.qos.logback.core FileAppender])
-  (:import [org.apache.log4j Level])
   (:import [java.io File])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]
@@ -70,13 +67,6 @@ Note that if anything goes wrong, this will throw an Error and exit."
          (filter #(.contains % grep) (.split tail-string "\n")))
        (.replaceAll tail-string "\n" "\n<br>"))))
 
-(defn log-level-page [name level]
-  (let [log (LogFactory/getLog name)]
-    (if level
-      (if (instance? Log4JLogger log)
-        (.setLevel (.getLogger log) (Level/toLevel level))))
-    (str "effective log level for " name " is " (.getLevel (.getLogger log)))))
-
 (defn log-template [body]
   (html4
    [:head
@@ -95,8 +85,6 @@ Note that if anything goes wrong, this will throw an Error and exit."
 (defroutes log-routes
   (GET "/log" [:as req & m]
        (log-template (log-page (:file m) (:tail m) (:grep m) (:log-root req))))
-  (GET "/loglevel" [:as {cookies :cookies} & m]
-       (log-template (log-level-page (:name m) (:level m))))
   (route/resources "/")
   (route/not-found "Page not found"))
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 921c259..16765d9 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -18,8 +18,10 @@
   (:use [backtype.storm bootstrap])
   (:require [backtype.storm.daemon [executor :as executor]])
   (:import [java.util.concurrent Executors])
+  (:import [java.util ArrayList HashMap])
+  (:import [backtype.storm.utils TransferDrainer])
   (:import [backtype.storm.messaging TransportFactory])
-  (:import [backtype.storm.messaging IContext IConnection])
+  (:import [backtype.storm.messaging TaskMessage IContext IConnection])
   (:gen-class))
 
 (bootstrap)
@@ -109,25 +111,32 @@
 (defn mk-transfer-fn [worker]
   (let [local-tasks (-> worker :task-ids set)
         local-transfer (:transfer-local-fn worker)
-        ^DisruptorQueue transfer-queue (:transfer-queue worker)]
+        ^DisruptorQueue transfer-queue (:transfer-queue worker)
+        task->node+port (:cached-task->node+port worker)]
     (fn [^KryoTupleSerializer serializer tuple-batch]
       (let [local (ArrayList.)
-            remote (ArrayList.)]
+            remoteMap (HashMap.)]
         (fast-list-iter [[task tuple :as pair] tuple-batch]
           (if (local-tasks task)
             (.add local pair)
-            (.add remote pair)
-            ))
+            
+            ;;Using java objects directly to avoid performance issues in java code
+            (let [node+port (get @task->node+port task)]
+              (when (not (.get remoteMap node+port))
+                (.put remoteMap node+port (ArrayList.)))
+              (let [remote (.get remoteMap node+port)]
+                (.add remote (TaskMessage. task (.serialize serializer tuple)))
+                 ))))
+        
         (local-transfer local)
-        ;; not using map because the lazy seq shows up in perf profiles
-        (let [serialized-pairs (fast-list-for [[task ^TupleImpl tuple] remote] [task (.serialize serializer tuple)])]
-          (disruptor/publish transfer-queue serialized-pairs)
-          )))))
+        (disruptor/publish transfer-queue remoteMap)
+          ))))
 
 (defn- mk-receive-queue-map [storm-conf executors]
   (->> executors
        ;; TODO: this depends on the type of executor
-       (map (fn [e] [e (disruptor/disruptor-queue (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
+       (map (fn [e] [e (disruptor/disruptor-queue (str "receive-queue" e)
+                                                  (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
                                                   :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))]))
        (into {})
        ))
@@ -158,18 +167,19 @@
   ;; actually just do it via interfaces. just need to make sure to hide setResource from tasks
   {})
 
-(defn mk-halting-timer []
+(defn mk-halting-timer [timer-name]
   (mk-timer :kill-fn (fn [t]
                        (log-error t "Error when processing event")
                        (halt-process! 20 "Error when processing an event")
-                       )))
+                       )
+            :timer-name timer-name))
 
 (defn worker-data [conf mq-context storm-id assignment-id port worker-id]
   (let [cluster-state (cluster/mk-distributed-cluster-state conf)
         storm-cluster-state (cluster/mk-storm-cluster-state cluster-state)
         storm-conf (read-supervisor-storm-conf conf storm-id)
         executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port))
-        transfer-queue (disruptor/disruptor-queue (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
+        transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
                                                   :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
         executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
         
@@ -195,11 +205,11 @@
       :storm-conf storm-conf
       :topology topology
       :system-topology (system-topology! storm-conf topology)
-      :heartbeat-timer (mk-halting-timer)
-      :refresh-connections-timer (mk-halting-timer)
-      :refresh-active-timer (mk-halting-timer)
-      :executor-heartbeat-timer (mk-halting-timer)
-      :user-timer (mk-halting-timer)
+      :heartbeat-timer (mk-halting-timer "heartbeat-timer")
+      :refresh-connections-timer (mk-halting-timer "refresh-connections-timer")
+      :refresh-active-timer (mk-halting-timer "refresh-active-timer")
+      :executor-heartbeat-timer (mk-halting-timer "executor-heartbeat-timer")
+      :user-timer (mk-halting-timer "user-timer")
       :task->component (HashMap. (storm-task-info topology storm-conf)) ; for optimized access when used in tasks later on
       :component->stream->fields (component->stream->fields (:system-topology <>))
       :component->sorted-tasks (->> (:task->component <>) reverse-map (map-val sort))
@@ -218,6 +228,7 @@
       :default-shared-resources (mk-default-resources <>)
       :user-shared-resources (mk-user-resources <>)
       :transfer-local-fn (mk-transfer-local-fn <>)
+      :receiver-thread-count (get storm-conf WORKER-RECEIVER-THREAD-COUNT)
       :transfer-fn (mk-transfer-fn <>)
       )))
 
@@ -296,28 +307,19 @@
 ;; TODO: consider having a max batch size besides what disruptor does automagically to prevent latency issues
 (defn mk-transfer-tuples-handler [worker]
   (let [^DisruptorQueue transfer-queue (:transfer-queue worker)
-        drainer (ArrayList.)
+        drainer (TransferDrainer.)
         node+port->socket (:cached-node+port->socket worker)
         task->node+port (:cached-task->node+port worker)
         endpoint-socket-lock (:endpoint-socket-lock worker)
         ]
     (disruptor/clojure-handler
       (fn [packets _ batch-end?]
-        (.addAll drainer packets)
+        (.add drainer packets)
+        
         (when batch-end?
           (read-locked endpoint-socket-lock
-            (let [node+port->socket @node+port->socket
-                  task->node+port @task->node+port]
-              ;; consider doing some automatic batching here (would need to not be serialized at this point to remove per-tuple overhead)
-              ;; try using multipart messages ... first sort the tuples by the target node (without changing the local ordering)
-            
-              (fast-list-iter [[task ser-tuple] drainer]
-                ;; TODO: consider write a batch of tuples here to every target worker  
-                ;; group by node+port, do multipart send              
-                (let [node-port (get task->node+port task)]
-                  (when node-port
-                    (.send ^IConnection (get node+port->socket node-port) task ser-tuple))
-                    ))))
+            (let [node+port->socket @node+port->socket]
+              (.send drainer node+port->socket)))
           (.clear drainer))))))
 
 (defn launch-receive-thread [worker]
@@ -325,6 +327,7 @@
   (msg-loader/launch-receive-thread!
     (:mq-context worker)
     (:storm-id worker)
+    (:receiver-thread-count worker)
     (:port worker)
     (:transfer-local-fn worker)
     (-> worker :storm-conf (get TOPOLOGY-RECEIVER-BUFFER-SIZE))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/disruptor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/disruptor.clj b/storm-core/src/clj/backtype/storm/disruptor.clj
index d8c5c91..40b47fc 100644
--- a/storm-core/src/clj/backtype/storm/disruptor.clj
+++ b/storm-core/src/clj/backtype/storm/disruptor.clj
@@ -45,8 +45,9 @@
 ;; wouldn't make it to the acker until the batch timed out and another tuple was played into the queue,
 ;; unblocking the consumer
 (defnk disruptor-queue
-  [buffer-size :claim-strategy :multi-threaded :wait-strategy :block]
-  (DisruptorQueue. ((CLAIM-STRATEGY claim-strategy) buffer-size)
+  [^String queue-name buffer-size :claim-strategy :multi-threaded :wait-strategy :block]
+  (DisruptorQueue. queue-name
+                   ((CLAIM-STRATEGY claim-strategy) buffer-size)
                    (mk-wait-strategy wait-strategy)))
 
 (defn clojure-handler
@@ -88,14 +89,12 @@
 
 (defnk consume-loop*
   [^DisruptorQueue queue handler
-   :kill-fn (fn [error] (halt-process! 1 "Async loop died!"))
-   :thread-name nil]
+   :kill-fn (fn [error] (halt-process! 1 "Async loop died!"))]
   (let [ret (async-loop
               (fn [] (consume-batch-when-available queue handler) 0)
               :kill-fn kill-fn
-              :thread-name thread-name)]
-    (consumer-started! queue)
-    ret))
+              :thread-name (.getName queue))]
+     (consumer-started! queue) ret))
 
 (defmacro consume-loop [queue & handler-args]
   `(let [handler# (handler ~@handler-args)]

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/messaging/loader.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/messaging/loader.clj b/storm-core/src/clj/backtype/storm/messaging/loader.clj
index 9e43c26..e13b5a8 100644
--- a/storm-core/src/clj/backtype/storm/messaging/loader.clj
+++ b/storm-core/src/clj/backtype/storm/messaging/loader.clj
@@ -15,7 +15,7 @@
 ;; limitations under the License.
 (ns backtype.storm.messaging.loader
   (:use [backtype.storm util log])
-  (:import [java.util ArrayList])
+  (:import [java.util ArrayList Iterator])
   (:import [backtype.storm.messaging IContext IConnection TaskMessage])
   (:import [backtype.storm.utils DisruptorQueue MutableObject])
   (:require [backtype.storm.messaging [local :as local]])
@@ -24,45 +24,62 @@
 (defn mk-local-context []
   (local/mk-context))
 
+(defn- mk-receive-thread [context storm-id port transfer-local-fn  daemon kill-fn priority socket max-buffer-size thread-id]
+    (async-loop
+       (fn []
+         (log-message "Starting receive-thread: [stormId: " storm-id ", port: " port ", thread-id: " thread-id  " ]")
+         (fn []
+           (let [batched (ArrayList.)
+                 ^Iterator iter (.recv ^IConnection socket 0 thread-id)
+                 closed (atom false)]
+             (when iter
+               (while (and (not @closed) (.hasNext iter)) 
+                  (let [packet (.next iter)
+                        task (if packet (.task ^TaskMessage packet))
+                        message (if packet (.message ^TaskMessage packet))]
+                      (if (= task -1)
+                         (do (log-message "Receiving-thread:[" storm-id ", " port "] received shutdown notice")
+                           (.close socket)
+                           (reset! closed  true))
+                         (when packet (.add batched [task message]))))))
+             
+             (when (not @closed)
+               (do
+                 (if (> (.size batched) 0)
+                   (transfer-local-fn batched))
+                 0)))))
+         :factory? true
+         :daemon daemon
+         :kill-fn kill-fn
+         :priority priority
+         :thread-name (str "worker-receiver-thread-" thread-id)))
+
+(defn- mk-receive-threads [context storm-id port transfer-local-fn  daemon kill-fn priority socket max-buffer-size thread-count]
+  (into [] (for [thread-id (range thread-count)] 
+             (mk-receive-thread context storm-id port transfer-local-fn  daemon kill-fn priority socket max-buffer-size thread-id))))
+
+
 (defnk launch-receive-thread!
-  [context storm-id port transfer-local-fn max-buffer-size
+  [context storm-id receiver-thread-count port transfer-local-fn max-buffer-size
    :daemon true
    :kill-fn (fn [t] (System/exit 1))
    :priority Thread/NORM_PRIORITY]
   (let [max-buffer-size (int max-buffer-size)
-        vthread (async-loop
-                 (fn []
-                   (let [socket (.bind ^IContext context storm-id port)]
-                     (fn []
-                       (let [batched (ArrayList.)
-                             init (.recv ^IConnection socket 0)]
-                         (loop [packet init]
-                           (let [task (if packet (.task ^TaskMessage packet))
-                                 message (if packet (.message ^TaskMessage packet))]
-                             (if (= task -1)
-                               (do (log-message "Receiving-thread:[" storm-id ", " port "] received shutdown notice")
-                                 (.close socket)
-                                 nil )
-                               (do
-                                 (when packet (.add batched [task message]))
-                                 (if (and packet (< (.size batched) max-buffer-size))
-                                   (recur (.recv ^IConnection socket 1))
-                                   (do (transfer-local-fn batched)
-                                     0 ))))))))))
-                 :factory? true
-                 :daemon daemon
-                 :kill-fn kill-fn
-                 :priority priority)]
+        socket (.bind ^IContext context storm-id port)
+        thread-count (if receiver-thread-count receiver-thread-count 1)
+        vthreads (mk-receive-threads context storm-id port transfer-local-fn daemon kill-fn priority socket max-buffer-size thread-count)]
     (fn []
       (let [kill-socket (.connect ^IContext context storm-id "localhost" port)]
         (log-message "Shutting down receiving-thread: [" storm-id ", " port "]")
         (.send ^IConnection kill-socket
-                  -1
-                  (byte-array []))
-        (log-message "Waiting for receiving-thread:[" storm-id ", " port "] to die")
-        (.join vthread)
+                  -1 (byte-array []))
+        
         (.close ^IConnection kill-socket)
+        
+        (log-message "Waiting for receiving-thread:[" storm-id ", " port "] to die")
+        
+        (for [thread-id (range thread-count)] 
+             (.join (vthreads thread-id)))
+        
         (log-message "Shutdown receiving-thread: [" storm-id ", " port "]")
-        ))))
-
-
+        ))))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/messaging/local.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/messaging/local.clj b/storm-core/src/clj/backtype/storm/messaging/local.clj
index bf4d5b2..801f22d 100644
--- a/storm-core/src/clj/backtype/storm/messaging/local.clj
+++ b/storm-core/src/clj/backtype/storm/messaging/local.clj
@@ -18,7 +18,8 @@
   (:use [backtype.storm log])
   (:import [backtype.storm.messaging IContext IConnection TaskMessage])
   (:import [java.util.concurrent LinkedBlockingQueue])
-  (:import [java.util Map])
+  (:import [java.util Map Iterator])
+  (:import [java.util Iterator ArrayList])
   (:gen-class))
 
 (defn add-queue! [queues-map lock storm-id port]
@@ -30,16 +31,25 @@
 
 (deftype LocalConnection [storm-id port queues-map lock queue]
   IConnection
-  (^TaskMessage recv [this ^int flags]
+  (^Iterator recv [this ^int flags ^int clientId]
     (when-not queue
       (throw (IllegalArgumentException. "Cannot receive on this socket")))
-    (if (= flags 1)
-      (.poll queue)
-      (.take queue)))
+    (let [ret (ArrayList.)
+          msg (if (= flags 1) (.poll queue) (.take queue))]
+      (if msg
+        (do 
+          (.add ret msg)
+          (.iterator ret))
+        nil)))
   (^void send [this ^int taskId ^bytes payload]
     (let [send-queue (add-queue! queues-map lock storm-id port)]
       (.put send-queue (TaskMessage. taskId payload))
       ))
+  (^void send [this ^Iterator iter]
+    (let [send-queue (add-queue! queues-map lock storm-id port)]
+      (while (.hasNext iter) 
+         (.put send-queue (.next iter)))
+      ))
   (^void close [this]
     ))
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/timer.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/timer.clj b/storm-core/src/clj/backtype/storm/timer.clj
index b105698..6f00a5c 100644
--- a/storm-core/src/clj/backtype/storm/timer.clj
+++ b/storm-core/src/clj/backtype/storm/timer.clj
@@ -24,44 +24,51 @@
 ;; it integrates with Storm's time simulation capabilities. This lets us test
 ;; code that does asynchronous work on the timer thread
 
-(defnk mk-timer
-  [:kill-fn (fn [& _] )]
-  (let [queue (PriorityQueue. 10
-                              (reify Comparator
-                                (compare [this o1 o2]
-                                         (- (first o1) (first o2)))
-                                (equals [this obj]
-                                        true)))
+(defnk mk-timer [:kill-fn (fn [& _] ) :timer-name nil]
+  (let [queue (PriorityQueue. 10 (reify Comparator
+                                   (compare
+                                     [this o1 o2]
+                                     (- (first o1) (first o2)))
+                                   (equals
+                                     [this obj]
+                                     true)))
         active (atom true)
         lock (Object.)
         notifier (Semaphore. 0)
+        thread-name (if timer-name timer-name "timer")
         timer-thread (Thread.
                        (fn []
                          (while @active
                            (try
                              (let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
                                (if (and elem (>= (current-time-millis) time-millis))
-                                 ;; imperative to not run the function inside the timer lock
-                                 ;; otherwise, it's possible to deadlock if function deals with other locks
-                                 ;; (like the submit lock)
+                                 ;; It is imperative to not run the function
+                                 ;; inside the timer lock. Otherwise, it is
+                                 ;; possible to deadlock if the fn deals with
+                                 ;; other locks, like the submit lock.
                                  (let [afn (locking lock (second (.poll queue)))]
                                    (afn))
-                                 (if time-millis ;; if any events are scheduled
-                                   ;; sleep until event generation
-                                   ;; note that if any recurring events are scheduled then we will always go through
-                                   ;; this branch, sleeping only the exact necessary amount of time
+                                 (if time-millis
+                                   ;; If any events are scheduled, sleep until
+                                   ;; event generation. If any recurring events
+                                   ;; are scheduled then we will always go
+                                   ;; through this branch, sleeping only the
+                                   ;; exact necessary amount of time.
                                    (Time/sleep (- time-millis (current-time-millis)))
-                                   ;; else poll to see if any new event was scheduled
-                                   ;; this is in essence the response time for detecting any new event schedulings when
-                                   ;; there are no scheduled events
+                                   ;; Otherwise poll to see if any new event
+                                   ;; was scheduled. This is, in essence, the
+                                   ;; response time for detecting any new event
+                                   ;; schedulings when there are no scheduled
+                                   ;; events.
                                    (Time/sleep 1000))))
                              (catch Throwable t
-                               ;; because the interrupted exception can be wrapped in a runtimeexception
+                               ;; Because the interrupted exception can be
+                               ;; wrapped in a RuntimeException.
                                (when-not (exception-cause? InterruptedException t)
                                  (kill-fn t)
                                  (reset! active false)
                                  (throw t)))))
-                         (.release notifier)))]
+                         (.release notifier)) thread-name)]
     (.setDaemon timer-thread true)
     (.setPriority timer-thread Thread/MAX_PRIORITY)
     (.start timer-thread)
@@ -90,7 +97,7 @@
             delay-secs
             (fn this []
               (afn)
-              ; this avoids a race condition with cancel-timer
+              ; This avoids a race condition with cancel-timer.
               (schedule timer recur-secs this :check-active false))))
 
 (defn cancel-timer


[06/13] Merge branch 'master' into idiomatic-clojure-01

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/js/script.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js
index e9902ab..8f7608e 100644
--- a/storm-core/src/ui/public/js/script.js
+++ b/storm-core/src/ui/public/js/script.js
@@ -71,7 +71,7 @@ function ensureInt(n) {
 function confirmAction(id, name, action, wait, defaultWait) {
     var opts = {
         type:'POST',
-        url:'/topology/' + id + '/' + action
+        url:'/api/v1/topology/' + id + '/' + action
     };
     if (wait) {
         var waitSecs = prompt('Do you really want to ' + action + ' topology "' + name + '"? ' +
@@ -91,7 +91,7 @@ function confirmAction(id, name, action, wait, defaultWait) {
     $.ajax(opts).always(function () {
         window.location.reload();
     }).fail(function () {
-        alert("Error while communicating with Nimbus.")
+        alert("Error while communicating with Nimbus.");
     });
 
     return false;
@@ -106,4 +106,49 @@ $(function () {
           delayIn: 1000
       });
     }
-})
+});
+
+function formatConfigData(data) {
+    var mustacheFormattedData = {'config':[]};
+    for (var prop in data) {
+       if(data.hasOwnProperty(prop)) {
+           mustacheFormattedData['config'].push({
+               'key': prop,
+               'value': data[prop]
+           });
+       }
+    }
+    return mustacheFormattedData;
+}
+
+
+function renderToggleSys(div) {
+    var sys = $.cookies.get("sys") || false;
+    if(sys) {
+       div.append("<span data-original-title=\"Use this to toggle inclusion of storm system components.\" class=\"tip right\"><input onclick=\"toggleSys()\" value=\"Hide System Stats\" type=\"button\"></span>");
+    } else {
+       div.append("<span class=\"tip right\" title=\"Use this to toggle inclusion of storm system components.\"><input onclick=\"toggleSys()\" value=\"Show System Stats\" type=\"button\"></span>");
+    }
+}
+
+function topologyActionJson(id,name,status,msgTimeout) {
+    var jsonData = {};
+    jsonData["id"] = id;
+    jsonData["name"] = name;
+    jsonData["msgTimeout"] = msgTimeout;
+    jsonData["activateStatus"] = (status === "ACTIVE") ? "disabled" : "enabled";
+    jsonData["deactivateStatus"] = (status === "ACTIVE") ? "enabled" : "disabled";
+    jsonData["rebalanceStatus"] = (status === "ACTIVE" || status === "INACTIVE" ) ? "enabled" : "disabled";
+    jsonData["killStatus"] = (status !== "KILLED") ? "enabled" : "disabled";
+    return jsonData;
+}
+
+function topologyActionButton(id,name,status,actionLabel,command,wait,defaultWait) {
+    var buttonData = {};
+    buttonData["buttonStatus"] = status ;
+    buttonData["actionLabel"] = actionLabel;
+    buttonData["command"] = command;
+    buttonData["isWait"] = wait;
+    buttonData["defaultWait"] = defaultWait;
+    return buttonData;
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/js/visualization.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/visualization.js b/storm-core/src/ui/public/js/visualization.js
new file mode 100644
index 0000000..87596b8
--- /dev/null
+++ b/storm-core/src/ui/public/js/visualization.js
@@ -0,0 +1,403 @@
+// Inspired by 
+// https://github.com/samizdatco/arbor/blob/master/docs/sample-project/main.js
+
+function renderGraph(elem) {
+
+    var canvas = $(elem).get(0);
+    canvas.width = $(window).width();
+    canvas.height = $(window).height();
+    var ctx = canvas.getContext("2d");
+    var gfx = arbor.Graphics(canvas);
+    var psys;
+
+    var totaltrans = 0;
+    var weights = {};
+    var texts = {};
+    var update = false;
+
+    var myRenderer = {
+        init: function(system){ 
+            psys = system;
+            psys.screenSize(canvas.width, canvas.height)
+            psys.screenPadding(20);
+            myRenderer.initMouseHandling();
+        },
+
+        signal_update: function() {
+            update = true;
+        },
+
+        redraw: function() { 
+            
+            if(!psys)
+                return;
+            
+            if(update) {
+                totaltrans = calculate_total_transmitted(psys);
+                weights = calculate_weights(psys, totaltrans);
+                texts = calculate_texts(psys, totaltrans);
+                update = false;
+            }
+
+
+
+            ctx.fillStyle = "white";
+            ctx.fillRect(0, 0, canvas.width, canvas.height);
+            var x = 0;
+            
+
+            psys.eachEdge(function(edge, pt1, pt2) {
+
+                var len = Math.sqrt(Math.pow(pt2.x - pt1.x,2) + Math.pow(pt2.y - pt1.y,2));
+                var sublen = len - (Math.max(50, 20 + gfx.textWidth(edge.target.name)) / 2);
+                var thirdlen = len/3;
+                var theta = Math.atan2(pt2.y - pt1.y, pt2.x - pt1.x);
+                
+                var newpt2 = {
+                    x : pt1.x + (Math.cos(theta) * sublen),
+                    y : pt1.y + (Math.sin(theta) * sublen)
+                };
+
+                var thirdpt = {
+                    x: pt1.x + (Math.cos(theta) * thirdlen),
+                    y: pt1.y + (Math.sin(theta) * thirdlen)
+                }
+
+                weight = weights[edge.source.name + edge.target.name];
+                
+                if(!weights[edge.source.name + edge.target.name])
+                {
+                    totaltrans = calculate_total_transmitted(psys);
+                    weights = calculate_weights(psys, totaltrans);
+                }
+
+                ctx.strokeStyle = "rgba(0,0,0, .333)";
+                ctx.lineWidth = 25 * weight + 5;
+                ctx.beginPath();
+
+                var arrlen = 15;
+                ctx.moveTo(pt1.x, pt1.y);
+                ctx.lineTo(newpt2.x, newpt2.y);
+                ctx.lineTo(newpt2.x - arrlen * Math.cos(theta-Math.PI/6), newpt2.y - arrlen * Math.sin(theta - Math.PI/6));
+                ctx.moveTo(newpt2.x, newpt2.y);
+                ctx.lineTo(newpt2.x - arrlen * Math.cos(theta+Math.PI/6), newpt2.y - arrlen * Math.sin(theta + Math.PI/6));
+
+                
+                if (texts[edge.source.name + edge.target.name] == null)
+                {
+                    totaltrans = calculate_total_transmitted(psys);
+                    texts = calculate_texts(psys, totaltrans);
+                }
+
+                gfx.text(texts[edge.source.name + edge.target.name], thirdpt.x, thirdpt.y + 10, {color:"black", align:"center", font:"Arial", size:10})
+                ctx.stroke();
+            });
+
+            psys.eachNode(function(node, pt) {
+                var col;
+
+                var real_trans = gather_stream_count(node.data[":stats"], "default", "600");
+                
+                if(node.data[":type"] === "bolt") {
+                    var cap = Math.min(node.data[":capacity"], 1);
+                    var red = Math.floor(cap * 225) + 30;
+                    var green = Math.floor(255 - red);
+                    var blue = Math.floor(green/5);
+                    col = arbor.colors.encode({r:red,g:green,b:blue,a:1});
+                } else {
+                    col = "#0000FF";
+                }
+                
+                var w = Math.max(55, 25 + gfx.textWidth(node.name));
+                
+                gfx.oval(pt.x - w/2, pt.y - w/2, w, w, {fill: col});
+                gfx.text(node.name, pt.x, pt.y+3, {color:"white", align:"center", font:"Arial", size:12});
+                gfx.text(node.name, pt.x, pt.y+3, {color:"white", align:"center", font:"Arial", size:12});
+                
+                gfx.text(parseFloat(node.data[":latency"]).toFixed(2) + " ms", pt.x, pt.y + 17, {color:"white", align:"center", font:"Arial", size:12});
+                
+            });
+
+            // Draw gradient sidebar
+            ctx.rect(0,0,50,canvas.height);
+            var grd = ctx.createLinearGradient(0,0,50,canvas.height);
+            grd.addColorStop(0, '#1ee12d');
+            grd.addColorStop(1, '#ff0000');
+            ctx.fillStyle=grd;
+            ctx.fillRect(0,0,50,canvas.height);
+            
+            
+        },
+        
+        initMouseHandling:function() {
+            var dragged = null;
+
+            var clicked = false;
+            
+            var handler = {
+                clicked:function(e){
+                    var pos = $(canvas).offset();
+                    _mouseP = arbor.Point(e.pageX-pos.left, e.pageY - pos.top);
+                    dragged = psys.nearest(_mouseP);
+                    
+                    if(dragged && dragged.node !== null) {
+                        dragged.node.fixed = true;
+                    }
+                    
+                    clicked = true;
+                    setTimeout(function(){clicked = false;}, 50);
+
+                    $(canvas).bind('mousemove', handler.dragged);
+                    $(window).bind('mouseup', handler.dropped);
+                    
+                    return false;
+                },
+                
+                dragged:function(e) {
+
+                    var pos = $(canvas).offset();
+                    var s = arbor.Point(e.pageX-pos.left, e.pageY-pos.top);
+                    
+                    if(dragged && dragged.node != null) {
+                        var p = psys.fromScreen(s);
+                        dragged.node.p = p;
+                    }
+                    
+                    return false;
+                    
+                },
+
+                dropped:function(e) {
+                    if(clicked) {
+                        if(dragged.distance < 50) {
+                            if(dragged && dragged.node != null) { 
+                                window.location = dragged.node.data[":link"];
+                            }
+                        }
+                    }
+
+                    if(dragged === null || dragged.node === undefined) return;
+                    if(dragged.node !== null) dragged.node.fixed = false;
+                    dragged.node.tempMass = 1000;
+                    dragged = null;
+                    $(canvas).unbind('mousemove', handler.dragged);
+                    $(window).unbind('mouseup', handler.dropped);
+                    _mouseP = null;
+                    return false;
+                }
+                
+            }
+            
+            $(canvas).mousedown(handler.clicked);
+        }
+    }
+    
+    return myRenderer;
+}
+
+function calculate_texts(psys, totaltrans) {
+    var texts = {};
+    psys.eachEdge(function(edge, pt1, pt2) {
+        var text = "";
+        for(var i = 0; i < edge.target.data[":inputs"].length; i++) {
+            var stream = edge.target.data[":inputs"][i][":stream"];
+            var sani_stream = edge.target.data[":inputs"][i][":sani-stream"];
+            if(stream_checked(sani_stream) 
+               && edge.target.data[":inputs"][i][":component"] === edge.source.name) {
+                stream_transfered = gather_stream_count(edge.source.data[":stats"], sani_stream, "600");
+                text += stream + ": " 
+                    + stream_transfered + ": " 
+                    + (totaltrans > 0  ? Math.round((stream_transfered/totaltrans) * 100) : 0) + "%\n";
+                
+            }
+        }
+        
+        texts[edge.source.name + edge.target.name] = text;
+    });
+
+    return texts;
+}
+
+function calculate_weights(psys, totaltrans) {
+    var weights = {};
+ 
+    psys.eachEdge(function(edge, pt1, pt2) {
+        var trans = 0;
+        for(var i = 0; i < edge.target.data[":inputs"].length; i++) {
+            var stream = edge.target.data[":inputs"][i][":sani-stream"];
+            if(stream_checked(stream) && edge.target.data[":inputs"][i][":component"] === edge.source.name)
+                trans += gather_stream_count(edge.source.data[":stats"], stream, "600");
+        }
+        weights[edge.source.name + edge.target.name] = (totaltrans > 0 ? trans/totaltrans : 0);
+    });
+    return weights;
+}
+
+function calculate_total_transmitted(psys) {
+    var totaltrans = 0;
+    var countedmap = {}
+    psys.eachEdge(function(node, pt, pt2) {
+        if(!countedmap[node.source.name])
+            countedmap[node.source.name] = {};
+
+        for(var i = 0; i < node.target.data[":inputs"].length; i++) {
+            var stream = node.target.data[":inputs"][i][":stream"];
+            if(stream_checked(node.target.data[":inputs"][i][":sani-stream"]))
+            {
+                if(!countedmap[node.source.name][stream]) {
+                    if(node.source.data[":stats"])
+                    {
+                        var toadd = gather_stream_count(node.source.data[":stats"], node.target.data[":inputs"][i][":sani-stream"], "600");
+                        totaltrans += toadd;
+                    }
+                    countedmap[node.source.name][stream] = true;
+                }
+            }
+        }
+        
+    });
+
+    return totaltrans;
+}
+
+function has_checked_stream_input(inputs) {
+    
+    for(var i = 0; i < inputs.length; i++) {
+        var x = stream_checked(inputs[i][":sani-stream"]);
+        if(x) 
+            return true;
+    }
+    return false;
+}
+
+function stream_checked(stream) {
+    var checked = $("#" + stream).is(":checked");
+    return checked;
+}
+
+function has_checked_stream_output(jdat, component) {
+    var ret = false;
+    $.each(jdat, function(k, v) {
+        for(var i = 0; i < v[":inputs"].length; i++) {
+            if(stream_checked(v[":inputs"][i][":sani-stream"]) 
+               && v[":inputs"][i][":component"] == component)
+                ret = true;
+        }
+    });
+    return ret;
+}
+
+function gather_stream_count(stats, stream, time) {
+    var transferred = 0;
+    if(stats)
+        for(var i = 0; i < stats.length; i++) {
+            if(stats[i][":transferred"] != null)
+            {
+                var stream_trans = stats[i][":transferred"][time][stream];
+                if(stream_trans != null)
+                    transferred += stream_trans;
+            }
+        }
+    return transferred;
+}
+
+
+function rechoose(jdat, sys, box) {
+    var id = box.id;
+    if($(box).is(':checked'))
+    {
+        //Check each node in our json data to see if it has inputs from or outputs to selected streams. If it does, add a node for it.
+        $.each(jdat,function(k,v) {
+            if( has_checked_stream_input(v[":inputs"]) || has_checked_stream_output(jdat, k))
+                sys.addNode(k,v);
+        });
+           
+        //Check each node in our json data and add necessary edges based on selected components.
+        $.each(jdat, function(k, v) {
+            for(var i = 0; i < v[":inputs"].length; i++)
+                if(v[":inputs"][i][":sani-stream"] === id) {
+                    
+                    sys.addEdge(v[":inputs"][i][":component"], k, v);
+                }
+        });
+    }
+    else {
+        //Check each node to see if it should be pruned.
+        sys.prune(function(node, from, to) {
+            return !has_checked_stream_input(node.data[":inputs"]) && !has_checked_stream_output(jdat, node.name);
+        });
+        
+        //Check each edge to see if it represents any selected streams. If not, prune it.
+        sys.eachEdge(function(edge, pt1, pt2) {
+            var inputs = edge.target.data[":inputs"];
+            
+            if($.grep(inputs, function(input) {
+                
+                return input[":component"] === edge.source.name 
+                    && stream_checked(input[":sani-stream"]);
+            
+            }).length == 0)
+            {
+                sys.pruneEdge(edge);
+            }
+        });
+    }
+
+    //Tell the particle system's renderer that it needs to update its labels, colors, widths, etc.
+    sys.renderer.signal_update();
+    sys.renderer.redraw();
+
+}
+
+var topology_data;
+function update_data(jdat, sys) {
+    $.each(jdat, function(k,v) {
+        if(sys.getNode(k))
+            sys.getNode(k).data = v;
+    });
+}
+
+var should_update;
+function show_visualization(sys) {
+
+    if(sys == null)
+    {
+        sys = arbor.ParticleSystem(20, 1000, 0.15, true, 55, 0.02, 0.6);
+        sys.renderer = renderGraph("#topoGraph");
+        sys.stop();
+
+        $(".stream-box").click(function () { rechoose(topology_data, sys, this) });    
+    }
+
+    should_update = true;
+    var update_freq_ms = 10000;
+    var update = function(should_rechoose){
+        $.ajax({
+            url: "/api/v1/topology/"+$.url().param("id")+"/visualization",
+            success: function(data, status, jqXHR) {
+                topology_data = data;
+                update_data(topology_data, sys);
+                sys.renderer.signal_update();
+                sys.renderer.redraw();
+                if(should_update)
+                    setTimeout(update, update_freq_ms);
+                if(should_rechoose)
+                    $(".stream-box").each(function () { rechoose(topology_data, sys, this) });
+            }
+        });
+    };
+    
+    update(true);
+    $("#visualization-container").show(500);
+    $("#show-hide-visualization").attr('value', 'Hide Visualization');
+    $("#show-hide-visualization").unbind("click");
+    $("#show-hide-visualization").click(function () { hide_visualization(sys) });
+}
+
+function hide_visualization(sys) {
+    should_update = false;
+    $("#visualization-container").hide(500);
+    $("#show-hide-visualization").attr('value', 'Show Visualization');
+    $("#show-hide-visualization").unbind("click");
+    $("#show-hide-visualization").click(function () { show_visualization(sys) });
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/templates/component-page-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/component-page-template.html b/storm-core/src/ui/public/templates/component-page-template.html
new file mode 100644
index 0000000..f2a5266
--- /dev/null
+++ b/storm-core/src/ui/public/templates/component-page-template.html
@@ -0,0 +1,152 @@
+<script id="component-summary-template" type="text/html">
+<h2>Component summary</h2>
+<table><thead><tr><th><span class="tip right" title="The ID assigned to a the Component by the Topology.">Id</span></th><th><span class="tip above" title="The name given to the topology by when it was submitted. Click the name to view the Topology's information.">Topology</span></th><th><span class="tip above" title="Executors are threads in a Worker process.">Executors</span></th><th><span class="tip above" 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></tr></thead>
+<tbody>
+<tr>
+<td>{{id}}</td>
+<td><a href="/topology.html?id={{topologyId}}">{{name}}</a></td>
+<td>{{executors}}</td>
+<td>{{tasks}}</td>
+</tbody>
+</table>
+</script>
+<script id="spout-stats-detail-template" type="text/html">
+<h2>Spout stats</h2>
+<table class="zebra-striped" id="spout-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The past period of time for which the statistics apply. Click on a value to set the window for this page.">Window</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span data-original-title="The number of Tuples emitted that sent to one or more bolts." class="tip above">Transferred</span></th><th class="header"><span class="tip above" title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">Complete latency (ms)</span></th><th class="header"><span class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">Acked</span></th><th class="header"><span data-original-title="The number of Tuple &quot;trees&quot; that were e
 xplicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">Failed</span></th></tr></thead>
+<tbody>
+{{#spoutSummary}}
+<tr>
+<td><a href="/component.html?id={{id}}&topology_id={{topologyId}}&window={{window}}">{{windowPretty}}</td>
+<td>{{transferred}}</td>
+<td>{{emitted}}</td>
+<td>{{completeLatency}}</td>
+<td>{{acked}}</td>
+<td>{{failed}}</td>
+</tr>
+{{/spoutSummary}}
+</tbody>
+</script>
+<script id="output-stats-template" type="text/html">
+<h2>Output stats ({{windowHint}})</h2>
+<table class="zebra-striped" id="output-stats-table"><thead><tr><th class="header headerSortDown"><span data-original-title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given." class="tip right">Stream</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span data-original-title="The number of Tuples emitted that sent to one or more bolts." class="tip above">Transferred</span></th><th class="header"><span data-original-title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done." class="tip above">Complete latency (ms)</span></th><th class="header"><span data-original-title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done." class="tip above">Acked</span></th><th class="header"><span data-original-title="The number of Tuple
  &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">Failed</span></th></tr></thead>
+<tbody>
+{{#outputStats}}
+<tr>
+<td>{{stream}}</td>
+<td>{{emitted}}</td>
+<td>{{transferred}}</td>
+<td>{{completeLatency}}</td>
+<td>{{acked}}</td>
+<td>{{failed}}</td>
+</tr>
+{{/outputStats}}
+</tbody>
+</table>
+</script>
+<script id="executor-stats-template" type="text/html">
+<h2>Executors ({{windowHint}})</h2>
+<table class="zebra-striped" id="executor-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The unique executor ID.">Id</span></th><th class="header"><span class="tip right" title="The length of time an Executor (thread) has been alive.">Uptime</span></th><th class="header"><span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">Host</span></th><th class="header"><span data-original-title="The port number used by the Worker to which an Executor is assigned. Click on the port number to open the logviewer page for this Worker." class="tip above">Port</span></th><th class="header"><span data-original-title="The number of Tuples emitted." class="tip above">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span class="tip above
 " title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">Complete latency (ms)</span></th><th class="header"><span class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">Acked</span></th><th class="header"><span data-original-title="The number of Tuple &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">Failed</span></th></tr></thead>
+<tbody>
+{{#executorStats}}
+<tr>
+<td>{{id}}</td>
+<td>{{uptime}}</td>
+<td>{{host}}</td>
+<td><a href="{{workerLogLink}}">{{port}}</a></td>
+<td>{{emitted}}</td>
+<td>{{transferred}}</td>
+<td>{{completeLatency}}</td>
+<td>{{acked}}</td>
+<td>{{failed}}</td>
+</tr>
+{{/executorStats}}
+</tbody>
+</table>
+</script>
+<script id="bolt-stats-template" type="text/html">
+<h2>Bolt stats</h2>
+<table class="zebra-striped" id="bolt-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The past period of time for which the statistics apply. Click on a value to set the window for this page.">Window</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span data-original-title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple." class="tip above">Execute latency (ms)</span></th><th class="header"><span class="tip above" title="The number of incoming Tuples processed.">Executed</span></th><th class="header"><span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple unti
 l a number of other Tuples have been received." class="tip above">Process latency (ms)</span></th><th class="header"><span data-original-title="The number of Tuples acknowledged by this Bolt." class="tip above">Acked</span></th><th class="header"><span data-original-title="The number of tuples Failed by this Bolt." class="tip left">Failed</span></th></tr></thead>
+<tbody>
+{{#boltStats}}
+<tr>
+<td><a href="/component.html?id={{id}}&topology_id={{topologyId}}&window={{window}}">{{windowPretty}}</td>
+<td>{{emitted}}</td>
+<td>{{transferred}}</td>
+<td>{{executeLatency}}</td>
+<td>{{executed}}</td>
+<td>{{processLatency}}</td>
+<td>{{acked}}</td>
+<td>{{failed}}</td>
+</tr>
+{{/boltStats}}
+</tbody>
+</script>
+<script id="bolt-input-stats-template" type="text/html">
+<h2>Input stats ({{windowHint}})</h2>
+<table class="zebra-striped" id="bolt-input-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The ID assigned to a the Component by the Topology.">Component</span></th><th class="header"><span class="tip right" title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given.">Stream</span></th><th class="header"><span class="tip above" title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple.">Execute latency (ms)</span></th><th class="header"><span class="tip above" title="The number of incoming Tuples processed.">Executed</span></th><th class="header"><span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received." class="tip above">Process latency (ms)</span></th><th class="header"><span class="ti
 p above" title="The number of Tuples acknowledged by this Bolt.">Acked</span></th><th class="header"><span data-original-title="The number of tuples Failed by this Bolt." class="tip left">Failed</span></th></tr></thead>
+<tbody>
+{{#inputStats}}
+<tr>
+<td>{{component}}</td>
+<td>{{stream}}</td>
+<td>{{executeLatency}}</td>
+<td>{{executed}}</td>
+<td>{{processLatency}}</td>
+<td>{{acked}}</td>
+<td>{{failed}}</td>
+</tr>
+{{/inputStats}}
+</tbody>
+</table>
+</script>
+<script id="bolt-output-stats-template" type="text/html">
+<h2>Output stats ({{windowHint}})</h2>
+<table class="zebra-striped" id="bolt-output-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given.">Stream</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th></tr></thead>
+<tbody>
+{{#outputStats}}
+<tr>
+<td>{{stream}}</td>
+<td>{{emitted}}</td>
+<td>{{transferred}}</td>
+</tr>
+{{/outputStats}}
+</tbody>
+</table>
+</script>
+<script id="bolt-executor-template" type="text/html">
+<h2>Executors</h2>
+<table class="zebra-striped" id="bolt-executor-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The unique executor ID.">Id</span></th><th class="header"><span data-original-title="The length of time an Executor (thread) has been alive." class="tip right">Uptime</span></th><th class="header"><span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">Host</span></th><th class="header"><span class="tip above" title="The port number used by the Worker to which an Executor is assigned. Click on the port number to open the logviewer page for this Worker.">Port</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span class="tip above" title="If thi
 s is around 1.0, the corresponding Bolt is running as fast as it can, so you may want to increase the Bolt's parallelism. This is (number executed * average execute latency) / measurement time.">Capacity (last 10m)</span></th><th class="header"><span data-original-title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple." class="tip above">Execute latency (ms)</span></th><th class="header"><span class="tip above" title="The number of incoming Tuples processed.">Executed</span></th><th class="header"><span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received." class="tip above">Process latency (ms)</span></th><th class="header"><span data-original-title="The number of Tuples acknowledged by this Bolt." class="tip above">Acked</span></th><th class="header"><span data-ori
 ginal-title="The number of tuples Failed by this Bolt." class="tip left">Failed</span></th></tr></thead>
+<tbody>
+{{#executorStats}}
+<tr>
+<td>{{id}}}</td>
+<td>{{uptime}}</td>
+<td>{{host}}</td>
+<td><a href="{{workerLogLink}}">{{port}}</a></td>
+<td>{{emitted}}</td>
+<td>{{transferred}}</td>
+<td>{{capacity}}</td>
+<td>{{executeLatency}}</td>
+<td>{{executed}}</td>
+<td>{{processLatency}}</td>
+<td>{{acked}}</td>
+<td>{{failed}}</td>
+</tr>
+{{/executorStats}}
+</tbody>
+</table>
+</script>
+
+<script id="component-errors-template" type="text/html">
+<h2>Errors</h2>
+<table class="zebra-striped" id="component-errors-table"><thead><tr><th>Time</th><th>Error</th></tr></thead>
+<tbody>
+{{#componentErrors}}
+<tr>
+<td>{{time}}</td>
+<td>{{error}}</td>
+</tr>
+{{/componentErrors}}
+</tbody>
+</table>
+</script>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/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
new file mode 100644
index 0000000..128f2d0
--- /dev/null
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -0,0 +1,62 @@
+<script id="cluster-summary-template" type="text/html">
+<table id="cluster-summary-table"><thead><tr><th><span class="tip right" title="The version of storm installed on the UI node. (Hopefully, this is the same on all storm nodes!)">Version</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></th><th><span class="tip above" title="Slots are Workers (processes).">Used slots</span></th><th><span class="tip above" title="Slots are Workers (processes).">Free slots</span></th><th><span class="tip above" title="Slots are Workers (processes).">Total slots</span></th><th><span class="tip above" title="Executors are threads in a Worker process.">Executors</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.">Tasks</span></th></tr></thead>
+<tbody>
+<tr>
+  <td>{{stormVersion}}</td>
+  <td>{{nimbusUptime}}</td>
+  <td>{{supervisors}}</td>
+  <td>{{slotsUsed}}</td>
+  <td>{{slotsFree}}</td>
+  <td>{{slotsTotal}}</td>
+  <td>{{executorsTotal}}</td>
+  <td>{{tasksTotal}}</td>
+</tr>
+</tbody>
+</table>
+</script>
+<script id="topology-summary-template" type="text/html">
+<table class="zebra-striped" id="topology-summary-table">
+<thead><tr><th><span class="tip right" title="The name given to the topology by when it was submitted. Click the name to view the Topology's information.">Name</span></th><th><span class="tip right" title="The unique ID given to a Topology each time it is launched.">Id</span></th><th><span class="tip above" title="The status can be one of ACTIVE, INACTIVE, KILLED, or REBALANCING.">Status</span></th><th><span class="tip above" title="The time since the Topology was submitted.">Uptime</span></th><th><span class="tip above" title="The number of Workers (processes).">Num workers</span></th><th><span class="tip above" title="Executors are threads in a Worker process.">Num executors</span></th><th><span class="tip above" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">Num tasks</span></th></tr></thead>
+<tbody>
+{{#topologies}}
+<tr>
+  <td><a href="/topology.html?id={{id}}">{{name}}</a></td>
+  <td>{{id}}</td>
+  <td>{{status}}</td>
+  <td>{{uptime}}</td>
+  <td>{{tasksTotal}}</td>
+  <td>{{workersTotal}}</td>
+  <td>{{executorsTotal}}</td>
+</tr>
+{{/topologies}}
+</tbody>
+</table>
+</script>
+<script id="supervisor-summary-template" type="text/html">
+<table class="zebra-striped" id="supervisor-summary-table"><thead><tr><th><span class="tip right" title="A unique identifier given to a Supervisor when it joins the cluster.">Id</span></th><th><span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">Host</span></th><th><span class="tip above" title="The length of time a Supervisor has been registered to the cluster.">Uptime</span></th><th><span class="tip above" title="Slots are Workers (processes).">Slots</span></th><th><span class="tip left" title="Slots are Workers (processes).">Used slots</span></th></tr></thead>
+<tbody>
+{{#supervisors}}
+<tr>
+  <td>{{id}}</td>
+  <td>{{host}}</td>
+  <td>{{uptime}}</td>
+  <td>{{slotsTotal}}</td>
+  <td>{{slotsUsed}}</td>
+</tr>
+{{/supervisors}}
+</tbody>
+</table>
+</script>
+
+<script id="configuration-template" type="text/html">
+<table class="zebra-striped" id="nimbus-configuration-table"><thead><tr><th>Key</th><th>Value</th></tr></thead>
+<tbody>
+{{#config}}
+<tr>
+<td>{{key}}</td>
+<td>{{value}}</td>
+</tr>
+{{/config}}
+</tbody>
+</table>
+</script>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/templates/json-error-template.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/templates/json-error-template.html b/storm-core/src/ui/public/templates/json-error-template.html
new file mode 100644
index 0000000..d797726
--- /dev/null
+++ b/storm-core/src/ui/public/templates/json-error-template.html
@@ -0,0 +1,4 @@
+<script id="json-error-template" type="text/html">
+<h2>{{error}}</h2>
+<pre>{{errorMessage}}</pre>
+</script>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/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
new file mode 100644
index 0000000..da58f9d
--- /dev/null
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@ -0,0 +1,128 @@
+<script id="topology-summary-template" type="text/html">
+  <table id="topology-summary-table">
+    <thead><tr><th><span class="tip right" title="The name given to the topology by when it was submitted.">Name</span></th><th><span class="tip right" title="The unique ID given to a Topology each time it is launched.">Id</span></th><th><span class="tip above" title="The status can be one of ACTIVE, INACTIVE, KILLED, or REBALANCING.">Status</span></th><th><span class="tip above" title="The time since the Topology was submitted.">Uptime</span></th><th><span class="tip above" title="The number of Workers (processes).">Num workers</span></th><th><span class="tip above" title="Executors are threads in a Worker process.">Num executors</span></th><th><span class="tip above" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">Num tasks</span></th></tr></thead>
+    <tbody>
+      <tr>
+        <td>{{name}}</td>
+        <td>{{id}}</td>
+        <td>{{status}}</td>
+        <td>{{uptime}}</td>
+        <td>{{tasksTotal}}</td>
+        <td>{{workersTotal}}</td>
+        <td>{{executorsTotal}}</td>
+      </tr>
+    </tbody>
+  </table>
+</script>
+<script id="topology-stats-template" type="text/html">
+  <h2>Topology stats</h2>
+  <table class="zebra-striped" id="topology-stats-table">
+    <thead><tr><th><span class="tip right" title="The past period of time for which the statistics apply. Click on a value to set the window for this page.">Window</span></th><th><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th><span class="tip above" title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">Complete latency (ms)</span></th><th><span class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">Acked</span></th><th><span class="tip left" title="The number of Tuple &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done.">Failed</span></th></tr></thead>
+    <tbody>
+      {{#topologyStats}}
+      <tr>
+        <td><a href="/topology.html?id={{id}}&window={{window}}">{{windowPretty}}</td>
+        <td>{{emitted}}</td>
+        <td>{{transferred}}</td>
+        <td>{{completeLatency}}</td>
+        <td>{{acked}}</td>
+        <td>{{failed}}</td>
+      </tr>
+      {{/topologyStats}}
+    </tbody>
+  </table>
+</script>
+<script id="topology-visualization-template" type="text/html">
+  <h2>Topology Visualization</h2>
+  <input type="button" id="show-hide-visualization" value="Show Visualization"/>
+  <p>
+    <div id="visualization-container" style="display:none;">
+      <p>
+        <table class="zebra-striped">
+          <thead>
+            <tr>
+              <th class="header" colspan=4>
+                Streams
+              </th>
+            </tr>
+          </thead>
+          {{#visualizationTable}}
+          <tr>
+            {{#:row}}
+            <td>
+              <input type="checkbox" id={{:sani-stream}} class="stream-box" {{#:checked}}checked{{/:checked}}/>
+              {{:stream}}
+            </td>
+            {{/:row}}
+          </tr>
+          {{/visualizationTable}}
+        </table>
+      </p>
+      <canvas id="topoGraph" width=1024 height=768 style="border:1px solid #000000;">
+    </div>
+  </p>
+</script>
+
+<script id="topology-configuration-template" type="text/html">
+  <h2>Topology Configuration</h2>
+  <table class="zebra-striped" id="topology-configuration-table"><thead><tr><th>Key</th><th>Value</th></tr></thead>
+    <tbody>
+      {{#config}}
+      <tr>
+        <td>{{key}}</td>
+        <td>{{value}}</td>
+      </tr>
+      {{/config}}
+    </tbody>
+  </table>
+</script>
+<script id="spout-stats-template" type="text/html">
+  <h2>Spouts ({{windowHint}})</h2>
+  <table class="zebra-striped" id="spout-stats-table">
+    <thead>
+      <tr><th class="header headerSortDown"><span data-original-title="The ID assigned to a the Component by the Topology. Click on the name to view the Component's page." class="tip right">Id</span></th><th class="header"><span data-original-title="Executors are threads in a Worker process." class="tip right">Executors</span></th><th class="header"><span class="tip above" 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><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span class="tip above" title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">Complete latency (ms)</span></th><th class="header"><span 
 class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">Acked</span></th><th class="header"><span class="tip above" title="The number of Tuple &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done.">Failed</span></th><th class="header">Last error</th>
+      </tr>
+    </thead>
+    <tbody>
+      {{#spouts}}
+      <tr>
+        <td><a href="/component.html?id={{spoutId}}&topology_id={{id}}">{{spoutId}}</a></td>
+        <td>{{executors}}</td>
+        <td>{{tasks}}</td>
+        <td>{{emitted}}</td>
+        <td>{{transferred}}</td>
+        <td>{{completeLatency}}</td>
+        <td>{{acked}}</td>
+        <td>{{failed}}</td>
+        <td>{{lastError}}</td>
+        {{/spouts}}
+    </tbody>
+  </table>
+</script>
+<script id="bolt-stats-template" type="text/html">
+  <h2>Bolts ({{windowHint}})</h2>
+  <table class="zebra-striped" id="bolt-stats-table"><thead>
+      <tr><th class="header headerSortDown"><span class="tip right" title="The ID assigned to a the Component by the Topology. Click on the name to view the Component's page.">Id</span></th><th class="header"><span data-original-title="Executors are threads in a Worker process." class="tip right">Executors</span></th><th class="header"><span class="tip above" 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><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span data-original-title="If this is around 1.0, the corresponding Bolt is running as fast as it can, so you may want to increase the Bolt's parallelism. This is (number executed * average execute latency) / measurement time." class="tip abov
 e">Capacity (last 10m)</span></th><th class="header"><span class="tip above" title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple.">Execute latency (ms)</span></th><th class="header"><span class="tip above" title="The number of incoming Tuples processed.">Executed</span></th><th class="header"><span class="tip above" title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received.">Process latency (ms)</span></th><th class="header"><span class="tip above" title="The number of Tuples acknowledged by this Bolt.">Acked</span></th><th class="header"><span class="tip left" title="The number of tuples Failed by this Bolt.">Failed</span></th><th class="header">Last error</th>
+    </tr></thead>
+    <tbody>
+      {{#bolts}}
+      <tr>
+        <td><a href="/component.html?id={{boltId}}&topology_id={{id}}">{{boltId}}</a></td>
+        <td>{{executors}}</td>
+        <td>{{tasks}}</td>
+        <td>{{emitted}}</td>
+        <td>{{transferred}}</td>
+        <td>{{capacity}}</td>
+        <td>{{executeLatency}}</td>
+        <td>{{executed}}</td>
+        <td>{{processLatency}}</td>
+        <td>{{acked}}</td>
+        <td>{{failed}}</td>
+        <td>{{lastError}}</td>
+        {{/bolts}}
+    </tbody>
+</script>
+
+<script id="topology-actions-template" type="text/html">
+  <input {{activateStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'activate', false, 0)" type="button" value="Activate"><input {{deactivateStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'deactivate', false, 0)" type="button" value="Deactivate"><input {{rebalanceStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'rebalance', true, {{msgTimeout}})" type="button" value="Rebalance"><input {{killStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'kill', true, 30)" type="button" value="Kill">
+</script>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html
new file mode 100644
index 0000000..df095ad
--- /dev/null
+++ b/storm-core/src/ui/public/topology.html
@@ -0,0 +1,90 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01//EN" "http://www.w3.org/TR/html4/strict.dtd">
+<html><head>
+<title>Storm UI</title>
+<link href="/css/bootstrap-1.4.0.css" rel="stylesheet" type="text/css">
+<link href="/css/style.css" rel="stylesheet" type="text/css">
+<script src="/js/jquery-1.6.2.min.js" type="text/javascript"></script>
+<script src="/js/jquery.tablesorter.min.js" type="text/javascript"></script>
+<script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
+<script src="/js/jquery.mustache.js" type="text/javascript"></script>
+<script src="/js/purl.js" type="text/javascript"></script>
+<script src="/js/bootstrap-twipsy.js" type="text/javascript"></script>
+<script src="/js/script.js" type="text/javascript"></script>
+<script src="/js/visualization.js" type="text/javascript"></script>
+<script src="/js/arbor.js" type="text/javascript"></script>
+<script src="/js/arbor-graphics.js" type="text/javascript"></script>
+</head>
+<body>
+<h1><a href="/">Storm UI</a></h1>
+<h2>Topology summary</h2>
+<div id="topology-summary">
+</div>
+<div id="topology-actions">
+<h2 class="js-only">Topology actions</h2>
+<p id="topology-actions" class="js-only">
+</p>
+</div>
+<div id="topology-stats"></div>
+<div id="spout-stats">
+</div>
+<div id="bolt-stats">
+</div>
+<div id="topology-visualization">
+</div>
+<div id="topology-configuration">
+</div>
+<p id="toggle-switch" style="display: block;" class="js-only"></p>
+<div id="json-response-error">
+</div>
+</body>
+<script>
+$(document).ready(function() {
+    var topologyId = $.url().param("id");
+    var window = $.url().param("window");
+    var sys = $.cookies.get("sys") || "false";
+    var url = "/api/v1/topology/"+topologyId+"?sys="+sys;
+    if(window) url += "&window="+window;
+    renderToggleSys($("#toggle-switch"));
+    $.ajaxSetup({
+        "error":function(jqXHR,textStatus,response) {
+            var errorJson = jQuery.parseJSON(jqXHR.responseText);
+            $.get("/templates/json-error-template.html", function(template) {
+                $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),errorJson));
+            });
+        }
+    });
+
+    $.getJSON(url,function(response,status,jqXHR) {
+        var topologySummary = $("#topology-summary");
+        var topologyStats = $("#topology-stats");
+        var spoutStats = $("#spout-stats");
+        var boltStats = $("#bolt-stats");
+        var config = $("#topology-configuration");
+        var topologyActions = $("#topology-actions");
+        var topologyVisualization = $("#topology-visualization")
+        var formattedConfig = formatConfigData(response["configuration"]);
+        var buttonJsonData = topologyActionJson(response["id"],response["name"],response["status"],response["msgTimeout"]);
+        $.get("/templates/topology-page-template.html", function(template) {
+            topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));
+            topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData));
+            topologyStats.append(Mustache.render($(template).filter("#topology-stats-template").html(),response));
+            $("#topology-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}});
+            spoutStats.append(Mustache.render($(template).filter("#spout-stats-template").html(),response));
+            if(response["spouts"].length > 0) {
+                $("#spout-stats-table").tablesorter({sortList: [[0,0]], headers:{}});
+            }
+            boltStats.append(Mustache.render($(template).filter("#bolt-stats-template").html(),response));
+            if(response["bolts"].length > 0) {
+                $("#bolt-stats-table").tablesorter({sortList: [[0,0]], headers:{}});
+            }
+
+            topologyVisualization.append(Mustache.render($(template).filter("#topology-visualization-template").html(), response));
+            $("#show-hide-visualization").click(function () { show_visualization(null) });
+
+            config.append(Mustache.render($(template).filter("#topology-configuration-template").html(),formattedConfig));
+            $("#topology-configuration-table").tablesorter({ sortList: [[0,0]], headers: {}});
+        });
+    });
+ });
+</script>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/test/clj/backtype/storm/drpc_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/drpc_test.clj b/storm-core/test/clj/backtype/storm/drpc_test.clj
index fbc60e6..6d0ba2b 100644
--- a/storm-core/test/clj/backtype/storm/drpc_test.clj
+++ b/storm-core/test/clj/backtype/storm/drpc_test.clj
@@ -22,9 +22,11 @@
   (:import [backtype.storm LocalDRPC LocalCluster])
   (:import [backtype.storm.tuple Fields])
   (:import [backtype.storm.generated DRPCExecutionException])
-  (:use [backtype.storm bootstrap testing])
-  (:use [backtype.storm.daemon common])
+  (:import [java.util.concurrent ConcurrentLinkedQueue])
+  (:use [backtype.storm bootstrap config testing])
+  (:use [backtype.storm.daemon common drpc])
   (:use [backtype.storm clojure])
+  (:use [conjure core])
   )
 
 (bootstrap)
@@ -218,3 +220,24 @@
     (.shutdown cluster)
     (.shutdown drpc)
     ))
+
+(deftest test-dequeue-req-after-timeout
+  (let [queue (ConcurrentLinkedQueue.)
+        delay-seconds 2]
+    (stubbing [acquire-queue queue
+               read-storm-config {DRPC-REQUEST-TIMEOUT-SECS delay-seconds}]
+      (let [drpc-handler (service-handler)]
+        (is (thrown? DRPCExecutionException
+          (.execute drpc-handler "ArbitraryDRPCFunctionName" "")))
+        (is (= 0 (.size queue)))))))
+
+(deftest test-drpc-timeout-cleanup 
+  (let [queue (ConcurrentLinkedQueue.)
+        delay-seconds 1]
+    (stubbing [acquire-queue queue
+               read-storm-config {DRPC-REQUEST-TIMEOUT-SECS delay-seconds}
+               timeout-check-secs delay-seconds]
+              (let [drpc-handler (service-handler)]
+                (is (thrown? DRPCExecutionException 
+                             (.execute drpc-handler "ArbitraryDRPCFunctionName" "no-args")))))))
+

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/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 f271607..ea7b8dc 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
@@ -37,7 +37,8 @@
         server (.bind context nil port)
         client (.connect context nil "localhost" port)
         _ (.send client task (.getBytes req_msg))
-        resp (.recv server 0)]
+        iter (.recv server 0 0)
+        resp (.next iter)]
     (is (= task (.task resp)))
     (is (= req_msg (String. (.message resp))))
     (.close client)
@@ -58,7 +59,8 @@
         server (.bind context nil port)
         client (.connect context nil "localhost" port)
         _ (.send client task (.getBytes req_msg))
-        resp (.recv server 0)]
+        iter (.recv server 0 0)
+        resp (.next iter)]
     (is (= task (.task resp)))
     (is (= req_msg (String. (.message resp))))
     (.close client)
@@ -77,15 +79,23 @@
                     }
         context (TransportFactory/makeContext storm-conf)
         client (.connect context nil "localhost" port)
+        
+        server (Thread.
+                (fn []
+                  (Thread/sleep 1000)
+                  (let [server (.bind context nil port)
+                        iter (.recv server 0 0)
+                        resp (.next iter)]
+                    (is (= task (.task resp)))
+                    (is (= req_msg (String. (.message resp))))
+                    (.close server) 
+                  )))
+        _ (.start server)
         _ (.send client task (.getBytes req_msg))
-        _ (Thread/sleep 1000)
-        server (.bind context nil port)
-        resp (.recv server 0)]
-    (is (= task (.task resp)))
-    (is (= req_msg (String. (.message resp))))
+        ]
     (.close client)
-    (.close server)
-    (.term context)))    
+    (.join server)
+    (.term context)))
 
 (deftest test-batch
   (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
@@ -102,11 +112,21 @@
     (doseq [num  (range 1 100000)]
       (let [req_msg (str num)]
         (.send client task (.getBytes req_msg))))
-    (doseq [num  (range 1 100000)]
+    
+    (let [resp (ArrayList.)
+          received (atom 0)]
+      (while (< @received (- 100000 1))
+        (let [iter (.recv server 0 0)]
+          (while (.hasNext iter)
+            (let [msg (.next iter)]
+              (.add resp msg)
+              (swap! received inc)
+              ))))
+      (doseq [num  (range 1 100000)]
       (let [req_msg (str num)
-            resp (.recv server 0)
-            resp_msg (String. (.message resp))]
-        (is (= req_msg resp_msg))))
+            resp_msg (String. (.message (.get resp (- num 1))))]
+        (is (= req_msg resp_msg)))))
+   
     (.close client)
     (.close server)
     (.term context)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/test/clj/backtype/storm/messaging_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/messaging_test.clj b/storm-core/test/clj/backtype/storm/messaging_test.clj
index 94b9168..c719c68 100644
--- a/storm-core/test/clj/backtype/storm/messaging_test.clj
+++ b/storm-core/test/clj/backtype/storm/messaging_test.clj
@@ -15,7 +15,7 @@
 ;; limitations under the License.
 (ns backtype.storm.messaging-test
   (:use [clojure test])
-  (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount])
+  (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestEventLogSpout TestEventOrderCheckBolt])
   (:use [backtype.storm bootstrap testing])
   (:use [backtype.storm.daemon common])
   )
@@ -56,3 +56,36 @@
         (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
                  (read-tuples results "2")))))))
 
+(extend-type TestEventLogSpout
+  CompletableSpout
+  (exhausted? [this]
+    (-> this .completed))
+  (cleanup [this]
+    (.cleanup this))
+  (startup [this]
+    ))
+
+;; Test Adding more receiver threads won't violate the message delivery order gurantee
+(deftest test-receiver-message-order 
+  (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 2
+                                        :daemon-conf {TOPOLOGY-WORKERS 2
+                                                      ;; Configure multiple receiver threads per worker 
+                                                      WORKER-RECEIVER-THREAD-COUNT 2
+                                                      STORM-LOCAL-MODE-ZMQ  true 
+                                                      STORM-MESSAGING-TRANSPORT 
+                                                      "backtype.storm.messaging.netty.Context"}]
+      (let [topology (thrift/mk-topology
+                       
+                       ;; TestEventLogSpout output(sourceId, eventId), eventId is Monotonically increasing
+                       {"1" (thrift/mk-spout-spec (TestEventLogSpout. 4000) :parallelism-hint 8)}
+                       
+                       ;; field grouping, message from same "source" task will be delivered to same bolt task
+                       ;; When received message order is not kept, Emit an error Tuple 
+                       {"2" (thrift/mk-bolt-spec {"1" ["source"]} (TestEventOrderCheckBolt.)
+                                                 :parallelism-hint 4)
+                        })
+            results (complete-topology cluster
+                                       topology)]
+        
+        ;; No error Tuple from Bolt TestEventOrderCheckBolt
+        (is (empty? (read-tuples results "2"))))))


[09/13] Merge branch 'master' into idiomatic-clojure-01

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/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 6da383e..d8ad29a 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -20,7 +20,8 @@
   (:use [hiccup core page-helpers])
   (:use [backtype.storm config util log])
   (:use [backtype.storm.ui helpers])
-  (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID system-id?]]])
+  (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID
+                                              ACKER-ACK-STREAM-ID ACKER-FAIL-STREAM-ID system-id?]]])
   (:use [ring.adapter.jetty :only [run-jetty]])
   (:use [clojure.string :only [trim]])
   (:import [backtype.storm.utils Utils])
@@ -51,93 +52,8 @@
        (map #(.get_stats ^ExecutorSummary %))
        (filter not-nil?)))
 
-(def tips
-  "Defines a mapping of help texts for elements of the UI pages."
-  {:sys-stats "Use this to toggle inclusion of storm system components."
-   :version (str "The version of storm installed on the UI node. (Hopefully, "
-                 "this is the same on all storm nodes!)")
-   :nimbus-uptime (str "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.)")
-   :num-supervisors "The number of nodes in the cluster currently."
-   :num-slots "Slots are Workers (processes)."
-   :num-execs "Executors are threads in a Worker process."
-   :num-tasks (str "A Task is an instance of a Bolt or Spout. The number of "
-                   "Tasks is almost always equal to the number of Executors.")
-   :name "The name given to the topology by when it was submitted."
-   :name-link "Click the name to view the Topology's information."
-   :topo-id "The unique ID given to a Topology each time it is launched."
-   :status "The status can be one of ACTIVE, INACTIVE, KILLED, or REBALANCING."
-   :topo-uptime "The time since the Topology was submitted."
-   :num-workers "The number of Workers (processes)."
-   :sup-id (str "A unique identifier given to a Supervisor when it joins the "
-                "cluster.")
-   :sup-host (str "The hostname reported by the remote host. (Note that this "
-                  "hostname is not the result of a reverse lookup at the "
-                  "Nimbus node.)")
-   :sup-uptime (str "The length of time a Supervisor has been registered to the "
-                    "cluster.")
-   :window (str "The past period of time for which the statistics apply. "
-                "Click on a value to set the window for this page.")
-   :emitted "The number of Tuples emitted."
-   :transferred "The number of Tuples emitted that sent to one or more bolts."
-   :complete-lat (str "The average time a Tuple \"tree\" takes to be completely "
-                      "processed by the Topology. A value of 0 is expected "
-                      "if no acking is done.")
-   :spout-acked (str "The number of Tuple \"trees\" successfully processed. A "
-                     "value of 0 is expected if no acking is done.")
-   :spout-failed (str "The number of Tuple \"trees\" that were explicitly "
-                      "failed or timed out before acking was completed. A value "
-                      "of 0 is expected if no acking is done.")
-   :comp-id "The ID assigned to a the Component by the Topology."
-   :comp-id-link "Click on the name to view the Component's page."
-   :capacity (str "If this is around 1.0, the corresponding Bolt is running as "
-                  "fast as it can, so you may want to increase the Bolt's "
-                  "parallelism. This is (number executed * average execute "
-                  "latency) / measurement time.")
-   :exec-lat (str "The average time a Tuple spends in the execute method. The "
-                  "execute method may complete without sending an Ack for the "
-                  "tuple.")
-   :num-executed "The number of incoming Tuples processed."
-   :proc-lat (str "The average time it takes to Ack a Tuple after it is first "
-                  "received.  Bolts that join, aggregate or batch may not Ack a "
-                  "tuple until a number of other Tuples have been received.")
-   :bolt-acked "The number of Tuples acknowledged by this Bolt."
-   :bolt-failed "The number of tuples Failed by this Bolt."
-   :stream (str "The name of the Tuple stream given in the Topolgy, or \""
-                Utils/DEFAULT_STREAM_ID "\" if none was given.")
-   :exec-id "The unique executor ID."
-   :exec-uptime "The length of time an Executor (thread) has been alive."
-   :port (str "The port number used by the Worker to which an Executor is "
-              "assigned. Click on the port number to open the logviewer page "
-              "for this Worker.")})
-
-(defn mk-system-toggle-button
-  [include-sys?]
-  [:p {:class "js-only"}
-   [:span.tip.right {:title (:sys-stats tips)}
-    [:input {:type "button"
-             :value (str (if include-sys? "Hide" "Show") " System Stats")
-             :onclick "toggleSys()"}]]])
-
-(defn ui-template
-  [body]
-  (html4
-    [:head
-     [:title "Storm UI"]
-     (include-css "/css/bootstrap-1.4.0.css")
-     (include-css "/css/style.css")
-     (include-js "/js/jquery-1.6.2.min.js")
-     (include-js "/js/jquery.tablesorter.min.js")
-     (include-js "/js/jquery.cookies.2.2.0.min.js")
-     (include-js "/js/bootstrap-twipsy.js")
-     (include-js "/js/script.js")]
-    [:body
-     [:h1 (link-to "/" "Storm UI")]
-     (seq body)]))
-
 (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)
@@ -146,117 +62,12 @@
       (trim (slurp release-path))
       "Unknown")))
 
-(defn cluster-summary-table
-  [^ClusterSummary summ]
-  (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 +))]
-    (table [{:text "Version" :attr {:class "tip right"
-                                    :title (:version tips)}}
-            {:text "Nimbus uptime" :attr {:class "tip right"
-                                          :title (:nimbus-uptime tips)}}
-            {:text "Supervisors" :attr {:class "tip above"
-                                        :title (:num-supervisors tips)}}
-            {:text "Used slots" :attr {:class "tip above"
-                                       :title (:num-slots tips)}}
-            {:text "Free slots" :attr {:class "tip above"
-                                       :title (:num-slots tips)}}
-            {:text "Total slots" :attr {:class "tip above"
-                                        :title (:num-slots tips)}}
-            {:text  "Executors" :attr {:class "tip above"
-                                       :title (:num-execs tips)}}
-            {:text "Tasks" :attr {:class "tip left"
-                                  :title (:num-tasks tips)}}]
-           [[(read-storm-version)
-             (pretty-uptime-sec (.get_nimbus_uptime_secs summ))
-             (count sups)
-             used-slots
-             free-slots
-             total-slots
-             total-executors
-             total-tasks]])))
-
-(defn topology-link
-  ([id] (topology-link id id))
-  ([id content]
-   (link-to (url-format "/topology/%s" id) (escape-html content))))
-
-(defn main-topology-summary-table
-  [summs]
-  (sorted-table
-    [{:text "Name" :attr {:class "tip right"
-                          :title (str (:name tips) " " (:name-link tips))}}
-     {:text "Id" :attr {:class "tip right"
-                        :title (:topo-id tips)}}
-     {:text "Status" :attr {:class "tip above"
-                            :title (:status tips)}}
-     {:text "Uptime" :attr {:class "tip above"
-                            :title (:topo-uptime tips)}}
-     {:text "Num workers" :attr {:class "tip above"
-                                 :title (:num-workers tips)}}
-     {:text "Num executors" :attr {:class "tip above"
-                                   :title (:num-execs tips)}}
-     {:text "Num tasks" :attr {:class "tip above"
-                               :title (:num-tasks tips)}}]
-    (for [^TopologySummary t summs]
-      [(topology-link (.get_id t) (.get_name t))
-       (escape-html (.get_id t))
-       (.get_status t)
-       (pretty-uptime-sec (.get_uptime_secs t))
-       (.get_num_workers t)
-       (.get_num_executors t)
-       (.get_num_tasks t)])
-    :time-cols [3]
-    :sort-list "[[0,0]]"))
-
-(defn supervisor-summary-table
-  [summs]
-  (sorted-table
-    [{:text "Id" :attr {:class "tip right"
-                        :title (:sup-id tips)}}
-     {:text "Host" :attr {:class "tip above"
-                          :title (:sup-host tips)}}
-     {:text "Uptime" :attr {:class "tip above"
-                            :title (:sup-uptime tips)}}
-     {:text "Slots" :attr {:class "tip above"
-                           :title (:num-slots tips)}}
-     {:text "Used slots" :attr {:class "tip left"
-                                :title (:num-slots tips)}}]
-    (for [^SupervisorSummary s summs]
-      [(.get_supervisor_id s)
-       (.get_host s)
-       (pretty-uptime-sec (.get_uptime_secs s))
-       (.get_num_workers s)
-       (.get_num_used_workers s)])
-    :time-cols [2]))
-
-(defn configuration-table
-  [conf]
-  (sorted-table ["Key" "Value"]
-                (map #(vector (key %) (str (val %))) conf)))
-
-(defn main-page
-  []
-  (with-nimbus nimbus
-               (let [summ (.getClusterInfo ^Nimbus$Client nimbus)]
-                 (concat
-                   [[:h2 "Cluster Summary"]]
-                   [(cluster-summary-table summ)]
-                   [[:h2 "Topology summary"]]
-                   (main-topology-summary-table (.get_topologies summ))
-                   [[:h2 "Supervisor summary"]]
-                   (supervisor-summary-table (.get_supervisors summ))
-                   [[:h2 "Nimbus Configuration"]]
-                   (configuration-table (from-json (.getNimbusConf ^Nimbus$Client nimbus)))))))
-
+;; TODO: What is the desired behavior if a particular component id
+;; corresponded to both a bolt and spout. Where should the error
+;; occur? -DCJ
 (defn component-type
+  "Returns the component type (either :bolt or :spout) for a given
+  topology and component id. Returns nil if not found."
   [^StormTopology topology id]
   (let [bolts (.get_bolts topology)
         spouts (.get_spouts topology)]
@@ -287,19 +98,14 @@
                        c]]
                      ))]))))
 
-
 (defn expand-averages-seq
   [average-seq counts-seq]
   (->> (map vector average-seq counts-seq)
        (map #(apply expand-averages %))
-       (apply merge-with
-              (fn [s1 s2]
-                (merge-with
-                  add-pairs
-                  s1
-                  s2)))))
+       (apply merge-with (fn [s1 s2] (merge-with add-pairs s1 s2)))))
 
-(defn- val-avg [[t c]]
+(defn- val-avg
+  [[t c]]
   (if (= t 0) 0
     (double (/ t c))))
 
@@ -341,6 +147,14 @@
     (fn [_] true)
     (fn [stream] (and (string? stream) (not (system-id? stream))))))
 
+(defn is-ack-stream
+  [stream]
+  (let [acker-streams
+        [ACKER-INIT-STREAM-ID
+         ACKER-ACK-STREAM-ID
+         ACKER-FAIL-STREAM-ID]]
+    (every? #(not= %1 stream) acker-streams)))
+
 (defn pre-process
   [stream-summary include-sys?]
   (let [filter-fn (mk-include-sys-fn include-sys?)
@@ -423,86 +237,6 @@
   [topology s]
   (= :bolt (executor-summary-type topology s)))
 
-(defn topology-summary-table
-  [^TopologyInfo summ]
-  (let [executors (.get_executors summ)
-        workers (set (for [^ExecutorSummary e executors] [(.get_host e) (.get_port e)]))]
-    (table [{:text "Name" :attr {:class "tip right"
-                                 :title (:name tips)}}
-            {:text "Id" :attr {:class "tip right"
-                               :title (:topo-id tips)}}
-            {:text "Status" :attr {:class "tip above"
-                                   :title (:status tips)}}
-            {:text "Uptime" :attr {:class "tip above"
-                                   :title (:topo-uptime tips)}}
-            {:text "Num workers" :attr {:class "tip above"
-                                        :title (:num-workers tips)}}
-            {:text "Num executors" :attr {:class "tip above"
-                                          :title (:num-execs tips)}}
-            {:text "Num tasks" :attr {:class "tip above"
-                                      :title (:num-tasks tips)}}]
-           [[(escape-html (.get_name summ))
-             (escape-html (.get_id summ))
-             (.get_status summ)
-             (pretty-uptime-sec (.get_uptime_secs summ))
-             (count workers)
-             (count executors)
-             (sum-tasks executors)]])))
-
-(defn total-aggregate-stats
-  [spout-summs bolt-summs include-sys?]
-  (let [spout-stats (get-filled-stats spout-summs)
-        bolt-stats (get-filled-stats bolt-summs)
-        agg-spout-stats (-> spout-stats
-                            (aggregate-spout-stats include-sys?)
-                            aggregate-spout-streams)
-        agg-bolt-stats (-> bolt-stats
-                           (aggregate-bolt-stats include-sys?)
-                           aggregate-bolt-streams)]
-    (merge-with
-      (fn [s1 s2]
-        (merge-with + s1 s2))
-      (select-keys agg-bolt-stats [:emitted :transferred])
-      agg-spout-stats)))
-
-(defn stats-times
-  [stats-map]
-  (sort-by #(Integer/parseInt %)
-           (-> stats-map
-               clojurify-structure
-               (dissoc ":all-time")
-               keys)))
-
-(defn topology-stats-table
-  [id window stats]
-  (let [times (stats-times (:emitted stats))
-        display-map (into {} (for [t times] [t pretty-uptime-sec]))
-        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
-    (sorted-table
-      [{:text "Window" :attr {:class "tip right"
-                              :title (:window tips)}}
-       {:text "Emitted" :attr {:class "tip above"
-                               :title (:emitted tips)}}
-       {:text "Transferred" :attr {:class "tip above"
-                                   :title (:transferred tips)}}
-       {:text "Complete latency (ms)" :attr {:class "tip above"
-                                             :title (:complete-lat tips)}}
-       {:text "Acked" :attr {:class "tip above"
-                             :title (:spout-acked tips)}}
-       {:text "Failed" :attr {:class "tip left"
-                              :title (:spout-failed tips)}}]
-      (for [k (concat times [":all-time"])
-            :let [disp ((display-map k) k)]]
-        [(link-to (if (= k window) {:class "red"} {})
-                  (url-format "/topology/%s?window=%s" id k)
-                  (escape-html disp))
-         (get-in stats [:emitted k])
-         (get-in stats [:transferred k])
-         (float-str (get-in stats [:complete-latencies k]))
-         (get-in stats [:acked k])
-         (get-in stats [:failed k])])
-      :time-cols [0])))
-
 (defn group-by-comp
   [summs]
   (let [ret (group-by #(.get_component_id ^ExecutorSummary %) summs)]
@@ -519,28 +253,23 @@
                    reverse
                    first)]
     (if error
-      [:span (if (< (time-delta (.get_error_time_secs ^ErrorInfo error))
-                    (* 60 30))
-               {:class "red"}
-               {})
-       (error-subset (.get_error ^ErrorInfo error))])))
-
-(defn component-link
-  [storm-id id]
-  (link-to (url-format "/topology/%s/component/%s" storm-id id) (escape-html id)))
-
-(defn worker-log-link
-  [host port]
-  (link-to (url-format "http://%s:%s/log?file=worker-%s.log"
-                       host (*STORM-CONF* LOGVIEWER-PORT) port) (str port)))
-
-(defn render-capacity
-  [capacity]
-  (let [capacity (nil-to-zero capacity)]
-    [:span (if (> capacity 0.9)
-             {:class "red"}
-             {})
-     (float-str capacity)]))
+      (error-subset (.get_error ^ErrorInfo error))
+      "")))
+
+(defn component-task-summs
+  [^TopologyInfo summ topology id]
+  (let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
+        bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
+        spout-comp-summs (group-by-comp spout-summs)
+        bolt-comp-summs (group-by-comp bolt-summs)
+        ret (if (contains? spout-comp-summs id)
+              (spout-comp-summs id)
+              (bolt-comp-summs id))]
+    (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret)))
+
+(defn worker-log-link [host port]
+  (url-format "http://%s:%s/log?file=worker-%s.log"
+              host (*STORM-CONF* LOGVIEWER-PORT) port))
 
 (defn compute-executor-capacity
   [^ExecutorSummary e]
@@ -565,91 +294,56 @@
        (map nil-to-zero)
        (apply max)))
 
-(defn spout-comp-table
-  [top-id summ-map errors window include-sys?]
-  (sorted-table
-    [{:text "Id" :attr {:class "tip right"
-                        :title (str (:comp-id tips) " " (:comp-id-link tips))}}
-     {:text "Executors" :attr {:class "tip right"
-                               :title (:num-execs tips)}}
-     {:text "Tasks" :attr {:class "tip above"
-                           :title (:num-tasks tips)}}
-     {:text "Emitted" :attr {:class "tip above"
-                             :title (:emitted tips)}}
-     {:text "Transferred" :attr {:class "tip above"
-                                 :title (:transferred tips)}}
-     {:text "Complete latency (ms)" :attr {:class "tip above"
-                                           :title (:complete-lat tips)}}
-     {:text "Acked" :attr {:class "tip above"
-                           :title (:spout-acked tips)}}
-     {:text "Failed" :attr {:class "tip above"
-                            :title (:spout-failed tips)}}
-     "Last error"]
-    (for [[id summs] summ-map
-          :let [stats-seq (get-filled-stats summs)
-                stats (aggregate-spout-streams
-                        (aggregate-spout-stats
-                          stats-seq include-sys?))]]
-      [(component-link top-id id)
-       (count summs)
-       (sum-tasks summs)
-       (get-in stats [:emitted window])
-       (get-in stats [:transferred window])
-       (float-str (get-in stats [:complete-latencies window]))
-       (get-in stats [:acked window])
-       (get-in stats [:failed window])
-       (most-recent-error (get errors id))])))
-
-(defn bolt-comp-table
-  [top-id summ-map errors window include-sys?]
-  (sorted-table
-    [{:text "Id" :attr {:class "tip right"
-                        :title (str (:comp-id tips) " " (:comp-id-link tips))}}
-     {:text "Executors" :attr {:class "tip right"
-                               :title (:num-execs tips)}}
-     {:text "Tasks" :attr {:class "tip above"
-                           :title (:num-tasks tips)}}
-     {:text "Emitted" :attr {:class "tip above"
-                             :title (:emitted tips)}}
-     {:text "Transferred" :attr {:class "tip above"
-                                 :title (:transferred tips)}}
-     {:text "Capacity (last 10m)" :attr {:class "tip above"
-                                         :title (:capacity tips)}}
-     {:text "Execute latency (ms)" :attr {:class "tip above"
-                                          :title (:exec-lat tips)}}
-     {:text "Executed" :attr {:class "tip above"
-                              :title (:num-executed tips)}}
-     {:text "Process latency (ms)":attr {:class "tip above"
-                                         :title (:proc-lat tips)}}
-     {:text "Acked" :attr {:class "tip above"
-                           :title (:bolt-acked tips)}}
-     {:text "Failed" :attr {:class "tip left"
-                            :title (:bolt-failed tips)}}
-     "Last error"]
-    (for [[id summs] summ-map
-          :let [stats-seq (get-filled-stats summs)
-                stats (aggregate-bolt-streams
-                        (aggregate-bolt-stats
-                          stats-seq include-sys?))]]
-      [(component-link top-id id)
-       (count summs)
-       (sum-tasks summs)
-       (get-in stats [:emitted window])
-       (get-in stats [:transferred window])
-       (render-capacity (compute-bolt-capacity summs))
-       (float-str (get-in stats [:execute-latencies window]))
-       (get-in stats [:executed window])
-       (float-str (get-in stats [:process-latencies window]))
-       (get-in stats [:acked window])
-       (get-in stats [:failed window])
-       (most-recent-error (get errors id))])))
-
-(defn window-hint [window]
+(defn spout-streams-stats
+  [summs include-sys?]
+  (let [stats-seq (get-filled-stats summs)]
+    (aggregate-spout-streams
+      (aggregate-spout-stats
+        stats-seq include-sys?))))
+
+(defn bolt-streams-stats
+  [summs include-sys?]
+  (let [stats-seq (get-filled-stats summs)]
+    (aggregate-bolt-streams
+      (aggregate-bolt-stats
+        stats-seq include-sys?))))
+
+(defn total-aggregate-stats
+  [spout-summs bolt-summs include-sys?]
+  (let [spout-stats (get-filled-stats spout-summs)
+        bolt-stats (get-filled-stats bolt-summs)
+        agg-spout-stats (-> spout-stats
+                            (aggregate-spout-stats include-sys?)
+                            aggregate-spout-streams)
+        agg-bolt-stats (-> bolt-stats
+                           (aggregate-bolt-stats include-sys?)
+                           aggregate-bolt-streams)]
+    (merge-with
+      (fn [s1 s2]
+        (merge-with + s1 s2))
+      (select-keys
+        agg-bolt-stats
+        [:emitted :transferred :acked :failed :complete-latencies])
+      (select-keys
+        agg-spout-stats
+        [:emitted :transferred :acked :failed :complete-latencies]))))
+
+(defn stats-times
+  [stats-map]
+  (sort-by #(Integer/parseInt %)
+           (-> stats-map
+               clojurify-structure
+               (dissoc ":all-time")
+               keys)))
+
+(defn window-hint
+  [window]
   (if (= window ":all-time")
     "All time"
     (pretty-uptime-sec window)))
 
-(defn topology-action-button [id name action command is-wait default-wait enabled]
+(defn topology-action-button
+  [id name action command is-wait default-wait enabled]
   [:input {:type "button"
            :value action
            (if enabled :enabled :disabled) ""
@@ -658,427 +352,507 @@
                          (StringEscapeUtils/escapeJavaScript name) "', '"
                          command "', " is-wait ", " default-wait ")")}])
 
-(defn topology-page
-  [id window include-sys?]
+(defn sanitize-stream-name
+  [name]
+  (let [sym-regex #"(?![A-Za-z_\-:\.])."]
+    (str
+     (if (re-find #"^[A-Za-z]" name)
+       (clojure.string/replace name sym-regex "_")
+       (clojure.string/replace (str \s name) sym-regex "_"))
+     (hash name))))
+
+(defn sanitize-transferred
+  [transferred]
+  (into {}
+        (for [[time, stream-map] transferred]
+          [time, (into {}
+                       (for [[stream, trans] stream-map]
+                         [(sanitize-stream-name stream), trans]))])))
+
+(defn visualization-data
+  [spout-bolt spout-comp-summs bolt-comp-summs window storm-id]
+  (let [components (for [[id spec] spout-bolt]
+            [id
+             (let [inputs (.get_inputs (.get_common spec))
+                   bolt-summs (get bolt-comp-summs id)
+                   spout-summs (get spout-comp-summs id)
+                   bolt-cap (if bolt-summs
+                              (compute-bolt-capacity bolt-summs)
+                              0)]
+               {:type (if bolt-summs "bolt" "spout")
+                :capacity bolt-cap
+                :latency (if bolt-summs
+                           (get-in
+                             (bolt-streams-stats bolt-summs true)
+                             [:process-latencies window])
+                           (get-in
+                             (spout-streams-stats spout-summs true)
+                             [:complete-latencies window]))
+                :transferred (or
+                               (get-in
+                                 (spout-streams-stats spout-summs true)
+                                 [:transferred window])
+                               (get-in
+                                 (bolt-streams-stats bolt-summs true)
+                                 [:transferred window]))
+                :stats (let [mapfn (fn [dat]
+                                     (map (fn [^ExecutorSummary summ]
+                                            {:host (.get_host summ)
+                                             :port (.get_port summ)
+                                             :uptime_secs (.get_uptime_secs summ)
+                                             :transferred (if-let [stats (.get_stats summ)]
+                                                            (sanitize-transferred (.get_transferred stats)))})
+                                          dat))]
+                         (if bolt-summs
+                           (mapfn bolt-summs)
+                           (mapfn spout-summs)))
+                :link (url-format "/component.html?id=%s&topology_id=%s" id storm-id)
+                :inputs (for [[global-stream-id group] inputs]
+                          {:component (.get_componentId global-stream-id)
+                           :stream (.get_streamId global-stream-id)
+                           :sani-stream (sanitize-stream-name (.get_streamId global-stream-id))
+                           :grouping (clojure.core/name (thrift/grouping-type group))})})])]
+    (into {} (doall components))))
+
+(defn stream-boxes [datmap]
+  (let [filter-fn (mk-include-sys-fn true)
+        streams
+        (vec (doall (distinct
+                     (apply concat
+                            (for [[k v] datmap]
+                              (for [m (get v :inputs)]
+                                {:stream (get m :stream)
+                                 :sani-stream (get m :sani-stream)
+                                 :checked (is-ack-stream (get m :stream))}))))))]
+    (map (fn [row]
+           {:row row}) (partition 4 4 nil streams))))
+
+(defn mk-visualization-data [id window include-sys?]
+  (with-nimbus
+    nimbus
+    (let [window (if window window ":all-time")
+          topology (.getTopology ^Nimbus$Client nimbus id)
+          spouts (.get_spouts topology)
+          bolts (.get_bolts topology)
+          summ (.getTopologyInfo ^Nimbus$Client nimbus id)
+          execs (.get_executors summ)
+          spout-summs (filter (partial spout-summary? topology) execs)
+          bolt-summs (filter (partial bolt-summary? topology) execs)
+          spout-comp-summs (group-by-comp spout-summs)
+          bolt-comp-summs (group-by-comp bolt-summs)
+          bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?)
+                                      bolt-comp-summs)
+          topology-conf (from-json
+                          (.getTopologyConf ^Nimbus$Client nimbus id))]
+      (visualization-data
+        (merge (hashmap-to-persistent spouts)
+               (hashmap-to-persistent bolts))
+        spout-comp-summs bolt-comp-summs window id))))
+
+(defn cluster-configuration []
   (with-nimbus nimbus
-               (let [window (if window window ":all-time")
-                     window-hint (window-hint window)
-                     summ (.getTopologyInfo ^Nimbus$Client nimbus id)
-                     topology (.getTopology ^Nimbus$Client nimbus id)
-                     topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id))
-                     spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
-                     bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
-                     spout-comp-summs (group-by-comp spout-summs)
-                     bolt-comp-summs (group-by-comp bolt-summs)
-                     bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs)
-                     name (.get_name summ)
-                     status (.get_status summ)
-                     msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)]
-                 (concat
-                   [[:h2 "Topology summary"]]
-                   [(topology-summary-table summ)]
-                   [[:h2 {:class "js-only"} "Topology actions"]]
-                   [[:p {:class "js-only"}
-                     (concat
-                       [(topology-action-button id name "Activate" "activate" false 0 (= "INACTIVE" status))]
-                       [(topology-action-button id name "Deactivate" "deactivate" false 0 (= "ACTIVE" status))]
-                       [(topology-action-button id name "Rebalance" "rebalance" true msg-timeout (or (= "ACTIVE" status) (= "INACTIVE" status)))]
-                       [(topology-action-button id name "Kill" "kill" true msg-timeout (not= "KILLED" status))])]]
-                   [[:h2 "Topology stats"]]
-                   (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
-                   [[:h2 "Spouts (" window-hint ")"]]
-                   (spout-comp-table id spout-comp-summs (.get_errors summ) window include-sys?)
-                   [[:h2 "Bolts (" window-hint ")"]]
-                   (bolt-comp-table id bolt-comp-summs (.get_errors summ) window include-sys?)
-                   [[:h2 "Topology Configuration"]]
-                   (configuration-table topology-conf)))))
-
-(defn component-task-summs
-  [^TopologyInfo summ topology id]
-  (let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
-        bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
-        spout-comp-summs (group-by-comp spout-summs)
-        bolt-comp-summs (group-by-comp bolt-summs)
-        ret (if (contains? spout-comp-summs id)
-              (spout-comp-summs id)
-              (bolt-comp-summs id))]
-    (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret)))
-
-(defn spout-summary-table
-  [topology-id id stats window]
+    (.getNimbusConf ^Nimbus$Client nimbus)))
+
+(defn cluster-summary
+  ([]
+     (with-nimbus nimbus
+        (cluster-summary (.getClusterInfo ^Nimbus$Client nimbus))))
+  ([^ClusterSummary summ]
+     (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 +))]
+       {"stormVersion" (read-storm-version)
+        "nimbusUptime" (pretty-uptime-sec (.get_nimbus_uptime_secs summ))
+        "supervisors" (count sups)
+        "slotsTotal" total-slots
+        "slotsUsed"  used-slots
+        "slotsFree" free-slots
+        "executorsTotal" total-executors
+        "tasksTotal" total-tasks})))
+
+(defn supervisor-summary
+  ([]
+   (with-nimbus nimbus
+                (supervisor-summary
+                  (.get_supervisors (.getClusterInfo ^Nimbus$Client nimbus)))))
+  ([summs]
+   {"supervisors"
+    (for [^SupervisorSummary s summs]
+      {"id" (.get_supervisor_id s)
+       "host" (.get_host s)
+       "uptime" (pretty-uptime-sec (.get_uptime_secs s))
+       "slotsTotal" (.get_num_workers s)
+       "slotsUsed" (.get_num_used_workers s)})}))
+
+(defn all-topologies-summary
+  ([]
+   (with-nimbus
+     nimbus
+     (all-topologies-summary
+       (.get_topologies (.getClusterInfo ^Nimbus$Client nimbus)))))
+  ([summs]
+   {"topologies"
+    (for [^TopologySummary t summs]
+      {"id" (.get_id t)
+       "name" (.get_name t)
+       "status" (.get_status t)
+       "uptime" (pretty-uptime-sec (.get_uptime_secs t))
+       "tasksTotal" (.get_num_tasks t)
+       "workersTotal" (.get_num_workers t)
+       "executorsTotal" (.get_num_executors t)})}))
+
+(defn topology-stats [id window stats]
   (let [times (stats-times (:emitted stats))
         display-map (into {} (for [t times] [t pretty-uptime-sec]))
         display-map (assoc display-map ":all-time" (fn [_] "All time"))]
-    (sorted-table
-      [{:text "Window" :attr {:class "tip right"
-                              :title (:window tips)}}
-       {:text "Emitted" :attr {:class "tip above"
-                               :title (:emitted tips)}}
-       {:text "Transferred" :attr {:class "tip above"
-                                   :title (:transferred tips)}}
-       {:text "Complete latency (ms)" :attr {:class "tip above"
-                                             :title (:complete-lat tips)}}
-       {:text "Acked" :attr {:class "tip above"
-                             :title (:spout-acked tips)}}
-       {:text "Failed" :attr {:class "tip left"
-                              :title (:spout-failed tips)}}]
-      (for [k (concat times [":all-time"])
-            :let [disp ((display-map k) k)]]
-        [(link-to (if (= k window) {:class "red"} {})
-                  (url-format "/topology/%s/component/%s?window=%s" topology-id id k)
-                  (escape-html disp))
-         (get-in stats [:emitted k])
-         (get-in stats [:transferred k])
-         (float-str (get-in stats [:complete-latencies k]))
-         (get-in stats [:acked k])
-         (get-in stats [:failed k])])
-      :time-cols [0])))
-
-(defn spout-output-summary-table
+    (for [k (concat times [":all-time"])
+          :let [disp ((display-map k) k)]]
+      {"windowPretty" disp
+       "window" k
+       "emitted" (get-in stats [:emitted k])
+       "transferred" (get-in stats [:transferred k])
+       "completeLatency" (float-str (get-in stats [:complete-latencies k]))
+       "acked" (get-in stats [:acked k])
+       "failed" (get-in stats [:failed k])})))
+
+(defn spout-comp [top-id summ-map errors window include-sys?]
+  (for [[id summs] summ-map
+        :let [stats-seq (get-filled-stats summs)
+              stats (aggregate-spout-streams
+                     (aggregate-spout-stats
+                      stats-seq include-sys?))]]
+    {"spoutId" id
+     "executors" (count summs)
+     "tasks" (sum-tasks summs)
+     "emitted" (get-in stats [:emitted window])
+     "transferred" (get-in stats [:transferred window])
+     "completeLatency" (float-str (get-in stats [:complete-latencies window]))
+     "acked" (get-in stats [:acked window])
+     "failed" (get-in stats [:failed window])
+     "lastError" (most-recent-error (get errors id))}))
+
+(defn bolt-comp [top-id summ-map errors window include-sys?]
+  (for [[id summs] summ-map
+        :let [stats-seq (get-filled-stats summs)
+              stats (aggregate-bolt-streams
+                     (aggregate-bolt-stats
+                      stats-seq include-sys?))]]
+    {"boltId" id
+     "executors" (count summs)
+     "tasks" (sum-tasks summs)
+     "emitted" (get-in stats [:emitted window])
+     "transferred" (get-in stats [:transferred window])
+     "capacity" (float-str (nil-to-zero (compute-bolt-capacity summs)))
+     "executeLatency" (float-str (get-in stats [:execute-latencies window]))
+     "executed" (get-in stats [:executed window])
+     "processLatency" (float-str (get-in stats [:process-latencies window]))
+     "acked" (get-in stats [:acked window])
+     "failed" (get-in stats [:failed window])
+     "lastError" (most-recent-error (get errors id))}))
+
+(defn topology-summary [^TopologyInfo summ]
+  (let [executors (.get_executors summ)
+        workers (set (for [^ExecutorSummary e executors]
+                       [(.get_host e) (.get_port e)]))]
+      {"id" (.get_id summ)
+       "name" (.get_name summ)
+       "status" (.get_status summ)
+       "uptime" (pretty-uptime-sec (.get_uptime_secs summ))
+       "tasksTotal" (sum-tasks executors)
+       "workersTotal" (count workers)
+       "executorsTotal" (count executors)}))
+
+(defn spout-summary-json [topology-id id stats window]
+  (let [times (stats-times (:emitted stats))
+        display-map (into {} (for [t times] [t pretty-uptime-sec]))
+        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
+     (for [k (concat times [":all-time"])
+           :let [disp ((display-map k) k)]]
+       {"windowPretty" disp
+        "window" k
+        "emitted" (get-in stats [:emitted k])
+        "transferred" (get-in stats [:transferred k])
+        "completeLatency" (float-str (get-in stats [:complete-latencies k]))
+        "acked" (get-in stats [:acked k])
+        "failed" (get-in stats [:failed k])})))
+
+(defn topology-page [id window include-sys?]
+  (with-nimbus nimbus
+    (let [window (if window window ":all-time")
+          window-hint (window-hint window)
+          summ (.getTopologyInfo ^Nimbus$Client nimbus id)
+          topology (.getTopology ^Nimbus$Client nimbus id)
+          topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id))
+          spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
+          bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
+          spout-comp-summs (group-by-comp spout-summs)
+          bolt-comp-summs (group-by-comp bolt-summs)
+          bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs)
+          name (.get_name summ)
+          status (.get_status summ)
+          msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)
+          spouts (.get_spouts topology)
+          bolts (.get_bolts topology)
+          visualizer-data (visualization-data (merge (hashmap-to-persistent spouts)
+                                                     (hashmap-to-persistent bolts))
+                                              spout-comp-summs
+                                              bolt-comp-summs
+                                              window
+                                              id)]
+      (merge
+       (topology-summary summ)
+       {"window" window
+        "windowHint" window-hint
+        "msgTimeout" msg-timeout
+        "topologyStats" (topology-stats id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
+        "spouts" (spout-comp id spout-comp-summs (.get_errors summ) window include-sys?)
+        "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys?)
+        "configuration" topology-conf
+        "visualizationTable" (stream-boxes visualizer-data)}))))
+
+(defn spout-output-stats
   [stream-summary window]
   (let [stream-summary (map-val swap-map-order (swap-map-order stream-summary))]
-    (sorted-table
-      [{:text "Stream" :attr {:class "tip right"
-                              :title (:stream tips)}}
-       {:text "Emitted" :attr {:class "tip above"
-                               :title (:emitted tips)}}
-       {:text "Transferred" :attr {:class "tip above"
-                                   :title (:transferred tips)}}
-       {:text "Complete latency (ms)" :attr {:class "tip above"
-                                             :title (:complete-lat tips)}}
-       {:text "Acked" :attr {:class "tip above"
-                             :title (:spout-acked tips)}}
-       {:text "Failed" :attr {:class "tip left"
-                              :title (:spout-failed tips)}}]
-      (for [[s stats] (stream-summary window)]
-        [s
-         (nil-to-zero (:emitted stats))
-         (nil-to-zero (:transferred stats))
-         (float-str (:complete-latencies stats))
-         (nil-to-zero (:acked stats))
-         (nil-to-zero (:failed stats))]))))
-
-(defn spout-executor-table
+    (for [[s stats] (stream-summary window)]
+      {"stream" s
+       "emitted" (nil-to-zero (:emitted stats))
+       "transferred" (nil-to-zero (:transferred stats))
+       "completeLatency" (float-str (:complete-latencies stats))
+       "acked" (nil-to-zero (:acked stats))
+       "failed" (nil-to-zero (:failed stats))})))
+
+(defn spout-executor-stats
   [topology-id executors window include-sys?]
-  (sorted-table
-    [{:text "Id" :attr {:class "tip right"
-                        :title (:exec-id tips)}}
-     {:text "Uptime" :attr {:class "tip right"
-                            :title (:exec-uptime tips)}}
-     {:text "Host" :attr {:class "tip above"
-                          :title (:sup-host tips)}}
-     {:text "Port" :attr {:class "tip above"
-                          :title (:port tips)}}
-     {:text "Emitted" :attr {:class "tip above"
-                             :title (:emitted tips)}}
-     {:text "Transferred" :attr {:class "tip above"
-                                 :title (:transferred tips)}}
-     {:text "Complete latency (ms)" :attr {:class "tip above"
-                                           :title (:complete-lat tips)}}
-     {:text "Acked" :attr {:class "tip above"
-                           :title (:spout-acked tips)}}
-     {:text "Failed" :attr {:class "tip left"
-                            :title (:spout-failed tips)}}]
-    (for [^ExecutorSummary e executors
-          :let [stats (.get_stats e)
-                stats (if stats
-                        (-> stats
-                            (aggregate-spout-stats include-sys?)
-                            aggregate-spout-streams
-                            swap-map-order
-                            (get window)))]]
-      [(pretty-executor-info (.get_executor_info e))
-       (pretty-uptime-sec (.get_uptime_secs e))
-       (.get_host e)
-       (worker-log-link (.get_host e) (.get_port e))
-       (nil-to-zero (:emitted stats))
-       (nil-to-zero (:transferred stats))
-       (float-str (:complete-latencies stats))
-       (nil-to-zero (:acked stats))
-       (nil-to-zero (:failed stats))])
-    :time-cols [1]))
-
-(defn spout-page
+  (for [^ExecutorSummary e executors
+        :let [stats (.get_stats e)
+              stats (if stats
+                      (-> stats
+                          (aggregate-spout-stats include-sys?)
+                          aggregate-spout-streams
+                          swap-map-order
+                          (get window)))]]
+    {"id" (pretty-executor-info (.get_executor_info e))
+     "uptime" (pretty-uptime-sec (.get_uptime_secs e))
+     "host" (.get_host e)
+     "port" (.get_port e)
+     "emitted" (nil-to-zero (:emitted stats))
+     "transferred" (nil-to-zero (:transferred stats))
+     "completeLatency" (float-str (:complete-latencies stats))
+     "acked" (nil-to-zero (:acked stats))
+     "failed" (nil-to-zero (:failed stats))
+     "workerLogLink" (worker-log-link (.get_host e) (.get_port e))}))
+
+(defn component-errors
+  [errors-list]
+  (let [errors (->> errors-list
+                    (sort-by #(.get_error_time_secs ^ErrorInfo %))
+                    reverse)]
+    {"componentErrors"
+     (for [^ErrorInfo e errors]
+       {"time" (date-str (.get_error_time_secs e))
+        "error" (.get_error e)})}))
+
+(defn spout-stats
   [window ^TopologyInfo topology-info component executors include-sys?]
   (let [window-hint (str " (" (window-hint window) ")")
         stats (get-filled-stats executors)
         stream-summary (-> stats (aggregate-spout-stats include-sys?))
         summary (-> stream-summary aggregate-spout-streams)]
-    (concat
-      [[:h2 "Spout stats"]]
-      (spout-summary-table (.get_id topology-info) component summary window)
-      [[:h2 "Output stats" window-hint]]
-      (spout-output-summary-table stream-summary window)
-      [[:h2 "Executors" window-hint]]
-      ;; task id, task uptime, stream aggregated stats, last error
-      (spout-executor-table (.get_id topology-info) executors window include-sys?))))
-
-(defn bolt-output-summary-table
+    {"spoutSummary" (spout-summary-json
+                      (.get_id topology-info) component summary window)
+     "outputStats" (spout-output-stats stream-summary window)
+     "executorStats" (spout-executor-stats (.get_id topology-info)
+                                           executors window include-sys?)}))
+
+(defn bolt-summary
+  [topology-id id stats window]
+  (let [times (stats-times (:emitted stats))
+        display-map (into {} (for [t times] [t pretty-uptime-sec]))
+        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
+    (for [k (concat times [":all-time"])
+          :let [disp ((display-map k) k)]]
+      {"window" k
+       "windowPretty" disp
+       "emitted" (get-in stats [:emitted k])
+       "transferred" (get-in stats [:transferred k])
+       "executeLatency" (float-str (get-in stats [:execute-latencies k]))
+       "executed" (get-in stats [:executed k])
+       "processLatency" (float-str (get-in stats [:process-latencies k]))
+       "acked" (get-in stats [:acked k])
+       "failed" (get-in stats [:failed k])})))
+
+(defn bolt-output-stats
   [stream-summary window]
   (let [stream-summary (-> stream-summary
                            swap-map-order
                            (get window)
                            (select-keys [:emitted :transferred])
                            swap-map-order)]
-    (sorted-table
-      [{:text "Stream" :attr {:class "tip right"
-                              :title (:stream tips)}}
-       {:text "Emitted" :attr {:class "tip above"
-                               :title (:emitted tips)}}
-       {:text "Transferred" :attr {:class "tip above"
-                                   :title (:transferred tips)}}]
-      (for [[s stats] stream-summary]
-        [s
-         (nil-to-zero (:emitted stats))
-         (nil-to-zero (:transferred stats))]))))
-
-(defn bolt-input-summary-table
+    (for [[s stats] stream-summary]
+      {"stream" s
+        "emitted" (nil-to-zero (:emitted stats))
+        "transferred" (nil-to-zero (:transferred stats))})))
+
+(defn bolt-input-stats
   [stream-summary window]
-  (let [stream-summary (-> stream-summary
-                           swap-map-order
-                           (get window)
-                           (select-keys [:acked :failed :process-latencies :executed :execute-latencies])
-                           swap-map-order)]
-    (sorted-table
-      [{:text "Component" :attr {:class "tip right"
-                                 :title (:comp-id tips)}}
-       {:text "Stream" :attr {:class "tip right"
-                              :title (:stream tips)}}
-       {:text "Execute latency (ms)" :attr {:class "tip above"
-                                            :title (:exec-lat tips)}}
-       {:text "Executed" :attr {:class "tip above"
-                                :title (:num-executed tips)}}
-       {:text "Process latency (ms)":attr {:class "tip above"
-                                           :title (:proc-lat tips)}}
-       {:text "Acked" :attr {:class "tip above"
-                             :title (:bolt-acked tips)}}
-       {:text "Failed" :attr {:class "tip left"
-                              :title (:bolt-failed tips)}}]
-      (for [[^GlobalStreamId s stats] stream-summary]
-        [(escape-html (.get_componentId s))
-         (.get_streamId s)
-         (float-str (:execute-latencies stats))
-         (nil-to-zero (:executed stats))
-         (float-str (:process-latencies stats))
-         (nil-to-zero (:acked stats))
-         (nil-to-zero (:failed stats))]))))
-
-(defn bolt-executor-table
+  (let [stream-summary
+        (-> stream-summary
+            swap-map-order
+            (get window)
+            (select-keys [:acked :failed :process-latencies
+                          :executed :execute-latencies])
+            swap-map-order)]
+    (for [[^GlobalStreamId s stats] stream-summary]
+      {"component" (.get_componentId s)
+       "stream" (.get_streamId s)
+       "executeLatency" (float-str (:execute-latencies stats))
+       "processLatency" (float-str (:execute-latencies stats))
+       "executed" (nil-to-zero (:executed stats))
+       "acked" (nil-to-zero (:acked stats))
+       "failed" (nil-to-zero (:failed stats))})))
+
+(defn bolt-executor-stats
   [topology-id executors window include-sys?]
-  (sorted-table
-    [{:text "Id" :attr {:class "tip right"
-                        :title (:exec-id tips)}}
-     {:text "Uptime" :attr {:class "tip right"
-                            :title (:exec-uptime tips)}}
-     {:text "Host" :attr {:class "tip above"
-                          :title (:sup-host tips)}}
-     {:text "Port" :attr {:class "tip above"
-                          :title (:port tips)}}
-     {:text "Emitted" :attr {:class "tip above"
-                             :title (:emitted tips)}}
-     {:text "Transferred" :attr {:class "tip above"
-                                 :title (:transferred tips)}}
-     {:text "Capacity (last 10m)" :attr {:class "tip above"
-                                         :title (:capacity tips)}}
-     {:text "Execute latency (ms)" :attr {:class "tip above"
-                                          :title (:exec-lat tips)}}
-     {:text "Executed" :attr {:class "tip above"
-                              :title (:num-executed tips)}}
-     {:text "Process latency (ms)":attr {:class "tip above"
-                                         :title (:proc-lat tips)}}
-     {:text "Acked" :attr {:class "tip above"
-                           :title (:bolt-acked tips)}}
-     {:text "Failed" :attr {:class "tip left"
-                            :title (:bolt-failed tips)}}]
-    (for [^ExecutorSummary e executors
-          :let [stats (.get_stats e)
-                stats (if stats
-                        (-> stats
-                            (aggregate-bolt-stats include-sys?)
-                            (aggregate-bolt-streams)
-                            swap-map-order
-                            (get window)))]]
-      [(pretty-executor-info (.get_executor_info e))
-       (pretty-uptime-sec (.get_uptime_secs e))
-       (.get_host e)
-       (worker-log-link (.get_host e) (.get_port e))
-       (nil-to-zero (:emitted stats))
-       (nil-to-zero (:transferred stats))
-       (render-capacity (compute-executor-capacity e))
-       (float-str (:execute-latencies stats))
-       (nil-to-zero (:executed stats))
-       (float-str (:process-latencies stats))
-       (nil-to-zero (:acked stats))
-       (nil-to-zero (:failed stats))])
-    :time-cols [1]))
-
-(defn bolt-summary-table
-  [topology-id id stats window]
-  (let [times (stats-times (:emitted stats))
-        display-map (into {} (for [t times] [t pretty-uptime-sec]))
-        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
-    (sorted-table
-      [{:text "Window" :attr {:class "tip right"
-                              :title (:window tips)}}
-       {:text "Emitted" :attr {:class "tip above"
-                               :title (:emitted tips)}}
-       {:text "Transferred" :attr {:class "tip above"
-                                   :title (:transferred tips)}}
-       {:text "Execute latency (ms)" :attr {:class "tip above"
-                                            :title (:exec-lat tips)}}
-       {:text "Executed" :attr {:class "tip above"
-                                :title (:num-executed tips)}}
-       {:text "Process latency (ms)":attr {:class "tip above"
-                                           :title (:proc-lat tips)}}
-       {:text "Acked" :attr {:class "tip above"
-                             :title (:bolt-acked tips)}}
-       {:text "Failed" :attr {:class "tip left"
-                              :title (:bolt-failed tips)}}]
-      (for [k (concat times [":all-time"])
-            :let [disp ((display-map k) k)]]
-        [(link-to (if (= k window) {:class "red"} {})
-                  (url-format "/topology/%s/component/%s?window=%s" topology-id id k)
-                  (escape-html disp))
-         (get-in stats [:emitted k])
-         (get-in stats [:transferred k])
-         (float-str (get-in stats [:execute-latencies k]))
-         (get-in stats [:executed k])
-         (float-str (get-in stats [:process-latencies k]))
-         (get-in stats [:acked k])
-         (get-in stats [:failed k])])
-      :time-cols [0])))
-
-(defn bolt-page
+  (for [^ExecutorSummary e executors
+        :let [stats (.get_stats e)
+              stats (if stats
+                      (-> stats
+                          (aggregate-bolt-stats include-sys?)
+                          (aggregate-bolt-streams)
+                          swap-map-order
+                          (get window)))]]
+    {"id" (pretty-executor-info (.get_executor_info e))
+     "uptime" (pretty-uptime-sec (.get_uptime_secs e))
+     "host" (.get_host e)
+     "port" (.get_port e)
+     "emitted" (nil-to-zero (:emitted stats))
+     "transferred" (nil-to-zero (:transferred stats))
+     "capacity" (float-str (nil-to-zero (compute-executor-capacity e)))
+     "executeLatency" (float-str (:execute-latencies stats))
+     "executed" (nil-to-zero (:executed stats))
+     "processLatency" (float-str (:process-latencies stats))
+     "acked" (nil-to-zero (:acked stats))
+     "failed" (nil-to-zero (:failed stats))
+     "workerLogLink" (worker-log-link (.get_host e) (.get_port e))}))
+
+(defn bolt-stats
   [window ^TopologyInfo topology-info component executors include-sys?]
   (let [window-hint (str " (" (window-hint window) ")")
         stats (get-filled-stats executors)
         stream-summary (-> stats (aggregate-bolt-stats include-sys?))
         summary (-> stream-summary aggregate-bolt-streams)]
-    (concat
-      [[:h2 "Bolt stats"]]
-      (bolt-summary-table (.get_id topology-info) component summary window)
-
-      [[:h2 "Input stats" window-hint]]
-      (bolt-input-summary-table stream-summary window)
-
-      [[:h2 "Output stats" window-hint]]
-      (bolt-output-summary-table stream-summary window)
-
-      [[:h2 "Executors"]]
-      (bolt-executor-table (.get_id topology-info) executors window include-sys?))))
-
-(defn errors-table
-  [errors-list]
-  (let [errors (->> errors-list
-                    (sort-by #(.get_error_time_secs ^ErrorInfo %))
-                    reverse)]
-    (sorted-table
-      ["Time" "Error"]
-      (for [^ErrorInfo e errors]
-        [(date-str (.get_error_time_secs e))
-         [:pre (.get_error e)]])
-      :sort-list "[[0,1]]")))
+    {"boltStats" (bolt-summary (.get_id topology-info) component summary window)
+     "inputStats" (bolt-input-stats stream-summary window)
+     "outputStats" (bolt-output-stats stream-summary window)
+     "executorStats" (bolt-executor-stats
+                       (.get_id topology-info) executors window include-sys?)}))
 
 (defn component-page
   [topology-id component window include-sys?]
   (with-nimbus nimbus
-               (let [window (if window window ":all-time")
-                     summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
-                     topology (.getTopology ^Nimbus$Client nimbus topology-id)
-                     type (component-type topology component)
-                     summs (component-task-summs summ topology component)
-                     spec (cond (= type :spout) (spout-page window summ component summs include-sys?)
-                                (= type :bolt) (bolt-page window summ component summs include-sys?))]
-                 (concat
-                   [[:h2 "Component summary"]
-                    (table [{:text "Id" :attr {:class "tip right"
-                                               :title (:comp-id tips)}}
-                            {:text "Topology" :attr {:class "tip above"
-                                                     :title (str (:name tips) " " (:name-link tips))}}
-                            {:text "Executors" :attr {:class "tip above"
-                                                      :title (:num-execs tips)}}
-                            {:text "Tasks" :attr {:class "tip above"
-                                                  :title (:num-tasks tips)}}]
-                           [[(escape-html component)
-                             (topology-link (.get_id summ) (.get_name summ))
-                             (count summs)
-                             (sum-tasks summs)
-                             ]])]
-                   spec
-                   [[:h2 "Errors"]
-                    (errors-table (get (.get_errors summ) component))]))))
-
-(defn get-include-sys?
-  [cookies]
-  (let [sys? (get cookies "sys")
-        sys? (if (or (nil? sys?) (= "false" (:value sys?))) false true)]
-    sys?))
+    (let [window (if window window ":all-time")
+          summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
+          topology (.getTopology ^Nimbus$Client nimbus topology-id)
+          type (component-type topology component)
+          summs (component-task-summs summ topology component)
+          spec (cond (= type :spout) (spout-stats window summ component summs include-sys?)
+                     (= type :bolt) (bolt-stats window summ component summs include-sys?))
+          errors (component-errors (get (.get_errors summ) component))]
+      (merge
+       {"id" component
+         "name" (.get_name summ)
+         "executors" (count summs)
+         "tasks" (sum-tasks summs)
+         "topologyId" topology-id
+         "window" window
+         "componentType" (name type)
+         "windowHint" (window-hint window)}
+       spec errors))))
+
+(defn check-include-sys?
+  [sys?]
+  (if (or (nil? sys?) (= "false" sys?)) false true))
+
+(defn json-response [data & [status]]
+  {:status (or status 200)
+   :headers {"Content-Type" "application/json"}
+   :body (to-json data)})
 
 (defroutes main-routes
-  (GET "/" [:as {cookies :cookies}]
-       (-> (main-page)
-           ui-template))
-  (GET "/topology/:id" [:as {cookies :cookies} id & m]
-       (let [include-sys? (get-include-sys? cookies)
-             id (url-decode id)]
-         (try
-           (-> (topology-page (url-decode id) (:window m) include-sys?)
-               (concat [(mk-system-toggle-button include-sys?)])
-               ui-template)
-           (catch Exception e (resp/redirect "/")))))
-  (GET "/topology/:id/component/:component" [:as {cookies :cookies} id component & m]
-       (let [include-sys? (get-include-sys? cookies)
-             id (url-decode id)
+  (GET "/api/v1/cluster/configuration" []
+       (cluster-configuration))
+  (GET "/api/v1/cluster/summary" []
+       (json-response (cluster-summary)))
+  (GET "/api/v1/supervisor/summary" []
+       (json-response (supervisor-summary)))
+  (GET "/api/v1/topology/summary" []
+       (json-response (all-topologies-summary)))
+  (GET  "/api/v1/topology/:id" [id & m]
+        (let [id (url-decode id)]
+          (json-response (topology-page id (:window m) (check-include-sys? (:sys m))))))
+  (GET "/api/v1/topology/:id/visualization" [:as {:keys [cookies servlet-request]} id & m]
+       (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m)))))
+  (GET "/api/v1/topology/:id/component/:component" [id component & m]
+       (let [id (url-decode id)
              component (url-decode component)]
-         (-> (component-page id component (:window m) include-sys?)
-             (concat [(mk-system-toggle-button include-sys?)])
-             ui-template)))
-  (POST "/topology/:id/activate" [id]
-        (with-nimbus nimbus
-                     (let [id (url-decode id)
-                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-                           name (.get_name tplg)]
-                       (.activate nimbus name)
-                       (log-message "Activating topology '" name "'")))
-        (resp/redirect (str "/topology/" id)))
-  (POST "/topology/:id/deactivate" [id]
-        (with-nimbus nimbus
-                     (let [id (url-decode id)
-                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-                           name (.get_name tplg)]
-                       (.deactivate nimbus name)
-                       (log-message "Deactivating topology '" name "'")))
-        (resp/redirect (str "/topology/" id)))
-  (POST "/topology/:id/rebalance/:wait-time" [id wait-time]
-        (with-nimbus nimbus
-                     (let [id (url-decode id)
-                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-                           name (.get_name tplg)
-                           options (RebalanceOptions.)]
-                       (.set_wait_secs options (Integer/parseInt wait-time))
-                       (.rebalance nimbus name options)
-                       (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
-        (resp/redirect (str "/topology/" id)))
-  (POST "/topology/:id/kill/:wait-time" [id wait-time]
-        (with-nimbus nimbus
-                     (let [id (url-decode id)
-                           tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
-                           name (.get_name tplg)
-                           options (KillOptions.)]
-                       (.set_wait_secs options (Integer/parseInt wait-time))
-                       (.killTopologyWithOpts nimbus name options)
-                       (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
-        (resp/redirect (str "/topology/" id)))
+         (json-response (component-page id component (:window m) (check-include-sys? (:sys m))))))
+  (POST "/api/v1/topology/:id/activate" [id]
+    (with-nimbus nimbus
+      (let [id (url-decode id)
+            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+            name (.get_name tplg)]
+        (.activate nimbus name)
+        (log-message "Activating topology '" name "'")))
+    (resp/redirect (str "/api/v1/topology/" id)))
+
+  (POST "/api/v1/topology/:id/deactivate" [id]
+    (with-nimbus nimbus
+      (let [id (url-decode id)
+            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+            name (.get_name tplg)]
+        (.deactivate nimbus name)
+        (log-message "Deactivating topology '" name "'")))
+    (resp/redirect (str "/api/v1/topology/" id)))
+  (POST "/api/v1/topology/:id/rebalance/:wait-time" [id wait-time]
+    (with-nimbus nimbus
+      (let [id (url-decode id)
+            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+            name (.get_name tplg)
+            options (RebalanceOptions.)]
+        (.set_wait_secs options (Integer/parseInt wait-time))
+        (.rebalance nimbus name options)
+        (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
+    (resp/redirect (str "/api/v1/topology/" id)))
+  (POST "/api/v1/topology/:id/kill/:wait-time" [id wait-time]
+    (with-nimbus nimbus
+      (let [id (url-decode id)
+            tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+            name (.get_name tplg)
+            options (KillOptions.)]
+        (.set_wait_secs options (Integer/parseInt wait-time))
+        (.killTopologyWithOpts nimbus name options)
+        (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
+    (resp/redirect (str "/api/v1/topology/" id)))
+
+  (GET "/" [:as {cookies :cookies}]
+       (resp/redirect "/index.html"))
   (route/resources "/")
   (route/not-found "Page not found"))
 
-(defn exception->html
+(defn exception->json
   [ex]
-  (concat
-    [[:h2 "Internal Server Error"]]
-    [[:pre (let [sw (java.io.StringWriter.)]
-             (.printStackTrace ex (java.io.PrintWriter. sw))
-             (.toString sw))]]))
+  {"error" "Internal Server Error"
+   "errorMessage"
+   (let [sw (java.io.StringWriter.)]
+     (.printStackTrace ex (java.io.PrintWriter. sw))
+     (.toString sw))})
 
 (defn catch-errors
   [handler]
@@ -1086,9 +860,7 @@
     (try
       (handler request)
       (catch Exception ex
-        (-> (resp/response (ui-template (exception->html ex)))
-            (resp/status 500)
-            (resp/content-type "text/html"))))))
+        (json-response (exception->json ex) 500)))))
 
 (def app
   (handler/site (-> main-routes

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/ui/helpers.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj
index dcd4a21..c2de7c7 100644
--- a/storm-core/src/clj/backtype/storm/ui/helpers.clj
+++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj
@@ -146,4 +146,3 @@ $(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers:
 
 (defn pretty-executor-info [^ExecutorInfo e]
   (str "[" (.get_task_start e) "-" (.get_task_end e) "]"))
-

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 891d8eb..91b0713 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -914,12 +914,13 @@
 
 (defmacro -<>
   ([x] x)
-  ([x form]
-   (if (seq? form)
-     (with-meta
-       (let [[begin [_ & end]] (split-with #(not= % '<>) form)]
-         (concat begin [x] end))
-       (meta form))
-     (list form x)))
-  ([x form & more]
-   `(-<> (-<> ~x ~form) ~@more)))
+  ([x form] (if (seq? form)
+              (with-meta
+                (let [[begin [_ & end]] (split-with #(not= % '<>) form)]
+                  (concat begin [x] end))
+                (meta form))
+              (list form x)))
+  ([x form & more] `(-<> (-<> ~x ~form) ~@more)))
+
+(defn hashmap-to-persistent [^HashMap m]
+  (zipmap (.keySet m) (.values m)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/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 83ee232..ff309a5 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -84,8 +84,21 @@ public class Config extends HashMap<String, Object> {
      */
     public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; 
     public static final Object STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS_SCHEMA = Number.class;
+    
+    /**
+     * If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of STORM_NETTY_MESSAGE_BATCH_SIZE bytes
+     */
+    public static final String STORM_NETTY_MESSAGE_BATCH_SIZE = "storm.messaging.netty.transfer.batch.size";
+    public static final Object STORM_NETTY_MESSAGE_BATCH_SIZE_SCHEMA = Number.class;
 
     /**
+     * We check with this interval that whether the Netty channel is writable and try to write pending messages
+     */
+    public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms";
+    public static final Object STORM_NETTY_FLUSH_CHECK_INTERVAL_MS_SCHEMA = Number.class;
+    
+    
+    /**
      * A list of hosts of ZooKeeper servers used to manage the cluster.
      */
     public static final String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers";
@@ -462,6 +475,12 @@ public class Config extends HashMap<String, Object> {
     public static final Object WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
 
     /**
+     * control how many worker receiver threads we need per worker
+     */
+    public static final String WORKER_RECEIVER_THREAD_COUNT = "topology.worker.receiver.thread.count";
+    public static final Object WORKER_RECEIVER_THREAD_COUNT_SCHEMA = Number.class;
+    
+    /**
      * How often this worker should heartbeat to the supervisor.
      */
     public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs";

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/IConnection.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/IConnection.java b/storm-core/src/jvm/backtype/storm/messaging/IConnection.java
index 41ae3f5..ead4935 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/IConnection.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/IConnection.java
@@ -17,13 +17,17 @@
  */
 package backtype.storm.messaging;
 
+import java.util.Iterator;
+
 public interface IConnection {   
+    
     /**
-     * receive a message (consists taskId and payload)
+     * receive a batch message iterator (consists taskId and payload)
      * @param flags 0: block, 1: non-block
      * @return
      */
-    public TaskMessage recv(int flags);
+    public Iterator<TaskMessage> recv(int flags, int clientId);
+    
     /**
      * send a message with taskId and payload
      * @param taskId task ID
@@ -32,6 +36,13 @@ public interface IConnection {
     public void send(int taskId,  byte[] payload);
     
     /**
+     * send batch messages
+     * @param msgs
+     */
+
+    public void send(Iterator<TaskMessage> msgs);
+    
+    /**
      * close this connection
      */
     public void close();


[07/13] Merge branch 'master' into idiomatic-clojure-01

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/js/arbor-graphics.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/arbor-graphics.js b/storm-core/src/ui/public/js/arbor-graphics.js
new file mode 100644
index 0000000..4c68425
--- /dev/null
+++ b/storm-core/src/ui/public/js/arbor-graphics.js
@@ -0,0 +1,51 @@
+//
+//  arbor-graphics.js
+//  canvas fructose
+//
+//  Copyright (c) 2014 Samizdat Drafting Co.
+// 
+//  Permission is hereby granted, free of charge, to any person
+//  obtaining a copy of this software and associated documentation
+//  files (the "Software"), to deal in the Software without
+//  restriction, including without limitation the rights to use,
+//  copy, modify, merge, publish, distribute, sublicense, and/or sell
+//  copies of the Software, and to permit persons to whom the
+//  Software is furnished to do so, subject to the following
+//  conditions:
+// 
+//  The above copyright notice and this permission notice shall be
+//  included in all copies or substantial portions of the Software.
+// 
+//  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+//  EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+//  OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+//  NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+//  HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+//  WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+//  FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+//  OTHER DEALINGS IN THE SOFTWARE.
+//
+
+(function($){
+
+  /*        etc.js */  var trace=function(msg){if(typeof(window)=="undefined"||!window.console){return}var len=arguments.length;var args=[];for(var i=0;i<len;i++){args.push("arguments["+i+"]")}eval("console.log("+args.join(",")+")")};var dirname=function(a){var b=a.replace(/^\/?(.*?)\/?$/,"$1").split("/");b.pop();return"/"+b.join("/")};var basename=function(b){var c=b.replace(/^\/?(.*?)\/?$/,"$1").split("/");var a=c.pop();if(a==""){return null}else{return a}};var _ordinalize_re=/(\d)(?=(\d\d\d)+(?!\d))/g;var ordinalize=function(a){var b=""+a;if(a<11000){b=(""+a).replace(_ordinalize_re,"$1,")}else{if(a<1000000){b=Math.floor(a/1000)+"k"}else{if(a<1000000000){b=(""+Math.floor(a/1000)).replace(_ordinalize_re,"$1,")+"m"}}}return b};var nano=function(a,b){return a.replace(/\{([\w\-\.]*)}/g,function(f,c){var d=c.split("."),e=b[d.shift()];$.each(d,function(){if(e.hasOwnProperty(this)){e=e[this]}else{e=f}});return e})};var objcopy=function(a){if(a===undefined){return undefined}if(a===null){r
 eturn null}if(a.parentNode){return a}switch(typeof a){case"string":return a.substring(0);break;case"number":return a+0;break;case"boolean":return a===true;break}var b=($.isArray(a))?[]:{};$.each(a,function(d,c){b[d]=objcopy(c)});return b};var objmerge=function(d,b){d=d||{};b=b||{};var c=objcopy(d);for(var a in b){c[a]=b[a]}return c};var objcmp=function(e,c,d){if(!e||!c){return e===c}if(typeof e!=typeof c){return false}if(typeof e!="object"){return e===c}else{if($.isArray(e)){if(!($.isArray(c))){return false}if(e.length!=c.length){return false}}else{var h=[];for(var f in e){if(e.hasOwnProperty(f)){h.push(f)}}var g=[];for(var f in c){if(c.hasOwnProperty(f)){g.push(f)}}if(!d){h.sort();g.sort()}if(h.join(",")!==g.join(",")){return false}}var i=true;$.each(e,function(a){var b=objcmp(e[a],c[a]);i=i&&b;if(!i){return false}});return i}};var objkeys=function(b){var a=[];$.each(b,function(d,c){if(b.hasOwnProperty(d)){a.push(d)}});return a};var objcontains=function(c){if(!c||typeof c!="object"
 ){return false}for(var b=1,a=arguments.length;b<a;b++){if(c.hasOwnProperty(arguments[b])){return true}}return false};var uniq=function(b){var a=b.length;var d={};for(var c=0;c<a;c++){d[b[c]]=true}return objkeys(d)};var arbor_path=function(){var a=$("script").map(function(b){var c=$(this).attr("src");if(!c){return}if(c.match(/arbor[^\/\.]*.js|dev.js/)){return c.match(/.*\//)||"/"}});if(a.length>0){return a[0]}else{return null}};
+  /*     colors.js */  var Colors=(function(){var f=/#[0-9a-f]{6}/i;var b=/#(..)(..)(..)/;var c=function(h){var g=h.toString(16);return(g.length==2)?g:"0"+g};var a=function(g){return parseInt(g,16)};var d=function(g){if(!g||typeof g!="object"){return false}var h=objkeys(g).sort().join("");if(h=="abgr"){return true}};var e={CSS:{aliceblue:"#f0f8ff",antiquewhite:"#faebd7",aqua:"#00ffff",aquamarine:"#7fffd4",azure:"#f0ffff",beige:"#f5f5dc",bisque:"#ffe4c4",black:"#000000",blanchedalmond:"#ffebcd",blue:"#0000ff",blueviolet:"#8a2be2",brown:"#a52a2a",burlywood:"#deb887",cadetblue:"#5f9ea0",chartreuse:"#7fff00",chocolate:"#d2691e",coral:"#ff7f50",cornflowerblue:"#6495ed",cornsilk:"#fff8dc",crimson:"#dc143c",cyan:"#00ffff",darkblue:"#00008b",darkcyan:"#008b8b",darkgoldenrod:"#b8860b",darkgray:"#a9a9a9",darkgrey:"#a9a9a9",darkgreen:"#006400",darkkhaki:"#bdb76b",darkmagenta:"#8b008b",darkolivegreen:"#556b2f",darkorange:"#ff8c00",darkorchid:"#9932cc",darkred:"#8b0000",darksalmon:"#e9967a",dark
 seagreen:"#8fbc8f",darkslateblue:"#483d8b",darkslategray:"#2f4f4f",darkslategrey:"#2f4f4f",darkturquoise:"#00ced1",darkviolet:"#9400d3",deeppink:"#ff1493",deepskyblue:"#00bfff",dimgray:"#696969",dimgrey:"#696969",dodgerblue:"#1e90ff",firebrick:"#b22222",floralwhite:"#fffaf0",forestgreen:"#228b22",fuchsia:"#ff00ff",gainsboro:"#dcdcdc",ghostwhite:"#f8f8ff",gold:"#ffd700",goldenrod:"#daa520",gray:"#808080",grey:"#808080",green:"#008000",greenyellow:"#adff2f",honeydew:"#f0fff0",hotpink:"#ff69b4",indianred:"#cd5c5c",indigo:"#4b0082",ivory:"#fffff0",khaki:"#f0e68c",lavender:"#e6e6fa",lavenderblush:"#fff0f5",lawngreen:"#7cfc00",lemonchiffon:"#fffacd",lightblue:"#add8e6",lightcoral:"#f08080",lightcyan:"#e0ffff",lightgoldenrodyellow:"#fafad2",lightgray:"#d3d3d3",lightgrey:"#d3d3d3",lightgreen:"#90ee90",lightpink:"#ffb6c1",lightsalmon:"#ffa07a",lightseagreen:"#20b2aa",lightskyblue:"#87cefa",lightslategray:"#778899",lightslategrey:"#778899",lightsteelblue:"#b0c4de",lightyellow:"#ffffe0",lime:"
 #00ff00",limegreen:"#32cd32",linen:"#faf0e6",magenta:"#ff00ff",maroon:"#800000",mediumaquamarine:"#66cdaa",mediumblue:"#0000cd",mediumorchid:"#ba55d3",mediumpurple:"#9370d8",mediumseagreen:"#3cb371",mediumslateblue:"#7b68ee",mediumspringgreen:"#00fa9a",mediumturquoise:"#48d1cc",mediumvioletred:"#c71585",midnightblue:"#191970",mintcream:"#f5fffa",mistyrose:"#ffe4e1",moccasin:"#ffe4b5",navajowhite:"#ffdead",navy:"#000080",oldlace:"#fdf5e6",olive:"#808000",olivedrab:"#6b8e23",orange:"#ffa500",orangered:"#ff4500",orchid:"#da70d6",palegoldenrod:"#eee8aa",palegreen:"#98fb98",paleturquoise:"#afeeee",palevioletred:"#d87093",papayawhip:"#ffefd5",peachpuff:"#ffdab9",peru:"#cd853f",pink:"#ffc0cb",plum:"#dda0dd",powderblue:"#b0e0e6",purple:"#800080",red:"#ff0000",rosybrown:"#bc8f8f",royalblue:"#4169e1",saddlebrown:"#8b4513",salmon:"#fa8072",sandybrown:"#f4a460",seagreen:"#2e8b57",seashell:"#fff5ee",sienna:"#a0522d",silver:"#c0c0c0",skyblue:"#87ceeb",slateblue:"#6a5acd",slategray:"#708090",slate
 grey:"#708090",snow:"#fffafa",springgreen:"#00ff7f",steelblue:"#4682b4",tan:"#d2b48c",teal:"#008080",thistle:"#d8bfd8",tomato:"#ff6347",turquoise:"#40e0d0",violet:"#ee82ee",wheat:"#f5deb3",white:"#ffffff",whitesmoke:"#f5f5f5",yellow:"#ffff00",yellowgreen:"#9acd32"},decode:function(h){var g=arguments.length;for(var l=g-1;l>=0;l--){if(arguments[l]===undefined){g--}}var k=arguments;if(!h){return null}if(g==1&&d(h)){return h}var j=null;if(typeof h=="string"){var o=1;if(g==2){o=k[1]}var n=e.CSS[h.toLowerCase()];if(n!==undefined){h=n}var m=h.match(f);if(m){vals=h.match(b);if(!vals||!vals.length||vals.length!=4){return null}j={r:a(vals[1]),g:a(vals[2]),b:a(vals[3]),a:o}}}else{if(typeof h=="number"){if(g>=3){j={r:k[0],g:k[1],b:k[2],a:1};if(g>=4){j.a*=k[3]}}else{if(g>=1){j={r:k[0],g:k[0],b:k[0],a:1};if(g==2){j.a*=k[1]}}}}}return j},validate:function(g){if(!g||typeof g!="string"){return false}if(e.CSS[g.toLowerCase()]!==undefined){return true}if(g.match(f)){return true}return false},mix:funct
 ion(h,g,k){var j=e.decode(h);var i=e.decode(g)},blend:function(g,j){j=(j!==undefined)?Math.max(0,Math.min(1,j)):1;var h=e.decode(g);if(!h){return null}if(j==1){return g}var h=g;if(typeof g=="string"){h=e.decode(g)}var i=objcopy(h);i.a*=j;return nano("rgba({r},{g},{b},{a})",i)},encode:function(g){if(!d(g)){g=e.decode(g);if(!d(g)){return null}}if(g.a==1){return nano("#{r}{g}{b}",{r:c(g.r),g:c(g.g),b:c(g.b)})}else{return nano("rgba({r},{g},{b},{a})",g)}}};return e})();
+  /* primitives.js */  var Primitives=function(c,f,g){var b=function(i,m,j,l,k){this.x=i;this.y=m;this.w=j;this.h=l;this.style=(k!==undefined)?k:{}};b.prototype={draw:function(h){this._draw(h)},_draw:function(i,n,j,l,k){if(objcontains(i,"stroke","fill","width")){k=i}if(this.x!==undefined){i=this.x,n=this.y,j=this.w,l=this.h;k=objmerge(this.style,k)}k=objmerge(f,k);if(!k.stroke&&!k.fill){return}var m=0.5522848;ox=(j/2)*m,oy=(l/2)*m,xe=i+j,ye=n+l,xm=i+j/2,ym=n+l/2;c.save();c.beginPath();c.moveTo(i,ym);c.bezierCurveTo(i,ym-oy,xm-ox,n,xm,n);c.bezierCurveTo(xm+ox,n,xe,ym-oy,xe,ym);c.bezierCurveTo(xe,ym+oy,xm+ox,ye,xm,ye);c.bezierCurveTo(xm-ox,ye,i,ym+oy,i,ym);c.closePath();if(k.fill!==null){if(k.alpha!==undefined){c.fillStyle=Colors.blend(k.fill,k.alpha)}else{c.fillStyle=Colors.encode(k.fill)}c.fill()}if(k.stroke!==null){c.strokeStyle=Colors.encode(k.stroke);if(!isNaN(k.width)){c.lineWidth=k.width}c.stroke()}c.restore()}};var a=function(i,n,j,l,m,k){if(objcontains(m,"stroke","fill","widt
 h")){k=m;m=0}this.x=i;this.y=n;this.w=j;this.h=l;this.r=(m!==undefined)?m:0;this.style=(k!==undefined)?k:{}};a.prototype={draw:function(h){this._draw(h)},_draw:function(j,o,k,m,n,l){if(objcontains(n,"stroke","fill","width","alpha")){l=n;n=0}else{if(objcontains(j,"stroke","fill","width","alpha")){l=j}}if(this.x!==undefined){j=this.x,o=this.y,k=this.w,m=this.h;l=objmerge(this.style,l)}l=objmerge(f,l);if(!l.stroke&&!l.fill){return}var i=(n>0);c.save();c.beginPath();c.moveTo(j+n,o);c.lineTo(j+k-n,o);if(i){c.quadraticCurveTo(j+k,o,j+k,o+n)}c.lineTo(j+k,o+m-n);if(i){c.quadraticCurveTo(j+k,o+m,j+k-n,o+m)}c.lineTo(j+n,o+m);if(i){c.quadraticCurveTo(j,o+m,j,o+m-n)}c.lineTo(j,o+n);if(i){c.quadraticCurveTo(j,o,j+n,o)}if(l.fill!==null){if(l.alpha!==undefined){c.fillStyle=Colors.blend(l.fill,l.alpha)}else{c.fillStyle=Colors.encode(l.fill)}c.fill()}if(l.stroke!==null){c.strokeStyle=Colors.encode(l.stroke);if(!isNaN(l.width)){c.lineWidth=l.width}c.stroke()}c.restore()}};var e=function(i,l,h,j,k){if
 (k!==undefined||typeof j=="number"){this.points=[{x:i,y:l},{x:h,y:j}];this.style=k||{}}else{if($.isArray(i)){this.points=i;this.style=l||{}}else{this.points=[i,l];this.style=h||{}}}};e.prototype={draw:function(h){if(this.points.length<2){return}var j=[];if(!$.isArray(this.points[0])){j.push(this.points)}else{j=this.points}c.save();c.beginPath();$.each(j,function(n,m){c.moveTo(m[0].x+0.5,m[0].y+0.5);$.each(m,function(o,p){if(o==0){return}c.lineTo(p.x+0.5,p.y+0.5)})});var i=$.extend(objmerge(f,this.style),h);if(i.closed){c.closePath()}if(i.fill!==undefined){var l=Colors.decode(i.fill,(i.alpha!==undefined)?i.alpha:1);if(l){c.fillStyle=Colors.encode(l)}c.fill()}if(i.stroke!==undefined){var k=Colors.decode(i.stroke,(i.alpha!==undefined)?i.alpha:1);if(k){c.strokeStyle=Colors.encode(k)}if(!isNaN(i.width)){c.lineWidth=i.width}c.stroke()}c.restore()}};var d=function(i,h,l,k){var j=Colors.decode(i,h,l,k);if(j){this.r=j.r;this.g=j.g;this.b=j.b;this.a=j.a}};d.prototype={toString:function(){retu
 rn Colors.encode(this)},blend:function(){trace("blend",this.r,this.g,this.b,this.a)}};return{_Oval:b,_Rect:a,_Color:d,_Path:e}};
+  /*   graphics.js */  var Graphics=function(c){var h=$(c);var q=$(h).get(0).getContext("2d");var i=null;var l="rgb";var e="origin";var m={};var p={background:null,fill:null,stroke:null,width:0};var b={};var g={font:"sans-serif",size:12,align:"left",color:Colors.decode("black"),alpha:1,baseline:"ideographic"};var k=[];var o=Primitives(q,p,g);var f=o._Oval;var n=o._Rect;var d=o._Color;var a=o._Path;var j={init:function(){if(!q){return null}return j},size:function(s,r){if(!isNaN(s)&&!isNaN(r)){h.attr({width:s,height:r})}return{width:h.attr("width"),height:h.attr("height")}},clear:function(r,u,s,t){if(arguments.length<4){r=0;u=0;s=h.attr("width");t=h.attr("height")}q.clearRect(r,u,s,t);if(p.background!==null){q.save();q.fillStyle=Colors.encode(p.background);q.fillRect(r,u,s,t);q.restore()}},background:function(s,r,v,t){if(s==null){p.background=null;return null}var u=Colors.decode(s,r,v,t);if(u){p.background=u;j.clear()}},noFill:function(){p.fill=null},fill:function(s,r,v,t){if(argument
 s.length==0){return p.fill}else{if(arguments.length>0){var u=Colors.decode(s,r,v,t);p.fill=u;q.fillStyle=Colors.encode(u)}}},noStroke:function(){p.stroke=null;q.strokeStyle=null},stroke:function(s,r,v,u){if(arguments.length==0&&p.stroke!==null){return p.stroke}else{if(arguments.length>0){var t=Colors.decode(s,r,v,u);p.stroke=t;q.strokeStyle=Colors.encode(t)}}},strokeWidth:function(r){if(r===undefined){return q.lineWidth}q.lineWidth=p.width=r},Color:function(r){return new d(r)},drawStyle:function(s){if(arguments.length==0){return objcopy(p)}if(arguments.length==2){var r=arguments[0];var v=arguments[1];if(typeof r=="string"&&typeof v=="object"){var u={};if(v.color!==undefined){var t=Colors.decode(v.color);if(t){u.color=t}}$.each("background fill stroke width".split(" "),function(w,x){if(v[x]!==undefined){u[x]=v[x]}});if(!$.isEmptyObject(u)){m[r]=u}}return}if(arguments.length==1&&m[arguments[0]]!==undefined){s=m[arguments[0]]}if(s.width!==undefined){p.width=s.width}q.lineWidth=p.width;
 $.each("background fill stroke",function(y,x){if(s[x]!==undefined){if(s[x]===null){p[x]=null}else{var w=Colors.decode(s[x]);if(w){p[x]=w}}}});q.fillStyle=p.fill;q.strokeStyle=p.stroke},textStyle:function(s){if(arguments.length==0){return objcopy(g)}if(arguments.length==2){var r=arguments[0];var v=arguments[1];if(typeof r=="string"&&typeof v=="object"){var u={};if(v.color!==undefined){var t=Colors.decode(v.color);if(t){u.color=t}}$.each("font size align baseline alpha".split(" "),function(w,x){if(v[x]!==undefined){u[x]=v[x]}});if(!$.isEmptyObject(u)){b[r]=u}}return}if(arguments.length==1&&b[arguments[0]]!==undefined){s=b[arguments[0]]}if(s.font!==undefined){g.font=s.font}if(s.size!==undefined){g.size=s.size}q.font=nano("{size}px {font}",g);if(s.align!==undefined){q.textAlign=g.align=s.align}if(s.baseline!==undefined){q.textBaseline=g.baseline=s.baseline}if(s.alpha!==undefined){g.alpha=s.alpha}if(s.color!==undefined){var t=Colors.decode(s.color);if(t){g.color=t}}if(g.color){var t=Colo
 rs.blend(g.color,g.alpha);if(t){q.fillStyle=t}}},text:function(s,r,z,v){if(arguments.length>=3&&!isNaN(r)){v=v||{};v.x=r;v.y=z}else{if(arguments.length==2&&typeof(r)=="object"){v=r}else{v=v||{}}}var u=objmerge(g,v);q.save();if(u.align!==undefined){q.textAlign=u.align}if(u.baseline!==undefined){q.textBaseline=u.baseline}if(u.font!==undefined&&!isNaN(u.size)){q.font=nano("{size}px {font}",u)}var w=(u.alpha!==undefined)?u.alpha:g.alpha;var t=(u.color!==undefined)?u.color:g.color;q.fillStyle=Colors.blend(t,w);if(w>0){q.fillText(s,Math.round(u.x),u.y)}q.restore()},textWidth:function(r,t){t=objmerge(g,t||{});q.save();q.font=nano("{size}px {font}",t);var s=q.measureText(r).width;q.restore();return s},Rect:function(s,A,t,v,z,u){return new n(s,A,t,v,z,u)},rect:function(s,A,t,v,z,u){n.prototype._draw(s,A,t,v,z,u)},Oval:function(r,v,s,u,t){return new f(r,v,s,u,t)},oval:function(r,v,s,u,t){t=t||{};f.prototype._draw(r,v,s,u,t)},line:function(s,v,r,t,u){var w=new a(s,v,r,t);w.draw(u)},lines:funct
 ion(s,u,r,t){if(typeof t=="number"){k.push([{x:s,y:u},{x:r,y:t}])}else{k.push([s,u])}},drawLines:function(r){var s=new a(k);s.draw(r);k=[]}};return j.init()};
+
+  arbor = (typeof(arbor)!=='undefined') ? arbor : {}
+  $.extend(arbor, {
+    // object constructor (don't use ‘new’, just call it)
+    Graphics:function(ctx){ return Graphics(ctx) },
+
+    // useful methods for dealing with the r/g/b
+    colors:{
+      CSS:Colors.CSS,           // dict:{colorname:"#fef2e2", ...}
+      validate:Colors.validate, // ƒ(str) -> t/f
+      decode:Colors.decode,     // ƒ(hexString_or_cssColor) -> {r,g,b,a}
+      encode:Colors.encode,     // ƒ({r,g,b,a}) -> hexOrRgbaString
+      blend:Colors.blend        // ƒ(color, opacity) -> rgbaString
+    }
+  })
+  
+})(this.jQuery)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/js/arbor-tween.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/arbor-tween.js b/storm-core/src/ui/public/js/arbor-tween.js
new file mode 100644
index 0000000..67f4c59
--- /dev/null
+++ b/storm-core/src/ui/public/js/arbor-tween.js
@@ -0,0 +1,86 @@
+//
+//  arbor-tween.js
+//  smooth transitions with a realtime clock
+//
+//  Copyright (c) 2011 Samizdat Drafting Co.
+// 
+//  Permission is hereby granted, free of charge, to any person
+//  obtaining a copy of this software and associated documentation
+//  files (the "Software"), to deal in the Software without
+//  restriction, including without limitation the rights to use,
+//  copy, modify, merge, publish, distribute, sublicense, and/or sell
+//  copies of the Software, and to permit persons to whom the
+//  Software is furnished to do so, subject to the following
+//  conditions:
+// 
+//  The above copyright notice and this permission notice shall be
+//  included in all copies or substantial portions of the Software.
+// 
+//  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+//  EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+//  OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+//  NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+//  HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+//  WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+//  FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+//  OTHER DEALINGS IN THE SOFTWARE.
+//
+
+//  Easing Equations in easing.js:
+//  Copyright © 2001 Robert Penner. All rights reserved.
+//  
+//  Open source under the BSD License. Redistribution and use in source
+//  and binary forms, with or without modification, are permitted
+//  provided that the following conditions are met:
+//  
+//  Redistributions of source code must retain the above copyright
+//  notice, this list of conditions and the following disclaimer.
+//  Redistributions in binary form must reproduce the above copyright
+//  notice, this list of conditions and the following disclaimer in the
+//  documentation and/or other materials provided with the distribution.
+//  
+//  Neither the name of the author nor the names of contributors may be
+//  used to endorse or promote products derived from this software
+//  without specific prior written permission.
+//  
+//  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+//  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+//  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+//  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+//  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+//  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+//  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+//  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+//  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+//  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+//  OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 
+
+
+(function($){
+
+  /*    etc.js */  var trace=function(msg){if(typeof(window)=="undefined"||!window.console){return}var len=arguments.length;var args=[];for(var i=0;i<len;i++){args.push("arguments["+i+"]")}eval("console.log("+args.join(",")+")")};var dirname=function(a){var b=a.replace(/^\/?(.*?)\/?$/,"$1").split("/");b.pop();return"/"+b.join("/")};var basename=function(b){var c=b.replace(/^\/?(.*?)\/?$/,"$1").split("/");var a=c.pop();if(a==""){return null}else{return a}};var _ordinalize_re=/(\d)(?=(\d\d\d)+(?!\d))/g;var ordinalize=function(a){var b=""+a;if(a<11000){b=(""+a).replace(_ordinalize_re,"$1,")}else{if(a<1000000){b=Math.floor(a/1000)+"k"}else{if(a<1000000000){b=(""+Math.floor(a/1000)).replace(_ordinalize_re,"$1,")+"m"}}}return b};var nano=function(a,b){return a.replace(/\{([\w\-\.]*)}/g,function(f,c){var d=c.split("."),e=b[d.shift()];$.each(d,function(){if(e.hasOwnProperty(this)){e=e[this]}else{e=f}});return e})};var objcopy=function(a){if(a===undefined){return undefined}if(a===null){retur
 n null}if(a.parentNode){return a}switch(typeof a){case"string":return a.substring(0);break;case"number":return a+0;break;case"boolean":return a===true;break}var b=($.isArray(a))?[]:{};$.each(a,function(d,c){b[d]=objcopy(c)});return b};var objmerge=function(d,b){d=d||{};b=b||{};var c=objcopy(d);for(var a in b){c[a]=b[a]}return c};var objcmp=function(e,c,d){if(!e||!c){return e===c}if(typeof e!=typeof c){return false}if(typeof e!="object"){return e===c}else{if($.isArray(e)){if(!($.isArray(c))){return false}if(e.length!=c.length){return false}}else{var h=[];for(var f in e){if(e.hasOwnProperty(f)){h.push(f)}}var g=[];for(var f in c){if(c.hasOwnProperty(f)){g.push(f)}}if(!d){h.sort();g.sort()}if(h.join(",")!==g.join(",")){return false}}var i=true;$.each(e,function(a){var b=objcmp(e[a],c[a]);i=i&&b;if(!i){return false}});return i}};var objkeys=function(b){var a=[];$.each(b,function(d,c){if(b.hasOwnProperty(d)){a.push(d)}});return a};var objcontains=function(c){if(!c||typeof c!="object"){re
 turn false}for(var b=1,a=arguments.length;b<a;b++){if(c.hasOwnProperty(arguments[b])){return true}}return false};var uniq=function(b){var a=b.length;var d={};for(var c=0;c<a;c++){d[b[c]]=true}return objkeys(d)};var arbor_path=function(){var a=$("script").map(function(b){var c=$(this).attr("src");if(!c){return}if(c.match(/arbor[^\/\.]*.js|dev.js/)){return c.match(/.*\//)||"/"}});if(a.length>0){return a[0]}else{return null}};
+  /* colors.js */  var Colors=(function(){var f=/#[0-9a-f]{6}/i;var b=/#(..)(..)(..)/;var c=function(h){var g=h.toString(16);return(g.length==2)?g:"0"+g};var a=function(g){return parseInt(g,16)};var d=function(g){if(!g||typeof g!="object"){return false}var h=objkeys(g).sort().join("");if(h=="abgr"){return true}};var e={CSS:{aliceblue:"#f0f8ff",antiquewhite:"#faebd7",aqua:"#00ffff",aquamarine:"#7fffd4",azure:"#f0ffff",beige:"#f5f5dc",bisque:"#ffe4c4",black:"#000000",blanchedalmond:"#ffebcd",blue:"#0000ff",blueviolet:"#8a2be2",brown:"#a52a2a",burlywood:"#deb887",cadetblue:"#5f9ea0",chartreuse:"#7fff00",chocolate:"#d2691e",coral:"#ff7f50",cornflowerblue:"#6495ed",cornsilk:"#fff8dc",crimson:"#dc143c",cyan:"#00ffff",darkblue:"#00008b",darkcyan:"#008b8b",darkgoldenrod:"#b8860b",darkgray:"#a9a9a9",darkgrey:"#a9a9a9",darkgreen:"#006400",darkkhaki:"#bdb76b",darkmagenta:"#8b008b",darkolivegreen:"#556b2f",darkorange:"#ff8c00",darkorchid:"#9932cc",darkred:"#8b0000",darksalmon:"#e9967a",darkseag
 reen:"#8fbc8f",darkslateblue:"#483d8b",darkslategray:"#2f4f4f",darkslategrey:"#2f4f4f",darkturquoise:"#00ced1",darkviolet:"#9400d3",deeppink:"#ff1493",deepskyblue:"#00bfff",dimgray:"#696969",dimgrey:"#696969",dodgerblue:"#1e90ff",firebrick:"#b22222",floralwhite:"#fffaf0",forestgreen:"#228b22",fuchsia:"#ff00ff",gainsboro:"#dcdcdc",ghostwhite:"#f8f8ff",gold:"#ffd700",goldenrod:"#daa520",gray:"#808080",grey:"#808080",green:"#008000",greenyellow:"#adff2f",honeydew:"#f0fff0",hotpink:"#ff69b4",indianred:"#cd5c5c",indigo:"#4b0082",ivory:"#fffff0",khaki:"#f0e68c",lavender:"#e6e6fa",lavenderblush:"#fff0f5",lawngreen:"#7cfc00",lemonchiffon:"#fffacd",lightblue:"#add8e6",lightcoral:"#f08080",lightcyan:"#e0ffff",lightgoldenrodyellow:"#fafad2",lightgray:"#d3d3d3",lightgrey:"#d3d3d3",lightgreen:"#90ee90",lightpink:"#ffb6c1",lightsalmon:"#ffa07a",lightseagreen:"#20b2aa",lightskyblue:"#87cefa",lightslategray:"#778899",lightslategrey:"#778899",lightsteelblue:"#b0c4de",lightyellow:"#ffffe0",lime:"#00f
 f00",limegreen:"#32cd32",linen:"#faf0e6",magenta:"#ff00ff",maroon:"#800000",mediumaquamarine:"#66cdaa",mediumblue:"#0000cd",mediumorchid:"#ba55d3",mediumpurple:"#9370d8",mediumseagreen:"#3cb371",mediumslateblue:"#7b68ee",mediumspringgreen:"#00fa9a",mediumturquoise:"#48d1cc",mediumvioletred:"#c71585",midnightblue:"#191970",mintcream:"#f5fffa",mistyrose:"#ffe4e1",moccasin:"#ffe4b5",navajowhite:"#ffdead",navy:"#000080",oldlace:"#fdf5e6",olive:"#808000",olivedrab:"#6b8e23",orange:"#ffa500",orangered:"#ff4500",orchid:"#da70d6",palegoldenrod:"#eee8aa",palegreen:"#98fb98",paleturquoise:"#afeeee",palevioletred:"#d87093",papayawhip:"#ffefd5",peachpuff:"#ffdab9",peru:"#cd853f",pink:"#ffc0cb",plum:"#dda0dd",powderblue:"#b0e0e6",purple:"#800080",red:"#ff0000",rosybrown:"#bc8f8f",royalblue:"#4169e1",saddlebrown:"#8b4513",salmon:"#fa8072",sandybrown:"#f4a460",seagreen:"#2e8b57",seashell:"#fff5ee",sienna:"#a0522d",silver:"#c0c0c0",skyblue:"#87ceeb",slateblue:"#6a5acd",slategray:"#708090",slategrey
 :"#708090",snow:"#fffafa",springgreen:"#00ff7f",steelblue:"#4682b4",tan:"#d2b48c",teal:"#008080",thistle:"#d8bfd8",tomato:"#ff6347",turquoise:"#40e0d0",violet:"#ee82ee",wheat:"#f5deb3",white:"#ffffff",whitesmoke:"#f5f5f5",yellow:"#ffff00",yellowgreen:"#9acd32"},decode:function(h){var g=arguments.length;for(var l=g-1;l>=0;l--){if(arguments[l]===undefined){g--}}var k=arguments;if(!h){return null}if(g==1&&d(h)){return h}var j=null;if(typeof h=="string"){var o=1;if(g==2){o=k[1]}var n=e.CSS[h.toLowerCase()];if(n!==undefined){h=n}var m=h.match(f);if(m){vals=h.match(b);if(!vals||!vals.length||vals.length!=4){return null}j={r:a(vals[1]),g:a(vals[2]),b:a(vals[3]),a:o}}}else{if(typeof h=="number"){if(g>=3){j={r:k[0],g:k[1],b:k[2],a:1};if(g>=4){j.a*=k[3]}}else{if(g>=1){j={r:k[0],g:k[0],b:k[0],a:1};if(g==2){j.a*=k[1]}}}}}return j},validate:function(g){if(!g||typeof g!="string"){return false}if(e.CSS[g.toLowerCase()]!==undefined){return true}if(g.match(f)){return true}return false},mix:function(
 h,g,k){var j=e.decode(h);var i=e.decode(g)},blend:function(g,j){j=(j!==undefined)?Math.max(0,Math.min(1,j)):1;var h=e.decode(g);if(!h){return null}if(j==1){return g}var h=g;if(typeof g=="string"){h=e.decode(g)}var i=objcopy(h);i.a*=j;return nano("rgba({r},{g},{b},{a})",i)},encode:function(g){if(!d(g)){g=e.decode(g);if(!d(g)){return null}}if(g.a==1){return nano("#{r}{g}{b}",{r:c(g.r),g:c(g.g),b:c(g.b)})}else{return nano("rgba({r},{g},{b},{a})",g)}}};return e})();
+  /* easing.js */  var Easing=(function(){var a={linear:function(f,e,h,g){return h*(f/g)+e},quadin:function(f,e,h,g){return h*(f/=g)*f+e},quadout:function(f,e,h,g){return -h*(f/=g)*(f-2)+e},quadinout:function(f,e,h,g){if((f/=g/2)<1){return h/2*f*f+e}return -h/2*((--f)*(f-2)-1)+e},cubicin:function(f,e,h,g){return h*(f/=g)*f*f+e},cubicout:function(f,e,h,g){return h*((f=f/g-1)*f*f+1)+e},cubicinout:function(f,e,h,g){if((f/=g/2)<1){return h/2*f*f*f+e}return h/2*((f-=2)*f*f+2)+e},quartin:function(f,e,h,g){return h*(f/=g)*f*f*f+e},quartout:function(f,e,h,g){return -h*((f=f/g-1)*f*f*f-1)+e},quartinout:function(f,e,h,g){if((f/=g/2)<1){return h/2*f*f*f*f+e}return -h/2*((f-=2)*f*f*f-2)+e},quintin:function(f,e,h,g){return h*(f/=g)*f*f*f*f+e},quintout:function(f,e,h,g){return h*((f=f/g-1)*f*f*f*f+1)+e},quintinout:function(f,e,h,g){if((f/=g/2)<1){return h/2*f*f*f*f*f+e}return h/2*((f-=2)*f*f*f*f+2)+e},sinein:function(f,e,h,g){return -h*Math.cos(f/g*(Math.PI/2))+h+e},sineout:function(f,e,h,g){retu
 rn h*Math.sin(f/g*(Math.PI/2))+e},sineinout:function(f,e,h,g){return -h/2*(Math.cos(Math.PI*f/g)-1)+e},expoin:function(f,e,h,g){return(f==0)?e:h*Math.pow(2,10*(f/g-1))+e},expoout:function(f,e,h,g){return(f==g)?e+h:h*(-Math.pow(2,-10*f/g)+1)+e},expoinout:function(f,e,h,g){if(f==0){return e}if(f==g){return e+h}if((f/=g/2)<1){return h/2*Math.pow(2,10*(f-1))+e}return h/2*(-Math.pow(2,-10*--f)+2)+e},circin:function(f,e,h,g){return -h*(Math.sqrt(1-(f/=g)*f)-1)+e},circout:function(f,e,h,g){return h*Math.sqrt(1-(f=f/g-1)*f)+e},circinout:function(f,e,h,g){if((f/=g/2)<1){return -h/2*(Math.sqrt(1-f*f)-1)+e}return h/2*(Math.sqrt(1-(f-=2)*f)+1)+e},elasticin:function(g,e,k,j){var h=1.70158;var i=0;var f=k;if(g==0){return e}if((g/=j)==1){return e+k}if(!i){i=j*0.3}if(f<Math.abs(k)){f=k;var h=i/4}else{var h=i/(2*Math.PI)*Math.asin(k/f)}return -(f*Math.pow(2,10*(g-=1))*Math.sin((g*j-h)*(2*Math.PI)/i))+e},elasticout:function(g,e,k,j){var h=1.70158;var i=0;var f=k;if(g==0){return e}if((g/=j)==1){return
  e+k}if(!i){i=j*0.3}if(f<Math.abs(k)){f=k;var h=i/4}else{var h=i/(2*Math.PI)*Math.asin(k/f)}return f*Math.pow(2,-10*g)*Math.sin((g*j-h)*(2*Math.PI)/i)+k+e},elasticinout:function(g,e,k,j){var h=1.70158;var i=0;var f=k;if(g==0){return e}if((g/=j/2)==2){return e+k}if(!i){i=j*(0.3*1.5)}if(f<Math.abs(k)){f=k;var h=i/4}else{var h=i/(2*Math.PI)*Math.asin(k/f)}if(g<1){return -0.5*(f*Math.pow(2,10*(g-=1))*Math.sin((g*j-h)*(2*Math.PI)/i))+e}return f*Math.pow(2,-10*(g-=1))*Math.sin((g*j-h)*(2*Math.PI)/i)*0.5+k+e},backin:function(f,e,i,h,g){if(g==undefined){g=1.70158}return i*(f/=h)*f*((g+1)*f-g)+e},backout:function(f,e,i,h,g){if(g==undefined){g=1.70158}return i*((f=f/h-1)*f*((g+1)*f+g)+1)+e},backinout:function(f,e,i,h,g){if(g==undefined){g=1.70158}if((f/=h/2)<1){return i/2*(f*f*(((g*=(1.525))+1)*f-g))+e}return i/2*((f-=2)*f*(((g*=(1.525))+1)*f+g)+2)+e},bouncein:function(f,e,h,g){return h-a.bounceOut(g-f,0,h,g)+e},bounceout:function(f,e,h,g){if((f/=g)<(1/2.75)){return h*(7.5625*f*f)+e}else{if(f
 <(2/2.75)){return h*(7.5625*(f-=(1.5/2.75))*f+0.75)+e}else{if(f<(2.5/2.75)){return h*(7.5625*(f-=(2.25/2.75))*f+0.9375)+e}else{return h*(7.5625*(f-=(2.625/2.75))*f+0.984375)+e}}}},bounceinout:function(f,e,h,g){if(f<g/2){return a.bounceIn(f*2,0,h,g)*0.5+e}return a.bounceOut(f*2-g,0,h,g)*0.5+h*0.5+e}};return a})();
+  /*  tween.js */  var Tween=function(){var a={};var c=true;var b={init:function(){return b},busy:function(){var e=false;for(var d in a){e=true;break}return e},to:function(g,e,p){var f=new Date().valueOf();var d={};var q={from:{},to:{},colors:{},node:g,t0:f,t1:f+e*1000,dur:e*1000};var o="linear";for(var j in p){if(j=="easing"){var h=p[j].toLowerCase();if(h in Easing){o=h}continue}else{if(j=="delay"){var m=(p[j]||0)*1000;q.t0+=m;q.t1+=m;continue}}if(Colors.validate(p[j])){q.colors[j]=[Colors.decode(g.data[j]),Colors.decode(p[j]),p[j]];d[j]=true}else{q.from[j]=(g.data[j]!=undefined)?g.data[j]:p[j];q.to[j]=p[j];d[j]=true}}q.ease=Easing[o];if(a[g._id]===undefined){a[g._id]=[]}a[g._id].push(q);if(a.length>1){for(var l=a.length-2;l>=0;l++){var n=a[l];for(var j in n.to){if(j in d){delete n.to[j]}else{d[j]=true}}for(var j in n.colors){if(j in d){delete n.colors[j]}else{d[j]=true}}if($.isEmptyObject(n.colors)&&$.isEmptyObject(n.to)){a.splice(l,1)}}}c=false},interpolate:function(e,h,i,g){g=(g
 ||"").toLowerCase();var d=Easing.linear;if(g in Easing){d=Easing[g]}var f=d(e,0,1,1);if(Colors.validate(h)&&Colors.validate(i)){return lerpRGB(f,h,i)}else{if(!isNaN(h)){return lerpNumber(f,h,i)}else{if(typeof h=="string"){return(f<0.5)?h:i}}}},tick:function(){var f=true;for(var d in a){f=false;break}if(f){return}var e=new Date().valueOf();$.each(a,function(i,h){var g=false;$.each(h,function(p,t){var o=t.ease((e-t.t0),0,1,t.dur);o=Math.min(1,o);var r=t.from;var s=t.to;var j=t.colors;var l=t.node.data;var m=(o==1);for(var n in s){switch(typeof s[n]){case"number":l[n]=lerpNumber(o,r[n],s[n]);if(n=="alpha"){l[n]=Math.max(0,Math.min(1,l[n]))}break;case"string":if(m){l[n]=s[n]}break}}for(var n in j){if(m){l[n]=j[n][2]}else{var q=lerpRGB(o,j[n][0],j[n][1]);l[n]=Colors.encode(q)}}if(m){t.completed=true;g=true}});if(g){a[i]=$.map(h,function(j){if(!j.completed){return j}});if(a[i].length==0){delete a[i]}}});c=$.isEmptyObject(a);return c}};return b.init()};var lerpNumber=function(a,c,b){return
  c+a*(b-c)};var lerpRGB=function(b,d,c){b=Math.max(Math.min(b,1),0);var a={};$.each("rgba".split(""),function(e,f){a[f]=Math.round(d[f]+b*(c[f]-d[f]))});return a};
+
+  arbor = (typeof(arbor)!=='undefined') ? arbor : {}
+  $.extend(arbor, {
+    // not really user-serviceable; use the ParticleSystem’s .tween* methods instead
+    Tween:Tween,
+    
+    // immutable object with useful methods
+    colors:{
+      CSS:Colors.CSS,           // dictionary: {colorname:#fef2e2,...}
+      validate:Colors.validate, // ƒ(str) -> t/f
+      decode:Colors.decode,     // ƒ(hexString_or_cssColor) -> {r,g,b,a}
+      encode:Colors.encode,     // ƒ({r,g,b,a}) -> hexOrRgbaString
+      blend:Colors.blend        // ƒ(color, opacity) -> rgbaString
+    }
+  })
+  
+})(this.jQuery)
+
+
+
+
+ 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/js/arbor.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/arbor.js b/storm-core/src/ui/public/js/arbor.js
new file mode 100644
index 0000000..a25dd24
--- /dev/null
+++ b/storm-core/src/ui/public/js/arbor.js
@@ -0,0 +1,67 @@
+//
+//  arbor.js - version 0.91
+//  a graph vizualization toolkit
+//
+//  Copyright (c) 2011 Samizdat Drafting Co.
+//  Physics code derived from springy.js, copyright (c) 2010 Dennis Hotson
+// 
+//  Permission is hereby granted, free of charge, to any person
+//  obtaining a copy of this software and associated documentation
+//  files (the "Software"), to deal in the Software without
+//  restriction, including without limitation the rights to use,
+//  copy, modify, merge, publish, distribute, sublicense, and/or sell
+//  copies of the Software, and to permit persons to whom the
+//  Software is furnished to do so, subject to the following
+//  conditions:
+// 
+//  The above copyright notice and this permission notice shall be
+//  included in all copies or substantial portions of the Software.
+// 
+//  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+//  EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES
+//  OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+//  NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT
+//  HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+//  WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
+//  FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+//  OTHER DEALINGS IN THE SOFTWARE.
+//
+
+(function($){
+
+  /*        etc.js */  var trace=function(msg){if(typeof(window)=="undefined"||!window.console){return}var len=arguments.length;var args=[];for(var i=0;i<len;i++){args.push("arguments["+i+"]")}eval("console.log("+args.join(",")+")")};var dirname=function(a){var b=a.replace(/^\/?(.*?)\/?$/,"$1").split("/");b.pop();return"/"+b.join("/")};var basename=function(b){var c=b.replace(/^\/?(.*?)\/?$/,"$1").split("/");var a=c.pop();if(a==""){return null}else{return a}};var _ordinalize_re=/(\d)(?=(\d\d\d)+(?!\d))/g;var ordinalize=function(a){var b=""+a;if(a<11000){b=(""+a).replace(_ordinalize_re,"$1,")}else{if(a<1000000){b=Math.floor(a/1000)+"k"}else{if(a<1000000000){b=(""+Math.floor(a/1000)).replace(_ordinalize_re,"$1,")+"m"}}}return b};var nano=function(a,b){return a.replace(/\{([\w\-\.]*)}/g,function(f,c){var d=c.split("."),e=b[d.shift()];$.each(d,function(){if(e.hasOwnProperty(this)){e=e[this]}else{e=f}});return e})};var objcopy=function(a){if(a===undefined){return undefined}if(a===null){r
 eturn null}if(a.parentNode){return a}switch(typeof a){case"string":return a.substring(0);break;case"number":return a+0;break;case"boolean":return a===true;break}var b=($.isArray(a))?[]:{};$.each(a,function(d,c){b[d]=objcopy(c)});return b};var objmerge=function(d,b){d=d||{};b=b||{};var c=objcopy(d);for(var a in b){c[a]=b[a]}return c};var objcmp=function(e,c,d){if(!e||!c){return e===c}if(typeof e!=typeof c){return false}if(typeof e!="object"){return e===c}else{if($.isArray(e)){if(!($.isArray(c))){return false}if(e.length!=c.length){return false}}else{var h=[];for(var f in e){if(e.hasOwnProperty(f)){h.push(f)}}var g=[];for(var f in c){if(c.hasOwnProperty(f)){g.push(f)}}if(!d){h.sort();g.sort()}if(h.join(",")!==g.join(",")){return false}}var i=true;$.each(e,function(a){var b=objcmp(e[a],c[a]);i=i&&b;if(!i){return false}});return i}};var objkeys=function(b){var a=[];$.each(b,function(d,c){if(b.hasOwnProperty(d)){a.push(d)}});return a};var objcontains=function(c){if(!c||typeof c!="object"
 ){return false}for(var b=1,a=arguments.length;b<a;b++){if(c.hasOwnProperty(arguments[b])){return true}}return false};var uniq=function(b){var a=b.length;var d={};for(var c=0;c<a;c++){d[b[c]]=true}return objkeys(d)};var arbor_path=function(){var a=$("script").map(function(b){var c=$(this).attr("src");if(!c){return}if(c.match(/arbor[^\/\.]*.js|dev.js/)){return c.match(/.*\//)||"/"}});if(a.length>0){return a[0]}else{return null}};
+  /*     kernel.js */  var Kernel=function(b){var k=window.location.protocol=="file:"&&navigator.userAgent.toLowerCase().indexOf("chrome")>-1;var a=(window.Worker!==undefined&&!k);var i=null;var c=null;var f=[];f.last=new Date();var l=null;var e=null;var d=null;var h=null;var g=false;var j={system:b,tween:null,nodes:{},init:function(){if(typeof(Tween)!="undefined"){c=Tween()}else{if(typeof(arbor.Tween)!="undefined"){c=arbor.Tween()}else{c={busy:function(){return false},tick:function(){return true},to:function(){trace("Please include arbor-tween.js to enable tweens");c.to=function(){};return}}}}j.tween=c;var m=b.parameters();if(a){trace("using web workers");l=setInterval(j.screenUpdate,m.timeout);i=new Worker(arbor_path()+"arbor.js");i.onmessage=j.workerMsg;i.onerror=function(n){trace("physics:",n)};i.postMessage({type:"physics",physics:objmerge(m,{timeout:Math.ceil(m.timeout)})})}else{trace("couldn't use web workers, be careful...");i=Physics(m.dt,m.stiffness,m.repulsion,m.friction,
 j.system._updateGeometry);j.start()}return j},graphChanged:function(m){if(a){i.postMessage({type:"changes",changes:m})}else{i._update(m)}j.start()},particleModified:function(n,m){if(a){i.postMessage({type:"modify",id:n,mods:m})}else{i.modifyNode(n,m)}j.start()},physicsModified:function(m){if(!isNaN(m.timeout)){if(a){clearInterval(l);l=setInterval(j.screenUpdate,m.timeout)}else{clearInterval(d);d=null}}if(a){i.postMessage({type:"sys",param:m})}else{i.modifyPhysics(m)}j.start()},workerMsg:function(n){var m=n.data.type;if(m=="geometry"){j.workerUpdate(n.data)}else{trace("physics:",n.data)}},_lastPositions:null,workerUpdate:function(m){j._lastPositions=m;j._lastBounds=m.bounds},_lastFrametime:new Date().valueOf(),_lastBounds:null,_currentRenderer:null,screenUpdate:function(){var n=new Date().valueOf();var m=false;if(j._lastPositions!==null){j.system._updateGeometry(j._lastPositions);j._lastPositions=null;m=true}if(c&&c.busy()){m=true}if(j.system._updateBounds(j._lastBounds)){m=true}if(m
 ){var o=j.system.renderer;if(o!==undefined){if(o!==e){o.init(j.system);e=o}if(c){c.tick()}o.redraw();var p=f.last;f.last=new Date();f.push(f.last-p);if(f.length>50){f.shift()}}}},physicsUpdate:function(){if(c){c.tick()}i.tick();var n=j.system._updateBounds();if(c&&c.busy()){n=true}var o=j.system.renderer;var m=new Date();var o=j.system.renderer;if(o!==undefined){if(o!==e){o.init(j.system);e=o}o.redraw({timestamp:m})}var q=f.last;f.last=m;f.push(f.last-q);if(f.length>50){f.shift()}var p=i.systemEnergy();if((p.mean+p.max)/2<0.05){if(h===null){h=new Date().valueOf()}if(new Date().valueOf()-h>1000){clearInterval(d);d=null}else{}}else{h=null}},fps:function(n){if(n!==undefined){var q=1000/Math.max(1,targetFps);j.physicsModified({timeout:q})}var r=0;for(var p=0,o=f.length;p<o;p++){r+=f[p]}var m=r/Math.max(1,f.length);if(!isNaN(m)){return Math.round(1000/m)}else{return 0}},start:function(m){if(d!==null){return}if(g&&!m){return}g=false;if(a){i.postMessage({type:"start"})}else{h=null;d=setInt
 erval(j.physicsUpdate,j.system.parameters().timeout)}},stop:function(){g=true;if(a){i.postMessage({type:"stop"})}else{if(d!==null){clearInterval(d);d=null}}}};return j.init()};
+  /*      atoms.js */  var Node=function(a){this._id=_nextNodeId++;this.data=a||{};this._mass=(a.mass!==undefined)?a.mass:1;this._fixed=(a.fixed===true)?true:false;this._p=new Point((typeof(a.x)=="number")?a.x:null,(typeof(a.y)=="number")?a.y:null);delete this.data.x;delete this.data.y;delete this.data.mass;delete this.data.fixed};var _nextNodeId=1;var Edge=function(b,c,a){this._id=_nextEdgeId--;this.source=b;this.target=c;this.length=(a.length!==undefined)?a.length:1;this.data=(a!==undefined)?a:{};delete this.data.length};var _nextEdgeId=-1;var Particle=function(a,b){this.p=a;this.m=b;this.v=new Point(0,0);this.f=new Point(0,0)};Particle.prototype.applyForce=function(a){this.f=this.f.add(a.divide(this.m))};var Spring=function(c,b,d,a){this.point1=c;this.point2=b;this.length=d;this.k=a};Spring.prototype.distanceToParticle=function(a){var c=that.point2.p.subtract(that.point1.p).normalize().normal();var b=a.p.subtract(that.point1.p);return Math.abs(b.x*c.x+b.y*c.y)};var Point=function
 (a,b){if(a&&a.hasOwnProperty("y")){b=a.y;a=a.x}this.x=a;this.y=b};Point.random=function(a){a=(a!==undefined)?a:5;return new Point(2*a*(Math.random()-0.5),2*a*(Math.random()-0.5))};Point.prototype={exploded:function(){return(isNaN(this.x)||isNaN(this.y))},add:function(a){return new Point(this.x+a.x,this.y+a.y)},subtract:function(a){return new Point(this.x-a.x,this.y-a.y)},multiply:function(a){return new Point(this.x*a,this.y*a)},divide:function(a){return new Point(this.x/a,this.y/a)},magnitude:function(){return Math.sqrt(this.x*this.x+this.y*this.y)},normal:function(){return new Point(-this.y,this.x)},normalize:function(){return this.divide(this.magnitude())}};
+  /*     system.js */  var ParticleSystem=function(d,p,e,f,t,l,q){var j=[];var h=null;var k=0;var u=null;var m=0.04;var i=[20,20,20,20];var n=null;var o=null;if(typeof p=="object"){var s=p;e=s.friction;d=s.repulsion;t=s.fps;l=s.dt;p=s.stiffness;f=s.gravity;q=s.precision}e=isNaN(e)?0.5:e;d=isNaN(d)?1000:d;t=isNaN(t)?55:t;p=isNaN(p)?600:p;l=isNaN(l)?0.02:l;q=isNaN(q)?0.6:q;f=(f===true);var r=(t!==undefined)?1000/t:1000/50;var b={repulsion:d,stiffness:p,friction:e,dt:l,gravity:f,precision:q,timeout:r};var a;var c={renderer:null,tween:null,nodes:{},edges:{},adjacency:{},names:{},kernel:null};var g={parameters:function(v){if(v!==undefined){if(!isNaN(v.precision)){v.precision=Math.max(0,Math.min(1,v.precision))}$.each(b,function(x,w){if(v[x]!==undefined){b[x]=v[x]}});c.kernel.physicsModified(v)}return b},fps:function(v){if(v===undefined){return c.kernel.fps()}else{g.parameters({timeout:1000/(v||50)})}},start:function(){c.kernel.start()},stop:function(){c.kernel.stop()},addNode:function(w,
 B){B=B||{};var C=c.names[w];if(C){C.data=B;return C}else{if(w!=undefined){var v=(B.x!=undefined)?B.x:null;var D=(B.y!=undefined)?B.y:null;var A=(B.fixed)?1:0;var z=new Node(B);z.name=w;c.names[w]=z;c.nodes[z._id]=z;j.push({t:"addNode",id:z._id,m:z.mass,x:v,y:D,f:A});g._notify();return z}}},pruneNode:function(w){var v=g.getNode(w);if(typeof(c.nodes[v._id])!=="undefined"){delete c.nodes[v._id];delete c.names[v.name]}$.each(c.edges,function(y,x){if(x.source._id===v._id||x.target._id===v._id){g.pruneEdge(x)}});j.push({t:"dropNode",id:v._id});g._notify()},getNode:function(v){if(v._id!==undefined){return v}else{if(typeof v=="string"||typeof v=="number"){return c.names[v]}}},eachNode:function(v){$.each(c.nodes,function(y,x){if(x._p.x==null||x._p.y==null){return}var w=(u!==null)?g.toScreen(x._p):x._p;v.call(g,x,w)})},addEdge:function(z,A,y){z=g.getNode(z)||g.addNode(z);A=g.getNode(A)||g.addNode(A);y=y||{};var x=new Edge(z,A,y);var B=z._id;var C=A._id;c.adjacency[B]=c.adjacency[B]||{};c.adja
 cency[B][C]=c.adjacency[B][C]||[];var w=(c.adjacency[B][C].length>0);if(w){$.extend(c.adjacency[B][C].data,x.data);return}else{c.edges[x._id]=x;c.adjacency[B][C].push(x);var v=(x.length!==undefined)?x.length:1;j.push({t:"addSpring",id:x._id,fm:B,to:C,l:v});g._notify()}return x},pruneEdge:function(A){j.push({t:"dropSpring",id:A._id});delete c.edges[A._id];for(var v in c.adjacency){for(var B in c.adjacency[v]){var w=c.adjacency[v][B];for(var z=w.length-1;z>=0;z--){if(c.adjacency[v][B][z]._id===A._id){c.adjacency[v][B].splice(z,1)}}}}g._notify()},getEdges:function(w,v){w=g.getNode(w);v=g.getNode(v);if(!w||!v){return[]}if(typeof(c.adjacency[w._id])!=="undefined"&&typeof(c.adjacency[w._id][v._id])!=="undefined"){return c.adjacency[w._id][v._id]}return[]},getEdgesFrom:function(v){v=g.getNode(v);if(!v){return[]}if(typeof(c.adjacency[v._id])!=="undefined"){var w=[];$.each(c.adjacency[v._id],function(y,x){w=w.concat(x)});return w}return[]},getEdgesTo:function(v){v=g.getNode(v);if(!v){return[
 ]}var w=[];$.each(c.edges,function(y,x){if(x.target==v){w.push(x)}});return w},eachEdge:function(v){$.each(c.edges,function(z,x){var y=c.nodes[x.source._id]._p;var w=c.nodes[x.target._id]._p;if(y.x==null||w.x==null){return}y=(u!==null)?g.toScreen(y):y;w=(u!==null)?g.toScreen(w):w;if(y&&w){v.call(g,x,y,w)}})},prune:function(w){var v={dropped:{nodes:[],edges:[]}};if(w===undefined){$.each(c.nodes,function(y,x){v.dropped.nodes.push(x);g.pruneNode(x)})}else{g.eachNode(function(y){var x=w.call(g,y,{from:g.getEdgesFrom(y),to:g.getEdgesTo(y)});if(x){v.dropped.nodes.push(y);g.pruneNode(y)}})}return v},graft:function(w){var v={added:{nodes:[],edges:[]}};if(w.nodes){$.each(w.nodes,function(y,x){var z=g.getNode(y);if(z){z.data=x}else{v.added.nodes.push(g.addNode(y,x))}c.kernel.start()})}if(w.edges){$.each(w.edges,function(z,x){var y=g.getNode(z);if(!y){v.added.nodes.push(g.addNode(z,{}))}$.each(x,function(D,A){var C=g.getNode(D);if(!C){v.added.nodes.push(g.addNode(D,{}))}var B=g.getEdges(z,D);i
 f(B.length>0){B[0].data=A}else{v.added.edges.push(g.addEdge(z,D,A))}})})}return v},merge:function(w){var v={added:{nodes:[],edges:[]},dropped:{nodes:[],edges:[]}};$.each(c.edges,function(A,z){if((w.edges[z.source.name]===undefined||w.edges[z.source.name][z.target.name]===undefined)){g.pruneEdge(z);v.dropped.edges.push(z)}});var y=g.prune(function(A,z){if(w.nodes[A.name]===undefined){v.dropped.nodes.push(A);return true}});var x=g.graft(w);v.added.nodes=v.added.nodes.concat(x.added.nodes);v.added.edges=v.added.edges.concat(x.added.edges);v.dropped.nodes=v.dropped.nodes.concat(y.dropped.nodes);v.dropped.edges=v.dropped.edges.concat(y.dropped.edges);return v},tweenNode:function(y,v,x){var w=g.getNode(y);if(w){c.tween.to(w,v,x)}},tweenEdge:function(w,v,z,y){if(y===undefined){g._tweenEdge(w,v,z)}else{var x=g.getEdges(w,v);$.each(x,function(A,B){g._tweenEdge(B,z,y)})}},_tweenEdge:function(w,v,x){if(w&&w._id!==undefined){c.tween.to(w,v,x)}},_updateGeometry:function(y){if(y!=undefined){var v
 =(y.epoch<k);a=y.energy;var z=y.geometry;if(z!==undefined){for(var x=0,w=z.length/3;x<w;x++){var A=z[3*x];if(v&&c.nodes[A]==undefined){continue}c.nodes[A]._p.x=z[3*x+1];c.nodes[A]._p.y=z[3*x+2]}}}},screen:function(v){if(v==undefined){return{size:(u)?objcopy(u):undefined,padding:i.concat(),step:m}}if(v.size!==undefined){g.screenSize(v.size.width,v.size.height)}if(!isNaN(v.step)){g.screenStep(v.step)}if(v.padding!==undefined){g.screenPadding(v.padding)}},screenSize:function(v,w){u={width:v,height:w};g._updateBounds()},screenPadding:function(y,z,v,w){if($.isArray(y)){trbl=y}else{trbl=[y,z,v,w]}var A=trbl[0];var x=trbl[1];var B=trbl[2];if(x===undefined){trbl=[A,A,A,A]}else{if(B==undefined){trbl=[A,x,A,x]}}i=trbl},screenStep:function(v){m=v},toScreen:function(x){if(!n||!u){return}var w=i||[0,0,0,0];var v=n.bottomright.subtract(n.topleft);var z=w[3]+x.subtract(n.topleft).divide(v.x).x*(u.width-(w[1]+w[3]));var y=w[0]+x.subtract(n.topleft).divide(v.y).y*(u.height-(w[0]+w[2]));return arbor.
 Point(z,y)},fromScreen:function(z){if(!n||!u){return}var y=i||[0,0,0,0];var x=n.bottomright.subtract(n.topleft);var w=(z.x-y[3])/(u.width-(y[1]+y[3]))*x.x+n.topleft.x;var v=(z.y-y[0])/(u.height-(y[0]+y[2]))*x.y+n.topleft.y;return arbor.Point(w,v)},_updateBounds:function(w){if(u===null){return}if(w){o=w}else{o=g.bounds()}var z=new Point(o.bottomright.x,o.bottomright.y);var y=new Point(o.topleft.x,o.topleft.y);var B=z.subtract(y);var v=y.add(B.divide(2));var x=4;var D=new Point(Math.max(B.x,x),Math.max(B.y,x));o.topleft=v.subtract(D.divide(2));o.bottomright=v.add(D.divide(2));if(!n){if($.isEmptyObject(c.nodes)){return false}n=o;return true}var C=m;_newBounds={bottomright:n.bottomright.add(o.bottomright.subtract(n.bottomright).multiply(C)),topleft:n.topleft.add(o.topleft.subtract(n.topleft).multiply(C))};var A=new Point(n.topleft.subtract(_newBounds.topleft).magnitude(),n.bottomright.subtract(_newBounds.bottomright).magnitude());if(A.x*u.width>1||A.y*u.height>1){n=_newBounds;return tru
 e}else{return false}},energy:function(){return a},bounds:function(){var w=null;var v=null;$.each(c.nodes,function(z,y){if(!w){w=new Point(y._p);v=new Point(y._p);return}var x=y._p;if(x.x===null||x.y===null){return}if(x.x>w.x){w.x=x.x}if(x.y>w.y){w.y=x.y}if(x.x<v.x){v.x=x.x}if(x.y<v.y){v.y=x.y}});if(w&&v){return{bottomright:w,topleft:v}}else{return{topleft:new Point(-1,-1),bottomright:new Point(1,1)}}},nearest:function(x){if(u!==null){x=g.fromScreen(x)}var w={node:null,point:null,distance:null};var v=g;$.each(c.nodes,function(B,y){var z=y._p;if(z.x===null||z.y===null){return}var A=z.subtract(x).magnitude();if(w.distance===null||A<w.distance){w={node:y,point:z,distance:A};if(u!==null){w.screenPoint=g.toScreen(z)}}});if(w.node){if(u!==null){w.distance=g.toScreen(w.node.p).subtract(g.toScreen(x)).magnitude()}return w}else{return null}},_notify:function(){if(h===null){k++}else{clearTimeout(h)}h=setTimeout(g._synchronize,20)},_synchronize:function(){if(j.length>0){c.kernel.graphChanged(j)
 ;j=[];h=null}},};c.kernel=Kernel(g);c.tween=c.kernel.tween||null;Node.prototype.__defineGetter__("p",function(){var w=this;var v={};v.__defineGetter__("x",function(){return w._p.x});v.__defineSetter__("x",function(x){c.kernel.particleModified(w._id,{x:x})});v.__defineGetter__("y",function(){return w._p.y});v.__defineSetter__("y",function(x){c.kernel.particleModified(w._id,{y:x})});v.__proto__=Point.prototype;return v});Node.prototype.__defineSetter__("p",function(v){this._p.x=v.x;this._p.y=v.y;c.kernel.particleModified(this._id,{x:v.x,y:v.y})});Node.prototype.__defineGetter__("mass",function(){return this._mass});Node.prototype.__defineSetter__("mass",function(v){this._mass=v;c.kernel.particleModified(this._id,{m:v})});Node.prototype.__defineSetter__("tempMass",function(v){c.kernel.particleModified(this._id,{_m:v})});Node.prototype.__defineGetter__("fixed",function(){return this._fixed});Node.prototype.__defineSetter__("fixed",function(v){this._fixed=v;c.kernel.particleModified(this
 ._id,{f:v?1:0})});return g};
+  /* barnes-hut.js */  var BarnesHutTree=function(){var b=[];var a=0;var e=null;var d=0.5;var c={init:function(g,h,f){d=f;a=0;e=c._newBranch();e.origin=g;e.size=h.subtract(g)},insert:function(j){var f=e;var g=[j];while(g.length){var h=g.shift();var m=h._m||h.m;var p=c._whichQuad(h,f);if(f[p]===undefined){f[p]=h;f.mass+=m;if(f.p){f.p=f.p.add(h.p.multiply(m))}else{f.p=h.p.multiply(m)}}else{if("origin" in f[p]){f.mass+=(m);if(f.p){f.p=f.p.add(h.p.multiply(m))}else{f.p=h.p.multiply(m)}f=f[p];g.unshift(h)}else{var l=f.size.divide(2);var n=new Point(f.origin);if(p[0]=="s"){n.y+=l.y}if(p[1]=="e"){n.x+=l.x}var o=f[p];f[p]=c._newBranch();f[p].origin=n;f[p].size=l;f.mass=m;f.p=h.p.multiply(m);f=f[p];if(o.p.x===h.p.x&&o.p.y===h.p.y){var k=l.x*0.08;var i=l.y*0.08;o.p.x=Math.min(n.x+l.x,Math.max(n.x,o.p.x-k/2+Math.random()*k));o.p.y=Math.min(n.y+l.y,Math.max(n.y,o.p.y-i/2+Math.random()*i))}g.push(o);g.unshift(h)}}}},applyForces:function(m,g){var f=[e];while(f.length){node=f.shift();if(node===und
 efined){continue}if(m===node){continue}if("f" in node){var k=m.p.subtract(node.p);var l=Math.max(1,k.magnitude());var i=((k.magnitude()>0)?k:Point.random(1)).normalize();m.applyForce(i.multiply(g*(node._m||node.m)).divide(l*l))}else{var j=m.p.subtract(node.p.divide(node.mass)).magnitude();var h=Math.sqrt(node.size.x*node.size.y);if(h/j>d){f.push(node.ne);f.push(node.nw);f.push(node.se);f.push(node.sw)}else{var k=m.p.subtract(node.p.divide(node.mass));var l=Math.max(1,k.magnitude());var i=((k.magnitude()>0)?k:Point.random(1)).normalize();m.applyForce(i.multiply(g*(node.mass)).divide(l*l))}}}},_whichQuad:function(i,f){if(i.p.exploded()){return null}var h=i.p.subtract(f.origin);var g=f.size.divide(2);if(h.y<g.y){if(h.x<g.x){return"nw"}else{return"ne"}}else{if(h.x<g.x){return"sw"}else{return"se"}}},_newBranch:function(){if(b[a]){var f=b[a];f.ne=f.nw=f.se=f.sw=undefined;f.mass=0;delete f.p}else{f={origin:null,size:null,nw:undefined,ne:undefined,sw:undefined,se:undefined,mass:0};b[a]=f}a+
 +;return f}};return c};
+  /*    physics.js */  var Physics=function(a,m,n,e,h){var f=BarnesHutTree();var c={particles:{},springs:{}};var l={particles:{}};var o=[];var k=[];var d=0;var b={sum:0,max:0,mean:0};var g={topleft:new Point(-1,-1),bottomright:new Point(1,1)};var j=1000;var i={stiffness:(m!==undefined)?m:1000,repulsion:(n!==undefined)?n:600,friction:(e!==undefined)?e:0.3,gravity:false,dt:(a!==undefined)?a:0.02,theta:0.4,init:function(){return i},modifyPhysics:function(p){$.each(["stiffness","repulsion","friction","gravity","dt","precision"],function(r,s){if(p[s]!==undefined){if(s=="precision"){i.theta=1-p[s];return}i[s]=p[s];if(s=="stiffness"){var q=p[s];$.each(c.springs,function(u,t){t.k=q})}}})},addNode:function(u){var t=u.id;var q=u.m;var p=g.bottomright.x-g.topleft.x;var s=g.bottomright.y-g.topleft.y;var r=new Point((u.x!=null)?u.x:g.topleft.x+p*Math.random(),(u.y!=null)?u.y:g.topleft.y+s*Math.random());c.particles[t]=new Particle(r,q);c.particles[t].connections=0;c.particles[t].fixed=(u.f===1);
 l.particles[t]=c.particles[t];o.push(c.particles[t])},dropNode:function(s){var r=s.id;var q=c.particles[r];var p=$.inArray(q,o);if(p>-1){o.splice(p,1)}delete c.particles[r];delete l.particles[r]},modifyNode:function(r,p){if(r in c.particles){var q=c.particles[r];if("x" in p){q.p.x=p.x}if("y" in p){q.p.y=p.y}if("m" in p){q.m=p.m}if("f" in p){q.fixed=(p.f===1)}if("_m" in p){if(q._m===undefined){q._m=q.m}q.m=p._m}}},addSpring:function(t){var s=t.id;var p=t.l;var r=c.particles[t.fm];var q=c.particles[t.to];if(r!==undefined&&q!==undefined){c.springs[s]=new Spring(r,q,p,i.stiffness);k.push(c.springs[s]);r.connections++;q.connections++;delete l.particles[t.fm];delete l.particles[t.to]}},dropSpring:function(s){var r=s.id;var q=c.springs[r];q.point1.connections--;q.point2.connections--;var p=$.inArray(q,k);if(p>-1){k.splice(p,1)}delete c.springs[r]},_update:function(p){d++;$.each(p,function(q,r){if(r.t in i){i[r.t](r)}});return d},tick:function(){i.tendParticles();i.eulerIntegrator(i.dt);i.t
 ock()},tock:function(){var p=[];$.each(c.particles,function(r,q){p.push(r);p.push(q.p.x);p.push(q.p.y)});if(h){h({geometry:p,epoch:d,energy:b,bounds:g})}},tendParticles:function(){$.each(c.particles,function(q,p){if(p._m!==undefined){if(Math.abs(p.m-p._m)<1){p.m=p._m;delete p._m}else{p.m*=0.98}}p.v.x=p.v.y=0})},eulerIntegrator:function(p){if(i.repulsion>0){if(i.theta>0){i.applyBarnesHutRepulsion()}else{i.applyBruteForceRepulsion()}}if(i.stiffness>0){i.applySprings()}i.applyCenterDrift();if(i.gravity){i.applyCenterGravity()}i.updateVelocity(p);i.updatePosition(p)},applyBruteForceRepulsion:function(){$.each(c.particles,function(q,p){$.each(c.particles,function(s,r){if(p!==r){var u=p.p.subtract(r.p);var v=Math.max(1,u.magnitude());var t=((u.magnitude()>0)?u:Point.random(1)).normalize();p.applyForce(t.multiply(i.repulsion*(r._m||r.m)*0.5).divide(v*v*0.5));r.applyForce(t.multiply(i.repulsion*(p._m||p.m)*0.5).divide(v*v*-0.5))}})})},applyBarnesHutRepulsion:function(){if(!g.topleft||!g.bot
 tomright){return}var q=new Point(g.bottomright);var p=new Point(g.topleft);f.init(p,q,i.theta);$.each(c.particles,function(s,r){f.insert(r)});$.each(c.particles,function(s,r){f.applyForces(r,i.repulsion)})},applySprings:function(){$.each(c.springs,function(t,p){var s=p.point2.p.subtract(p.point1.p);var q=p.length-s.magnitude();var r=((s.magnitude()>0)?s:Point.random(1)).normalize();p.point1.applyForce(r.multiply(p.k*q*-0.5));p.point2.applyForce(r.multiply(p.k*q*0.5))})},applyCenterDrift:function(){var q=0;var r=new Point(0,0);$.each(c.particles,function(t,s){r.add(s.p);q++});if(q==0){return}var p=r.divide(-q);$.each(c.particles,function(t,s){s.applyForce(p)})},applyCenterGravity:function(){$.each(c.particles,function(r,p){var q=p.p.multiply(-1);p.applyForce(q.multiply(i.repulsion/100))})},updateVelocity:function(p){$.each(c.particles,function(t,q){if(q.fixed){q.v=new Point(0,0);q.f=new Point(0,0);return}var s=q.v.magnitude();q.v=q.v.add(q.f.multiply(p)).multiply(1-i.friction);q.f.x=
 q.f.y=0;var r=q.v.magnitude();if(r>j){q.v=q.v.divide(r*r)}})},updatePosition:function(q){var r=0,p=0,u=0;var t=null;var s=null;$.each(c.particles,function(w,v){v.p=v.p.add(v.v.multiply(q));var x=v.v.magnitude();var z=x*x;r+=z;p=Math.max(z,p);u++;if(!t){t=new Point(v.p.x,v.p.y);s=new Point(v.p.x,v.p.y);return}var y=v.p;if(y.x===null||y.y===null){return}if(y.x>t.x){t.x=y.x}if(y.y>t.y){t.y=y.y}if(y.x<s.x){s.x=y.x}if(y.y<s.y){s.y=y.y}});b={sum:r,max:p,mean:r/u,n:u};g={topleft:s||new Point(-1,-1),bottomright:t||new Point(1,1)}},systemEnergy:function(p){return b}};return i.init()};var _nearParticle=function(b,c){var c=c||0;var a=b.x;var f=b.y;var e=c*2;return new Point(a-c+Math.random()*e,f-c+Math.random()*e)};
+
+  // if called as a worker thread, set up a run loop for the Physics object and bail out
+  if (typeof(window)=='undefined') return (function(){
+  /* hermetic.js */  $={each:function(d,e){if($.isArray(d)){for(var c=0,b=d.length;c<b;c++){e(c,d[c])}}else{for(var a in d){e(a,d[a])}}},map:function(a,c){var b=[];$.each(a,function(f,e){var d=c(e);if(d!==undefined){b.push(d)}});return b},extend:function(c,b){if(typeof b!="object"){return c}for(var a in b){if(b.hasOwnProperty(a)){c[a]=b[a]}}return c},isArray:function(a){if(!a){return false}return(a.constructor.toString().indexOf("Array")!=-1)},inArray:function(c,a){for(var d=0,b=a.length;d<b;d++){if(a[d]===c){return d}}return -1},isEmptyObject:function(a){if(typeof a!=="object"){return false}var b=true;$.each(a,function(c,d){b=false});return b},};
+  /*     worker.js */  var PhysicsWorker=function(){var b=20;var a=null;var d=null;var c=null;var g=[];var f=new Date().valueOf();var e={init:function(h){e.timeout(h.timeout);a=Physics(h.dt,h.stiffness,h.repulsion,h.friction,e.tock);return e},timeout:function(h){if(h!=b){b=h;if(d!==null){e.stop();e.go()}}},go:function(){if(d!==null){return}c=null;d=setInterval(e.tick,b)},stop:function(){if(d===null){return}clearInterval(d);d=null},tick:function(){a.tick();var h=a.systemEnergy();if((h.mean+h.max)/2<0.05){if(c===null){c=new Date().valueOf()}if(new Date().valueOf()-c>1000){e.stop()}else{}}else{c=null}},tock:function(h){h.type="geometry";postMessage(h)},modifyNode:function(i,h){a.modifyNode(i,h);e.go()},modifyPhysics:function(h){a.modifyPhysics(h)},update:function(h){var i=a._update(h)}};return e};var physics=PhysicsWorker();onmessage=function(a){if(!a.data.type){postMessage("¿kérnèl?");return}if(a.data.type=="physics"){var b=a.data.physics;physics.init(a.data.physics);return}switch(
 a.data.type){case"modify":physics.modifyNode(a.data.id,a.data.mods);break;case"changes":physics.update(a.data.changes);physics.go();break;case"start":physics.go();break;case"stop":physics.stop();break;case"sys":var b=a.data.param||{};if(!isNaN(b.timeout)){physics.timeout(b.timeout)}physics.modifyPhysics(b);physics.go();break}};
+  })()
+
+
+  arbor = (typeof(arbor)!=='undefined') ? arbor : {}
+  $.extend(arbor, {
+    // object constructors (don't use ‘new’, just call them)
+    ParticleSystem:ParticleSystem,
+    Point:function(x, y){ return new Point(x, y) },
+
+    // immutable object with useful methods
+    etc:{      
+      trace:trace,              // ƒ(msg) -> safe console logging
+      dirname:dirname,          // ƒ(path) -> leading part of path
+      basename:basename,        // ƒ(path) -> trailing part of path
+      ordinalize:ordinalize,    // ƒ(num) -> abbrev integers (and add commas)
+      objcopy:objcopy,          // ƒ(old) -> clone an object
+      objcmp:objcmp,            // ƒ(a, b, strict_ordering) -> t/f comparison
+      objkeys:objkeys,          // ƒ(obj) -> array of all keys in obj
+      objmerge:objmerge,        // ƒ(dst, src) -> like $.extend but non-destructive
+      uniq:uniq,                // ƒ(arr) -> array of unique items in arr
+      arbor_path:arbor_path,    // ƒ() -> guess the directory of the lib code
+    }
+  })
+  
+})(this.jQuery)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/js/jquery.mustache.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/jquery.mustache.js b/storm-core/src/ui/public/js/jquery.mustache.js
new file mode 100644
index 0000000..14925bf
--- /dev/null
+++ b/storm-core/src/ui/public/js/jquery.mustache.js
@@ -0,0 +1,592 @@
+/*
+Shameless port of a shameless port
+@defunkt => @janl => @aq
+ 
+See http://github.com/defunkt/mustache for more info.
+*/
+ 
+;(function($) {
+
+/*!
+ * mustache.js - Logic-less {{mustache}} templates with JavaScript
+ * http://github.com/janl/mustache.js
+ */
+
+/*global define: false*/
+
+(function (root, factory) {
+  if (typeof exports === "object" && exports) {
+    factory(exports); // CommonJS
+  } else {
+    var mustache = {};
+    factory(mustache);
+    if (typeof define === "function" && define.amd) {
+      define(mustache); // AMD
+    } else {
+      root.Mustache = mustache; // <script>
+    }
+  }
+}(this, function (mustache) {
+
+  var whiteRe = /\s*/;
+  var spaceRe = /\s+/;
+  var nonSpaceRe = /\S/;
+  var eqRe = /\s*=/;
+  var curlyRe = /\s*\}/;
+  var tagRe = /#|\^|\/|>|\{|&|=|!/;
+
+  // Workaround for https://issues.apache.org/jira/browse/COUCHDB-577
+  // See https://github.com/janl/mustache.js/issues/189
+  var RegExp_test = RegExp.prototype.test;
+  function testRegExp(re, string) {
+    return RegExp_test.call(re, string);
+  }
+
+  function isWhitespace(string) {
+    return !testRegExp(nonSpaceRe, string);
+  }
+
+  var Object_toString = Object.prototype.toString;
+  var isArray = Array.isArray || function (object) {
+    return Object_toString.call(object) === '[object Array]';
+  };
+
+  function isFunction(object) {
+    return typeof object === 'function';
+  }
+
+  function escapeRegExp(string) {
+    return string.replace(/[\-\[\]{}()*+?.,\\\^$|#\s]/g, "\\$&");
+  }
+
+  var entityMap = {
+    "&": "&amp;",
+    "<": "&lt;",
+    ">": "&gt;",
+    '"': '&quot;',
+    "'": '&#39;',
+    "/": '&#x2F;'
+  };
+
+  function escapeHtml(string) {
+    return String(string).replace(/[&<>"'\/]/g, function (s) {
+      return entityMap[s];
+    });
+  }
+
+  function escapeTags(tags) {
+    if (!isArray(tags) || tags.length !== 2) {
+      throw new Error('Invalid tags: ' + tags);
+    }
+
+    return [
+      new RegExp(escapeRegExp(tags[0]) + "\\s*"),
+      new RegExp("\\s*" + escapeRegExp(tags[1]))
+    ];
+  }
+
+  /**
+   * Breaks up the given `template` string into a tree of tokens. If the `tags`
+   * argument is given here it must be an array with two string values: the
+   * opening and closing tags used in the template (e.g. [ "<%", "%>" ]). Of
+   * course, the default is to use mustaches (i.e. mustache.tags).
+   *
+   * A token is an array with at least 4 elements. The first element is the
+   * mustache symbol that was used inside the tag, e.g. "#" or "&". If the tag
+   * did not contain a symbol (i.e. {{myValue}}) this element is "name". For
+   * all template text that appears outside a symbol this element is "text".
+   *
+   * The second element of a token is its "value". For mustache tags this is
+   * whatever else was inside the tag besides the opening symbol. For text tokens
+   * this is the text itself.
+   *
+   * The third and fourth elements of the token are the start and end indices
+   * in the original template of the token, respectively.
+   *
+   * Tokens that are the root node of a subtree contain two more elements: an
+   * array of tokens in the subtree and the index in the original template at which
+   * the closing tag for that section begins.
+   */
+  function parseTemplate(template, tags) {
+    tags = tags || mustache.tags;
+    template = template || '';
+
+    if (typeof tags === 'string') {
+      tags = tags.split(spaceRe);
+    }
+
+    var tagRes = escapeTags(tags);
+    var scanner = new Scanner(template);
+
+    var sections = [];     // Stack to hold section tokens
+    var tokens = [];       // Buffer to hold the tokens
+    var spaces = [];       // Indices of whitespace tokens on the current line
+    var hasTag = false;    // Is there a {{tag}} on the current line?
+    var nonSpace = false;  // Is there a non-space char on the current line?
+
+    // Strips all whitespace tokens array for the current line
+    // if there was a {{#tag}} on it and otherwise only space.
+    function stripSpace() {
+      if (hasTag && !nonSpace) {
+        while (spaces.length) {
+          delete tokens[spaces.pop()];
+        }
+      } else {
+        spaces = [];
+      }
+
+      hasTag = false;
+      nonSpace = false;
+    }
+
+    var start, type, value, chr, token, openSection;
+    while (!scanner.eos()) {
+      start = scanner.pos;
+
+      // Match any text between tags.
+      value = scanner.scanUntil(tagRes[0]);
+      if (value) {
+        for (var i = 0, len = value.length; i < len; ++i) {
+          chr = value.charAt(i);
+
+          if (isWhitespace(chr)) {
+            spaces.push(tokens.length);
+          } else {
+            nonSpace = true;
+          }
+
+          tokens.push(['text', chr, start, start + 1]);
+          start += 1;
+
+          // Check for whitespace on the current line.
+          if (chr === '\n') {
+            stripSpace();
+          }
+        }
+      }
+
+      // Match the opening tag.
+      if (!scanner.scan(tagRes[0])) break;
+      hasTag = true;
+
+      // Get the tag type.
+      type = scanner.scan(tagRe) || 'name';
+      scanner.scan(whiteRe);
+
+      // Get the tag value.
+      if (type === '=') {
+        value = scanner.scanUntil(eqRe);
+        scanner.scan(eqRe);
+        scanner.scanUntil(tagRes[1]);
+      } else if (type === '{') {
+        value = scanner.scanUntil(new RegExp('\\s*' + escapeRegExp('}' + tags[1])));
+        scanner.scan(curlyRe);
+        scanner.scanUntil(tagRes[1]);
+        type = '&';
+      } else {
+        value = scanner.scanUntil(tagRes[1]);
+      }
+
+      // Match the closing tag.
+      if (!scanner.scan(tagRes[1])) {
+        throw new Error('Unclosed tag at ' + scanner.pos);
+      }
+
+      token = [ type, value, start, scanner.pos ];
+      tokens.push(token);
+
+      if (type === '#' || type === '^') {
+        sections.push(token);
+      } else if (type === '/') {
+        // Check section nesting.
+        openSection = sections.pop();
+
+        if (!openSection) {
+          throw new Error('Unopened section "' + value + '" at ' + start);
+        }
+        if (openSection[1] !== value) {
+          throw new Error('Unclosed section "' + openSection[1] + '" at ' + start);
+        }
+      } else if (type === 'name' || type === '{' || type === '&') {
+        nonSpace = true;
+      } else if (type === '=') {
+        // Set the tags for the next time around.
+        tagRes = escapeTags(tags = value.split(spaceRe));
+      }
+    }
+
+    // Make sure there are no open sections when we're done.
+    openSection = sections.pop();
+    if (openSection) {
+      throw new Error('Unclosed section "' + openSection[1] + '" at ' + scanner.pos);
+    }
+
+    return nestTokens(squashTokens(tokens));
+  }
+
+  /**
+   * Combines the values of consecutive text tokens in the given `tokens` array
+   * to a single token.
+   */
+  function squashTokens(tokens) {
+    var squashedTokens = [];
+
+    var token, lastToken;
+    for (var i = 0, len = tokens.length; i < len; ++i) {
+      token = tokens[i];
+
+      if (token) {
+        if (token[0] === 'text' && lastToken && lastToken[0] === 'text') {
+          lastToken[1] += token[1];
+          lastToken[3] = token[3];
+        } else {
+          squashedTokens.push(token);
+          lastToken = token;
+        }
+      }
+    }
+
+    return squashedTokens;
+  }
+
+  /**
+   * Forms the given array of `tokens` into a nested tree structure where
+   * tokens that represent a section have two additional items: 1) an array of
+   * all tokens that appear in that section and 2) the index in the original
+   * template that represents the end of that section.
+   */
+  function nestTokens(tokens) {
+    var nestedTokens = [];
+    var collector = nestedTokens;
+    var sections = [];
+
+    var token, section;
+    for (var i = 0, len = tokens.length; i < len; ++i) {
+      token = tokens[i];
+
+      switch (token[0]) {
+      case '#':
+      case '^':
+        collector.push(token);
+        sections.push(token);
+        collector = token[4] = [];
+        break;
+      case '/':
+        section = sections.pop();
+        section[5] = token[2];
+        collector = sections.length > 0 ? sections[sections.length - 1][4] : nestedTokens;
+        break;
+      default:
+        collector.push(token);
+      }
+    }
+
+    return nestedTokens;
+  }
+
+  /**
+   * A simple string scanner that is used by the template parser to find
+   * tokens in template strings.
+   */
+  function Scanner(string) {
+    this.string = string;
+    this.tail = string;
+    this.pos = 0;
+  }
+
+  /**
+   * Returns `true` if the tail is empty (end of string).
+   */
+  Scanner.prototype.eos = function () {
+    return this.tail === "";
+  };
+
+  /**
+   * Tries to match the given regular expression at the current position.
+   * Returns the matched text if it can match, the empty string otherwise.
+   */
+  Scanner.prototype.scan = function (re) {
+    var match = this.tail.match(re);
+
+    if (match && match.index === 0) {
+      var string = match[0];
+      this.tail = this.tail.substring(string.length);
+      this.pos += string.length;
+      return string;
+    }
+
+    return "";
+  };
+
+  /**
+   * Skips all text until the given regular expression can be matched. Returns
+   * the skipped string, which is the entire tail if no match can be made.
+   */
+  Scanner.prototype.scanUntil = function (re) {
+    var index = this.tail.search(re), match;
+
+    switch (index) {
+    case -1:
+      match = this.tail;
+      this.tail = "";
+      break;
+    case 0:
+      match = "";
+      break;
+    default:
+      match = this.tail.substring(0, index);
+      this.tail = this.tail.substring(index);
+    }
+
+    this.pos += match.length;
+
+    return match;
+  };
+
+  /**
+   * Represents a rendering context by wrapping a view object and
+   * maintaining a reference to the parent context.
+   */
+  function Context(view, parentContext) {
+    this.view = view == null ? {} : view;
+    this.cache = { '.': this.view };
+    this.parent = parentContext;
+  }
+
+  /**
+   * Creates a new context using the given view with this context
+   * as the parent.
+   */
+  Context.prototype.push = function (view) {
+    return new Context(view, this);
+  };
+
+  /**
+   * Returns the value of the given name in this context, traversing
+   * up the context hierarchy if the value is absent in this context's view.
+   */
+  Context.prototype.lookup = function (name) {
+    var value;
+    if (name in this.cache) {
+      value = this.cache[name];
+    } else {
+      var context = this;
+
+      while (context) {
+        if (name.indexOf('.') > 0) {
+          value = context.view;
+
+          var names = name.split('.'), i = 0;
+          while (value != null && i < names.length) {
+            value = value[names[i++]];
+          }
+        } else {
+          value = context.view[name];
+        }
+
+        if (value != null) break;
+
+        context = context.parent;
+      }
+
+      this.cache[name] = value;
+    }
+
+    if (isFunction(value)) {
+      value = value.call(this.view);
+    }
+
+    return value;
+  };
+
+  /**
+   * A Writer knows how to take a stream of tokens and render them to a
+   * string, given a context. It also maintains a cache of templates to
+   * avoid the need to parse the same template twice.
+   */
+  function Writer() {
+    this.cache = {};
+  }
+
+  /**
+   * Clears all cached templates in this writer.
+   */
+  Writer.prototype.clearCache = function () {
+    this.cache = {};
+  };
+
+  /**
+   * Parses and caches the given `template` and returns the array of tokens
+   * that is generated from the parse.
+   */
+  Writer.prototype.parse = function (template, tags) {
+    var cache = this.cache;
+    var tokens = cache[template];
+
+    if (tokens == null) {
+      tokens = cache[template] = parseTemplate(template, tags);
+    }
+
+    return tokens;
+  };
+
+  /**
+   * High-level method that is used to render the given `template` with
+   * the given `view`.
+   *
+   * The optional `partials` argument may be an object that contains the
+   * names and templates of partials that are used in the template. It may
+   * also be a function that is used to load partial templates on the fly
+   * that takes a single argument: the name of the partial.
+   */
+  Writer.prototype.render = function (template, view, partials) {
+    var tokens = this.parse(template);
+    var context = (view instanceof Context) ? view : new Context(view);
+    return this.renderTokens(tokens, context, partials, template);
+  };
+
+  /**
+   * Low-level method that renders the given array of `tokens` using
+   * the given `context` and `partials`.
+   *
+   * Note: The `originalTemplate` is only ever used to extract the portion
+   * of the original template that was contained in a higher-order section.
+   * If the template doesn't use higher-order sections, this argument may
+   * be omitted.
+   */
+  Writer.prototype.renderTokens = function (tokens, context, partials, originalTemplate) {
+    var buffer = '';
+
+    // This function is used to render an arbitrary template
+    // in the current context by higher-order sections.
+    var self = this;
+    function subRender(template) {
+      return self.render(template, context, partials);
+    }
+
+    var token, value;
+    for (var i = 0, len = tokens.length; i < len; ++i) {
+      token = tokens[i];
+
+      switch (token[0]) {
+      case '#':
+        value = context.lookup(token[1]);
+        if (!value) continue;
+
+        if (isArray(value)) {
+          for (var j = 0, jlen = value.length; j < jlen; ++j) {
+            buffer += this.renderTokens(token[4], context.push(value[j]), partials, originalTemplate);
+          }
+        } else if (typeof value === 'object' || typeof value === 'string') {
+          buffer += this.renderTokens(token[4], context.push(value), partials, originalTemplate);
+        } else if (isFunction(value)) {
+          if (typeof originalTemplate !== 'string') {
+            throw new Error('Cannot use higher-order sections without the original template');
+          }
+
+          // Extract the portion of the original template that the section contains.
+          value = value.call(context.view, originalTemplate.slice(token[3], token[5]), subRender);
+
+          if (value != null) buffer += value;
+        } else {
+          buffer += this.renderTokens(token[4], context, partials, originalTemplate);
+        }
+
+        break;
+      case '^':
+        value = context.lookup(token[1]);
+
+        // Use JavaScript's definition of falsy. Include empty arrays.
+        // See https://github.com/janl/mustache.js/issues/186
+        if (!value || (isArray(value) && value.length === 0)) {
+          buffer += this.renderTokens(token[4], context, partials, originalTemplate);
+        }
+
+        break;
+      case '>':
+        if (!partials) continue;
+        value = isFunction(partials) ? partials(token[1]) : partials[token[1]];
+        if (value != null) buffer += this.renderTokens(this.parse(value), context, partials, value);
+        break;
+      case '&':
+        value = context.lookup(token[1]);
+        if (value != null) buffer += value;
+        break;
+      case 'name':
+        value = context.lookup(token[1]);
+        if (value != null) buffer += mustache.escape(value);
+        break;
+      case 'text':
+        buffer += token[1];
+        break;
+      }
+    }
+
+    return buffer;
+  };
+
+  mustache.name = "mustache.js";
+  mustache.version = "0.8.1";
+  mustache.tags = [ "{{", "}}" ];
+
+  // All high-level mustache.* functions use this writer.
+  var defaultWriter = new Writer();
+
+  /**
+   * Clears all cached templates in the default writer.
+   */
+  mustache.clearCache = function () {
+    return defaultWriter.clearCache();
+  };
+
+  /**
+   * Parses and caches the given template in the default writer and returns the
+   * array of tokens it contains. Doing this ahead of time avoids the need to
+   * parse templates on the fly as they are rendered.
+   */
+  mustache.parse = function (template, tags) {
+    return defaultWriter.parse(template, tags);
+  };
+
+  /**
+   * Renders the `template` with the given `view` and `partials` using the
+   * default writer.
+   */
+  mustache.render = function (template, view, partials) {
+    return defaultWriter.render(template, view, partials);
+  };
+
+  // This is here for backwards compatibility with 0.4.x.
+  mustache.to_html = function (template, view, partials, send) {
+    var result = mustache.render(template, view, partials);
+
+    if (isFunction(send)) {
+      send(result);
+    } else {
+      return result;
+    }
+  };
+
+  // Export the escaping function so that the user may override it.
+  // See https://github.com/janl/mustache.js/issues/244
+  mustache.escape = escapeHtml;
+
+  // Export these mainly for testing, but also for advanced usage.
+  mustache.Scanner = Scanner;
+  mustache.Context = Context;
+  mustache.Writer = Writer;
+
+}));
+  $.mustache = function (template, view, partials) {
+    return Mustache.render(template, view, partials);
+  };
+
+  $.fn.mustache = function (view, partials) {
+    return $(this).map(function (i, elm) {
+      var template = $.trim($(elm).html());
+      var output = $.mustache(template, view, partials);
+      return $(output).get();
+    });
+  };
+
+})(jQuery);

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/js/purl.js
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/js/purl.js b/storm-core/src/ui/public/js/purl.js
new file mode 100644
index 0000000..b5799c6
--- /dev/null
+++ b/storm-core/src/ui/public/js/purl.js
@@ -0,0 +1,267 @@
+/*
+ * Purl (A JavaScript URL parser) v2.3.1
+ * Developed and maintanined by Mark Perkins, mark@allmarkedup.com
+ * Source repository: https://github.com/allmarkedup/jQuery-URL-Parser
+ * Licensed under an MIT-style license. See https://github.com/allmarkedup/jQuery-URL-Parser/blob/master/LICENSE for details.
+ */
+
+;(function(factory) {
+    if (typeof define === 'function' && define.amd) {
+        define(factory);
+    } else {
+        window.purl = factory();
+    }
+})(function() {
+
+    var tag2attr = {
+            a       : 'href',
+            img     : 'src',
+            form    : 'action',
+            base    : 'href',
+            script  : 'src',
+            iframe  : 'src',
+            link    : 'href',
+            embed   : 'src',
+            object  : 'data'
+        },
+
+        key = ['source', 'protocol', 'authority', 'userInfo', 'user', 'password', 'host', 'port', 'relative', 'path', 'directory', 'file', 'query', 'fragment'], // keys available to query
+
+        aliases = { 'anchor' : 'fragment' }, // aliases for backwards compatability
+
+        parser = {
+            strict : /^(?:([^:\/?#]+):)?(?:\/\/((?:(([^:@]*):?([^:@]*))?@)?([^:\/?#]*)(?::(\d*))?))?((((?:[^?#\/]*\/)*)([^?#]*))(?:\?([^#]*))?(?:#(.*))?)/,  //less intuitive, more accurate to the specs
+            loose :  /^(?:(?![^:@]+:[^:@\/]*@)([^:\/?#.]+):)?(?:\/\/)?((?:(([^:@]*):?([^:@]*))?@)?([^:\/?#]*)(?::(\d*))?)(((\/(?:[^?#](?![^?#\/]*\.[^?#\/.]+(?:[?#]|$)))*\/?)?([^?#\/]*))(?:\?([^#]*))?(?:#(.*))?)/ // more intuitive, fails on relative paths and deviates from specs
+        },
+
+        isint = /^[0-9]+$/;
+
+    function parseUri( url, strictMode ) {
+        var str = decodeURI( url ),
+        res   = parser[ strictMode || false ? 'strict' : 'loose' ].exec( str ),
+        uri = { attr : {}, param : {}, seg : {} },
+        i   = 14;
+
+        while ( i-- ) {
+            uri.attr[ key[i] ] = res[i] || '';
+        }
+
+        // build query and fragment parameters
+        uri.param['query'] = parseString(uri.attr['query']);
+        uri.param['fragment'] = parseString(uri.attr['fragment']);
+
+        // split path and fragement into segments
+        uri.seg['path'] = uri.attr.path.replace(/^\/+|\/+$/g,'').split('/');
+        uri.seg['fragment'] = uri.attr.fragment.replace(/^\/+|\/+$/g,'').split('/');
+
+        // compile a 'base' domain attribute
+        uri.attr['base'] = uri.attr.host ? (uri.attr.protocol ?  uri.attr.protocol+'://'+uri.attr.host : uri.attr.host) + (uri.attr.port ? ':'+uri.attr.port : '') : '';
+
+        return uri;
+    }
+
+    function getAttrName( elm ) {
+        var tn = elm.tagName;
+        if ( typeof tn !== 'undefined' ) return tag2attr[tn.toLowerCase()];
+        return tn;
+    }
+
+    function promote(parent, key) {
+        if (parent[key].length === 0) return parent[key] = {};
+        var t = {};
+        for (var i in parent[key]) t[i] = parent[key][i];
+        parent[key] = t;
+        return t;
+    }
+
+    function parse(parts, parent, key, val) {
+        var part = parts.shift();
+        if (!part) {
+            if (isArray(parent[key])) {
+                parent[key].push(val);
+            } else if ('object' == typeof parent[key]) {
+                parent[key] = val;
+            } else if ('undefined' == typeof parent[key]) {
+                parent[key] = val;
+            } else {
+                parent[key] = [parent[key], val];
+            }
+        } else {
+            var obj = parent[key] = parent[key] || [];
+            if (']' == part) {
+                if (isArray(obj)) {
+                    if ('' !== val) obj.push(val);
+                } else if ('object' == typeof obj) {
+                    obj[keys(obj).length] = val;
+                } else {
+                    obj = parent[key] = [parent[key], val];
+                }
+            } else if (~part.indexOf(']')) {
+                part = part.substr(0, part.length - 1);
+                if (!isint.test(part) && isArray(obj)) obj = promote(parent, key);
+                parse(parts, obj, part, val);
+                // key
+            } else {
+                if (!isint.test(part) && isArray(obj)) obj = promote(parent, key);
+                parse(parts, obj, part, val);
+            }
+        }
+    }
+
+    function merge(parent, key, val) {
+        if (~key.indexOf(']')) {
+            var parts = key.split('[');
+            parse(parts, parent, 'base', val);
+        } else {
+            if (!isint.test(key) && isArray(parent.base)) {
+                var t = {};
+                for (var k in parent.base) t[k] = parent.base[k];
+                parent.base = t;
+            }
+            if (key !== '') {
+                set(parent.base, key, val);
+            }
+        }
+        return parent;
+    }
+
+    function parseString(str) {
+        return reduce(String(str).split(/&|;/), function(ret, pair) {
+            try {
+                pair = decodeURIComponent(pair.replace(/\+/g, ' '));
+            } catch(e) {
+                // ignore
+            }
+            var eql = pair.indexOf('='),
+                brace = lastBraceInKey(pair),
+                key = pair.substr(0, brace || eql),
+                val = pair.substr(brace || eql, pair.length);
+
+            val = val.substr(val.indexOf('=') + 1, val.length);
+
+            if (key === '') {
+                key = pair;
+                val = '';
+            }
+
+            return merge(ret, key, val);
+        }, { base: {} }).base;
+    }
+
+    function set(obj, key, val) {
+        var v = obj[key];
+        if (typeof v === 'undefined') {
+            obj[key] = val;
+        } else if (isArray(v)) {
+            v.push(val);
+        } else {
+            obj[key] = [v, val];
+        }
+    }
+
+    function lastBraceInKey(str) {
+        var len = str.length,
+            brace,
+            c;
+        for (var i = 0; i < len; ++i) {
+            c = str[i];
+            if (']' == c) brace = false;
+            if ('[' == c) brace = true;
+            if ('=' == c && !brace) return i;
+        }
+    }
+
+    function reduce(obj, accumulator){
+        var i = 0,
+            l = obj.length >> 0,
+            curr = arguments[2];
+        while (i < l) {
+            if (i in obj) curr = accumulator.call(undefined, curr, obj[i], i, obj);
+            ++i;
+        }
+        return curr;
+    }
+
+    function isArray(vArg) {
+        return Object.prototype.toString.call(vArg) === "[object Array]";
+    }
+
+    function keys(obj) {
+        var key_array = [];
+        for ( var prop in obj ) {
+            if ( obj.hasOwnProperty(prop) ) key_array.push(prop);
+        }
+        return key_array;
+    }
+
+    function purl( url, strictMode ) {
+        if ( arguments.length === 1 && url === true ) {
+            strictMode = true;
+            url = undefined;
+        }
+        strictMode = strictMode || false;
+        url = url || window.location.toString();
+
+        return {
+
+            data : parseUri(url, strictMode),
+
+            // get various attributes from the URI
+            attr : function( attr ) {
+                attr = aliases[attr] || attr;
+                return typeof attr !== 'undefined' ? this.data.attr[attr] : this.data.attr;
+            },
+
+            // return query string parameters
+            param : function( param ) {
+                return typeof param !== 'undefined' ? this.data.param.query[param] : this.data.param.query;
+            },
+
+            // return fragment parameters
+            fparam : function( param ) {
+                return typeof param !== 'undefined' ? this.data.param.fragment[param] : this.data.param.fragment;
+            },
+
+            // return path segments
+            segment : function( seg ) {
+                if ( typeof seg === 'undefined' ) {
+                    return this.data.seg.path;
+                } else {
+                    seg = seg < 0 ? this.data.seg.path.length + seg : seg - 1; // negative segments count from the end
+                    return this.data.seg.path[seg];
+                }
+            },
+
+            // return fragment segments
+            fsegment : function( seg ) {
+                if ( typeof seg === 'undefined' ) {
+                    return this.data.seg.fragment;
+                } else {
+                    seg = seg < 0 ? this.data.seg.fragment.length + seg : seg - 1; // negative segments count from the end
+                    return this.data.seg.fragment[seg];
+                }
+            }
+
+        };
+
+    }
+    
+    purl.jQuery = function($){
+        if ($ != null) {
+            $.fn.url = function( strictMode ) {
+                var url = '';
+                if ( this.length ) {
+                    url = $(this).attr( getAttrName(this[0]) ) || '';
+                }
+                return purl( url, strictMode );
+            };
+
+            $.url = purl;
+        }
+    };
+
+    purl.jQuery(window.jQuery);
+
+    return purl;
+
+});


[03/13] move towards idiomatic Clojure style

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index 7bbe238..32f7f88 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -13,6 +13,7 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.testing
   (:require [backtype.storm.daemon
              [nimbus :as nimbus]
@@ -43,45 +44,48 @@
   (:require [backtype.storm.daemon.acker :as acker])
   (:use [backtype.storm cluster util thrift config log]))
 
-(defn feeder-spout [fields]
+(defn feeder-spout
+  [fields]
   (FeederSpout. (Fields. fields)))
 
-(defn local-temp-path []
+(defn local-temp-path
+  []
   (str (System/getProperty "java.io.tmpdir") (if-not on-windows? "/") (uuid)))
 
-(defn delete-all [paths]
+(defn delete-all
+  [paths]
   (dorun
     (for [t paths]
       (if (.exists (File. t))
         (try
           (FileUtils/forceDelete (File. t))
           (catch Exception e
-            (log-message (.getMessage e))))
-        ))))
+            (log-message (.getMessage e))))))))
 
-(defmacro with-local-tmp [[& tmp-syms] & body]
+(defmacro with-local-tmp
+  [[& tmp-syms] & body]
   (let [tmp-paths (mapcat (fn [t] [t `(local-temp-path)]) tmp-syms)]
     `(let [~@tmp-paths]
-      (try
-        ~@body
-      (finally
-       (delete-all ~(vec tmp-syms)))
-      ))
-    ))
-
-(defn start-simulating-time! []
+       (try
+         ~@body
+         (finally
+           (delete-all ~(vec tmp-syms)))))))
+
+(defn start-simulating-time!
+  []
   (Time/startSimulating))
 
-(defn stop-simulating-time! []
+(defn stop-simulating-time!
+  []
   (Time/stopSimulating))
 
-(defmacro with-simulated-time [& body]
+(defmacro with-simulated-time
+  [& body]
   `(do
      (start-simulating-time!)
      (let [ret# (do ~@body)]
        (stop-simulating-time!)
-       ret#
-       )))
+       ret#)))
 
 (defn advance-time-ms! [ms]
   (Time/advanceTime ms))
@@ -89,26 +93,25 @@
 (defn advance-time-secs! [secs]
   (advance-time-ms! (* (long secs) 1000)))
 
-
-(defnk add-supervisor [cluster-map :ports 2 :conf {} :id nil]
+(defnk add-supervisor
+  [cluster-map :ports 2 :conf {} :id nil]
   (let [tmp-dir (local-temp-path)
-        port-ids (if (sequential? ports) ports (doall (repeatedly ports (:port-counter cluster-map))))
+        port-ids (if (sequential? ports)
+                   ports
+                   (doall (repeatedly ports (:port-counter cluster-map))))
         supervisor-conf (merge (:daemon-conf cluster-map)
                                conf
                                {STORM-LOCAL-DIR tmp-dir
-                                SUPERVISOR-SLOTS-PORTS port-ids
-                               })
+                                SUPERVISOR-SLOTS-PORTS port-ids})
         id-fn (if id (fn [] id) supervisor/generate-supervisor-id)
         daemon (with-var-roots [supervisor/generate-supervisor-id id-fn] (supervisor/mk-supervisor supervisor-conf (:shared-context cluster-map) (supervisor/standalone-supervisor)))]
     (swap! (:supervisors cluster-map) conj daemon)
     (swap! (:tmp-dirs cluster-map) conj tmp-dir)
-    daemon
-    ))
+    daemon))
 
 (defn mk-shared-context [conf]
   (if-not (conf STORM-LOCAL-MODE-ZMQ)
-    (msg-loader/mk-local-context)
-    ))
+    (msg-loader/mk-local-context)))
 
 ;; returns map containing cluster info
 ;; local dir is always overridden in maps
@@ -121,8 +124,7 @@
                            {TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS true
                             ZMQ-LINGER-MILLIS 0
                             TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS false
-                            TOPOLOGY-TRIDENT-BATCH-EMIT-INTERVAL-MILLIS 50
-                            }
+                            TOPOLOGY-TRIDENT-BATCH-EMIT-INTERVAL-MILLIS 50}
                            daemon-conf
                            {STORM-CLUSTER-MODE "local"
                             STORM-ZOOKEEPER-PORT zk-port
@@ -130,8 +132,8 @@
         nimbus-tmp (local-temp-path)
         port-counter (mk-counter supervisor-slot-port-min)
         nimbus (nimbus/service-handler
-                (assoc daemon-conf STORM-LOCAL-DIR nimbus-tmp)
-                (if inimbus inimbus (nimbus/standalone-nimbus)))
+                 (assoc daemon-conf STORM-LOCAL-DIR nimbus-tmp)
+                 (if inimbus inimbus (nimbus/standalone-nimbus)))
         context (mk-shared-context daemon-conf)
         cluster-map {:nimbus nimbus
                      :port-counter port-counter
@@ -147,13 +149,11 @@
                            (repeat supervisors {}))]
     (doseq [sc supervisor-confs]
       (add-supervisor cluster-map :ports ports-per-supervisor :conf sc))
-    cluster-map
-    ))
+    cluster-map))
 
 (defn get-supervisor [cluster-map supervisor-id]
   (let [finder-fn #(= (.get-id %) supervisor-id)]
-    (find-first finder-fn @(:supervisors cluster-map))
-    ))
+    (find-first finder-fn @(:supervisors cluster-map))))
 
 (defn kill-supervisor [cluster-map supervisor-id]
   (let [finder-fn #(= (.get-id %) supervisor-id)
@@ -162,8 +162,7 @@
                         supervisors)]
     ;; tmp-dir will be taken care of by shutdown
     (reset! (:supervisors cluster-map) (remove-first finder-fn supervisors))
-    (.shutdown sup)
-    ))
+    (.shutdown sup)))
 
 (defn kill-local-storm-cluster [cluster-map]
   (.shutdown (:nimbus cluster-map))
@@ -181,17 +180,17 @@
     (log-message "Deleting temporary path " t)
     (try
       (rmr t)
-      (catch Exception e (log-message (.getMessage e)))) ;; on windows, the host process still holds lock on the logfile
-    ))
+      ;; on windows, the host process still holds lock on the logfile
+      (catch Exception e (log-message (.getMessage e)))) ))
 
 (def TEST-TIMEOUT-MS 5000)
 
 (defmacro while-timeout [timeout-ms condition & body]
   `(let [end-time# (+ (System/currentTimeMillis) ~timeout-ms)]
-    (while ~condition
-      (when (> (System/currentTimeMillis) end-time#)
-        (throw (AssertionError. (str "Test timed out (" ~timeout-ms "ms)"))))
-      ~@body)))
+     (while ~condition
+       (when (> (System/currentTimeMillis) end-time#)
+         (throw (AssertionError. (str "Test timed out (" ~timeout-ms "ms)"))))
+       ~@body)))
 
 (defn wait-until-cluster-waiting
   "Wait until the cluster is idle. Should be used with time simulation."
@@ -202,60 +201,59 @@
         daemons (concat
                   [(:nimbus cluster-map)]
                   supervisors
-                  workers) ; because a worker may already be dead
-        ]
+                  ; because a worker may already be dead
+                  workers)]
     (while-timeout TEST-TIMEOUT-MS (not (every? (memfn waiting?) daemons))
-      (Thread/sleep 10)
-;;      (doseq [d daemons]
-;;        (if-not ((memfn waiting?) d)
-;;          (println d)))
-      )))
+                   (Thread/sleep 10)
+                   ;;      (doseq [d daemons]
+                   ;;        (if-not ((memfn waiting?) d)
+                   ;;          (println d)))
+                   )))
 
 (defn advance-cluster-time
   ([cluster-map secs increment-secs]
-    (loop [left secs]
-      (when (> left 0)
-        (let [diff (min left increment-secs)]
-          (advance-time-secs! diff)
-          (wait-until-cluster-waiting cluster-map)
-          (recur (- left diff))
-          ))))
+   (loop [left secs]
+     (when (> left 0)
+       (let [diff (min left increment-secs)]
+         (advance-time-secs! diff)
+         (wait-until-cluster-waiting cluster-map)
+         (recur (- left diff))))))
   ([cluster-map secs]
-    (advance-cluster-time cluster-map secs 1)
-    ))
+   (advance-cluster-time cluster-map secs 1)))
 
-(defmacro with-local-cluster [[cluster-sym & args] & body]
+(defmacro with-local-cluster
+  [[cluster-sym & args] & body]
   `(let [~cluster-sym (mk-local-storm-cluster ~@args)]
      (try
        ~@body
-     (catch Throwable t#
-       (log-error t# "Error in cluster")
-       (throw t#)
-       )
-     (finally
-       (kill-local-storm-cluster ~cluster-sym)))
-       ))
-
-(defmacro with-simulated-time-local-cluster [& args]
+       (catch Throwable t#
+         (log-error t# "Error in cluster")
+         (throw t#))
+       (finally
+         (kill-local-storm-cluster ~cluster-sym)))))
+
+(defmacro with-simulated-time-local-cluster
+  [& args]
   `(with-simulated-time
-    (with-local-cluster ~@args)))
+     (with-local-cluster ~@args)))
 
-(defmacro with-inprocess-zookeeper [port-sym & body]
+(defmacro with-inprocess-zookeeper
+  [port-sym & body]
   `(with-local-tmp [tmp#]
-     (let [[~port-sym zks#] (zk/mk-inprocess-zookeeper tmp#)]
-       (try
-         ~@body
-       (finally
-         (zk/shutdown-inprocess-zookeeper zks#)
-         ))
-       )))
-
-(defn submit-local-topology [nimbus storm-name conf topology]
+                   (let [[~port-sym zks#] (zk/mk-inprocess-zookeeper tmp#)]
+                     (try
+                       ~@body
+                       (finally
+                         (zk/shutdown-inprocess-zookeeper zks#))))))
+
+(defn submit-local-topology
+  [nimbus storm-name conf topology]
   (when-not (Utils/isValidConf conf)
     (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
   (.submitTopology nimbus storm-name nil (to-json conf) topology))
 
-(defn submit-local-topology-with-opts [nimbus storm-name conf topology submit-opts]
+(defn submit-local-topology-with-opts
+  [nimbus storm-name conf topology submit-opts]
   (when-not (Utils/isValidConf conf)
     (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
   (.submitTopologyWithOpts nimbus storm-name nil (to-json conf) topology submit-opts))
@@ -269,141 +267,148 @@
           new-assignments (assoc existing-assignments topology-id executor->node+port)]
       new-assignments)))
 
-(defn submit-mocked-assignment [nimbus storm-name conf topology task->component executor->node+port]
+(defn submit-mocked-assignment
+  [nimbus storm-name conf topology task->component executor->node+port]
   (with-var-roots [common/storm-task-info (fn [& ignored] task->component)
                    nimbus/compute-new-topology->executor->node+port (mocked-compute-new-topology->executor->node+port
-                                                                     storm-name
-                                                                     executor->node+port)]
-    (submit-local-topology nimbus storm-name conf topology)
-    ))
+                                                                      storm-name
+                                                                      executor->node+port)]
+                  (submit-local-topology nimbus storm-name conf topology)))
 
 (defn mk-capture-launch-fn [capture-atom]
   (fn [supervisor storm-id port worker-id]
     (let [supervisor-id (:supervisor-id supervisor)
           existing (get @capture-atom [supervisor-id port] [])]
-      (swap! capture-atom assoc [supervisor-id port] (conj existing storm-id))
-      )))
+      (swap! capture-atom assoc [supervisor-id port] (conj existing storm-id)))))
 
-(defn find-worker-id [supervisor-conf port]
+(defn find-worker-id
+  [supervisor-conf port]
   (let [supervisor-state (supervisor-state supervisor-conf)
         worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)]
-    (first ((reverse-map worker->port) port))
-    ))
+    (first ((reverse-map worker->port) port))))
 
-(defn find-worker-port [supervisor-conf worker-id]
+(defn find-worker-port
+  [supervisor-conf worker-id]
   (let [supervisor-state (supervisor-state supervisor-conf)
-        worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)
-        ]
-    (worker->port worker-id)
-    ))
+        worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)]
+    (worker->port worker-id)))
 
-(defn mk-capture-shutdown-fn [capture-atom]
+(defn mk-capture-shutdown-fn
+  [capture-atom]
   (let [existing-fn supervisor/shutdown-worker]
     (fn [supervisor worker-id]
       (let [conf (:conf supervisor)
             supervisor-id (:supervisor-id supervisor)
             port (find-worker-port conf worker-id)
-            existing (get @capture-atom [supervisor-id port] 0)]      
+            existing (get @capture-atom [supervisor-id port] 0)]
         (swap! capture-atom assoc [supervisor-id port] (inc existing))
-        (existing-fn supervisor worker-id)
-        ))))
+        (existing-fn supervisor worker-id)))))
 
-(defmacro capture-changed-workers [& body]
+(defmacro capture-changed-workers
+  [& body]
   `(let [launch-captured# (atom {})
          shutdown-captured# (atom {})]
-    (with-var-roots [supervisor/launch-worker (mk-capture-launch-fn launch-captured#)
-                     supervisor/shutdown-worker (mk-capture-shutdown-fn shutdown-captured#)]
-      ~@body
-      {:launched @launch-captured#
-       :shutdown @shutdown-captured#}
-      )))
-
-(defmacro capture-launched-workers [& body]
+     (with-var-roots [supervisor/launch-worker (mk-capture-launch-fn launch-captured#)
+                      supervisor/shutdown-worker (mk-capture-shutdown-fn shutdown-captured#)]
+                     ~@body
+                     {:launched @launch-captured#
+                      :shutdown @shutdown-captured#})))
+
+(defmacro capture-launched-workers
+  [& body]
   `(:launched (capture-changed-workers ~@body)))
 
-(defmacro capture-shutdown-workers [& body]
+(defmacro capture-shutdown-workers
+  [& body]
   `(:shutdown (capture-changed-workers ~@body)))
 
-(defnk aggregated-stat [cluster-map storm-name stat-key :component-ids nil]
+(defnk aggregated-stat
+  [cluster-map storm-name stat-key :component-ids nil]
   (let [state (:storm-cluster-state cluster-map)
         nimbus (:nimbus cluster-map)
         storm-id (common/get-storm-id state storm-name)
-        
         component->tasks (reverse-map
-                          (common/storm-task-info
-                           (.getUserTopology nimbus storm-id)
-                           (from-json (.getTopologyConf nimbus storm-id))))
+                           (common/storm-task-info
+                             (.getUserTopology nimbus storm-id)
+                             (from-json (.getTopologyConf nimbus storm-id))))
         component->tasks (if component-ids
                            (select-keys component->tasks component-ids)
                            component->tasks)
         task-ids (apply concat (vals component->tasks))
-        assignment (.assignment-info state storm-id nil) 
+        assignment (.assignment-info state storm-id nil)
         taskbeats (.taskbeats state storm-id (:task->node+port assignment))
         heartbeats (dofor [id task-ids] (get taskbeats id))
         stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
-    (reduce + stats)
-    ))
+    (reduce + stats)))
 
-(defn emitted-spout-tuples [cluster-map topology storm-name]
-  (aggregated-stat cluster-map
-                   storm-name
-                   :emitted
-                   :component-ids (keys (.get_spouts topology))))
+(defn emitted-spout-tuples
+  [cluster-map topology storm-name]
+  (aggregated-stat
+    cluster-map
+    storm-name
+    :emitted
+    :component-ids (keys (.get_spouts topology))))
 
-(defn transferred-tuples [cluster-map storm-name]
+(defn transferred-tuples
+  [cluster-map storm-name]
   (aggregated-stat cluster-map storm-name :transferred))
 
-(defn acked-tuples [cluster-map storm-name]
+(defn acked-tuples
+  [cluster-map storm-name]
   (aggregated-stat cluster-map storm-name :acked))
 
-(defn simulate-wait [cluster-map]
+(defn simulate-wait
+  [cluster-map]
   (if (Time/isSimulating)
     (advance-cluster-time cluster-map 10)
-    (Thread/sleep 100)
-    ))
+    (Thread/sleep 100)))
 
 (defprotocol CompletableSpout
-  (exhausted? [this] "Whether all the tuples for this spout have been completed.")
-  (cleanup [this] "Cleanup any global state kept")
-  (startup [this] "Prepare the spout (globally) before starting the topology"))
+  (exhausted?
+    [this]
+    "Whether all the tuples for this spout have been completed.")
+  (cleanup
+    [this]
+    "Cleanup any global state kept")
+  (startup
+    [this]
+    "Prepare the spout (globally) before starting the topology"))
 
 (extend-type FixedTupleSpout
   CompletableSpout
   (exhausted? [this]
-    (= (-> this .getSourceTuples count)
-       (.getCompleted this)))
+              (= (-> this .getSourceTuples count)
+                 (.getCompleted this)))
   (cleanup [this]
-    (.cleanup this))
-  (startup [this]
-    ))
+           (.cleanup this))
+  (startup [this]))
 
 (extend-type TransactionalSpoutCoordinator
   CompletableSpout
   (exhausted? [this]
-    (exhausted? (.getSpout this)))
+              (exhausted? (.getSpout this)))
   (cleanup [this]
-    (cleanup (.getSpout this)))
+           (cleanup (.getSpout this)))
   (startup [this]
-    (startup (.getSpout this))))
+           (startup (.getSpout this))))
 
 (extend-type PartitionedTransactionalSpoutExecutor
   CompletableSpout
   (exhausted? [this]
-    (exhausted? (.getPartitionedSpout this)))
+              (exhausted? (.getPartitionedSpout this)))
   (cleanup [this]
-    (cleanup (.getPartitionedSpout this)))
+           (cleanup (.getPartitionedSpout this)))
   (startup [this]
-    (startup (.getPartitionedSpout this))
-    ))
+           (startup (.getPartitionedSpout this))))
 
 (extend-type MemoryTransactionalSpout
   CompletableSpout
   (exhausted? [this]
-    (.isExhaustedTuples this))
+              (.isExhaustedTuples this))
   (cleanup [this]
-    (.cleanup this))
+           (.cleanup this))
   (startup [this]
-    (.startup this)))
+           (.startup this)))
 
 (defn spout-objects [spec-map]
   (for [[_ spout-spec] spec-map]
@@ -411,7 +416,8 @@
         .get_spout_object
         deserialized-component-object)))
 
-(defn capture-topology [topology]
+(defn capture-topology
+  [topology]
   (let [topology (.deepCopy topology)
         spouts (.get_spouts topology)
         bolts (.get_bolts topology)
@@ -424,21 +430,24 @@
     (.set_bolts topology
                 (assoc (clojurify-structure bolts)
                   (uuid)
-                  (Bolt.                   
-                   (serialize-component-object capturer)
-                   (mk-plain-component-common (into {} (for [[id direct?] all-streams]
-                                                         [id (if direct?
-                                                               (mk-direct-grouping)
-                                                               (mk-global-grouping))]))
-                                              {}
-                                              nil))
-                  ))
+                  (Bolt.
+                    (serialize-component-object capturer)
+                    (mk-plain-component-common (into {} (for [[id direct?] all-streams]
+                                                          [id (if direct?
+                                                                (mk-direct-grouping)
+                                                                (mk-global-grouping))]))
+                                               {}
+                                               nil))))
     {:topology topology
-     :capturer capturer}
-    ))
+     :capturer capturer}))
 
 ;; TODO: mock-sources needs to be able to mock out state spouts as well
-(defnk complete-topology [cluster-map topology :mock-sources {} :storm-conf {} :cleanup-state true :topology-name nil]
+(defnk complete-topology
+  [cluster-map topology
+   :mock-sources {}
+   :storm-conf {}
+   :cleanup-state true
+   :topology-name nil]
   ;; TODO: the idea of mocking for transactional topologies should be done an
   ;; abstraction level above... should have a complete-transactional-topology for this
   (let [{topology :topology capturer :capturer} (capture-topology topology)
@@ -447,153 +456,144 @@
         spouts (.get_spouts topology)
         replacements (map-val (fn [v]
                                 (FixedTupleSpout.
-                                 (for [tup v]
-                                   (if (map? tup)
-                                     (FixedTuple. (:stream tup) (:values tup))
-                                     tup))))
-                              mock-sources)
-        
-
-        ]
+                                  (for [tup v]
+                                    (if (map? tup)
+                                      (FixedTuple. (:stream tup) (:values tup))
+                                      tup))))
+                              mock-sources)]
     (doseq [[id spout] replacements]
       (let [spout-spec (get spouts id)]
-        (.set_spout_object spout-spec (serialize-component-object spout))
-        ))
+        (.set_spout_object spout-spec (serialize-component-object spout))))
     (doseq [spout (spout-objects spouts)]
       (when-not (extends? CompletableSpout (.getClass spout))
-        (throw (RuntimeException. "Cannot complete topology unless every spout is a CompletableSpout (or mocked to be)"))
-        ))
+        (throw (RuntimeException. "Cannot complete topology unless every spout is a CompletableSpout (or mocked to be)"))))
 
     (doseq [spout (spout-objects spouts)]
       (startup spout))
-    
+
     (submit-local-topology (:nimbus cluster-map) storm-name storm-conf topology)
-    
-    
+
     (let [storm-id (common/get-storm-id state storm-name)]
       (while-timeout TEST-TIMEOUT-MS (not (every? exhausted? (spout-objects spouts)))
-        (simulate-wait cluster-map))
+                     (simulate-wait cluster-map))
 
       (.killTopologyWithOpts (:nimbus cluster-map) storm-name (doto (KillOptions.) (.set_wait_secs 0)))
       (while-timeout TEST-TIMEOUT-MS (.assignment-info state storm-id nil)
-        (simulate-wait cluster-map))
+                     (simulate-wait cluster-map))
       (when cleanup-state
         (doseq [spout (spout-objects spouts)]
           (cleanup spout))))
 
     (if cleanup-state
       (.getAndRemoveResults capturer)
-      (.getAndClearResults capturer))
-    ))
+      (.getAndClearResults capturer))))
 
 (defn read-tuples
   ([results component-id stream-id]
-     (let [fixed-tuples (get results component-id [])]
-       (mapcat
-        (fn [ft]
-          (if (= stream-id (. ft stream))
-            [(vec (. ft values))]))
-        fixed-tuples)
-       ))
-  ([results component-id]
-     (read-tuples results component-id Utils/DEFAULT_STREAM_ID)
+   (let [fixed-tuples (get results component-id [])]
+     (mapcat
+       (fn [ft]
+         (if (= stream-id (. ft stream))
+           [(vec (. ft values))]))
+       fixed-tuples)
      ))
+  ([results component-id]
+   (read-tuples results component-id Utils/DEFAULT_STREAM_ID)))
 
-(defn ms= [& args]  
+(defn ms=
+  [& args]
   (apply = (map multi-set args)))
 
 (def TRACKER-BOLT-ID "+++tracker-bolt")
 
-;; TODO:  should override system-topology! and wrap everything there
+;; TODO: should override system-topology! and wrap everything there
 (defn mk-tracked-topology
   ([tracked-cluster topology]
-     (let [track-id (::track-id tracked-cluster)
-           ret (.deepCopy topology)]
-       (dofor [[_ bolt] (.get_bolts ret)
-               :let [obj (deserialized-component-object (.get_bolt_object bolt))]]
-              (.set_bolt_object bolt (serialize-component-object
-                                      (BoltTracker. obj track-id))))
-       (dofor [[_ spout] (.get_spouts ret)
-               :let [obj (deserialized-component-object (.get_spout_object spout))]]
-              (.set_spout_object spout (serialize-component-object
-                                        (SpoutTracker. obj track-id))))
-       {:topology ret
-        :last-spout-emit (atom 0)
-        :cluster tracked-cluster
-        }
-       )))
-
-(defn assoc-track-id [cluster track-id]
+   (let [track-id (::track-id tracked-cluster)
+         ret (.deepCopy topology)]
+     (dofor [[_ bolt] (.get_bolts ret)
+             :let [obj (deserialized-component-object (.get_bolt_object bolt))]]
+            (.set_bolt_object bolt (serialize-component-object
+                                     (BoltTracker. obj track-id))))
+     (dofor [[_ spout] (.get_spouts ret)
+             :let [obj (deserialized-component-object (.get_spout_object spout))]]
+            (.set_spout_object spout (serialize-component-object
+                                       (SpoutTracker. obj track-id))))
+     {:topology ret
+      :last-spout-emit (atom 0)
+      :cluster tracked-cluster})))
+
+(defn assoc-track-id
+  [cluster track-id]
   (assoc cluster ::track-id track-id))
 
-(defn increment-global! [id key amt]
+(defn increment-global!
+  [id key amt]
   (-> (RegisteredGlobalState/getState id)
       (get key)
       (.addAndGet amt)))
 
-(defn global-amt [id key]
+(defn global-amt
+  [id key]
   (-> (RegisteredGlobalState/getState id)
       (get key)
-      .get
-      ))
+      .get))
 
-(defmacro with-tracked-cluster [[cluster-sym & cluster-args] & body]
+(defmacro with-tracked-cluster
+  [[cluster-sym & cluster-args] & body]
   `(let [id# (uuid)]
-     (RegisteredGlobalState/setState id#
-                                     (doto (ConcurrentHashMap.)
-                                       (.put "spout-emitted" (AtomicInteger. 0))
-                                       (.put "transferred" (AtomicInteger. 0))
-                                       (.put "processed" (AtomicInteger. 0))))
-     (with-var-roots [acker/mk-acker-bolt (let [old# acker/mk-acker-bolt]
-                                            (fn [& args#]
-                                              (NonRichBoltTracker. (apply old# args#) id#)
-                                              ))
-                      ;; critical that this particular function is overridden here,
-                      ;; since the transferred stat needs to be incremented at the moment
-                      ;; of tuple emission (and not on a separate thread later) for
-                      ;; topologies to be tracked correctly. This is because "transferred" *must*
-                      ;; be incremented before "processing".
-                      executor/mk-executor-transfer-fn
-                          (let [old# executor/mk-executor-transfer-fn]
-                            (fn [& args#]
-                              (let [transferrer# (apply old# args#)]
-                                (fn [& args2#]
-                                  ;; (log-message "Transferring: " transfer-args#)
-                                  (increment-global! id# "transferred" 1)
-                                  (apply transferrer# args2#)
-                                  ))))
-                      ]
+     (RegisteredGlobalState/setState
+       id#
+       (doto (ConcurrentHashMap.)
+         (.put "spout-emitted" (AtomicInteger. 0))
+         (.put "transferred" (AtomicInteger. 0))
+         (.put "processed" (AtomicInteger. 0))))
+     (with-var-roots
+       [acker/mk-acker-bolt
+        (let [old# acker/mk-acker-bolt]
+          (fn [& args#] (NonRichBoltTracker. (apply old# args#) id#)))
+        ;; critical that this particular function is overridden here,
+        ;; since the transferred stat needs to be incremented at the moment
+        ;; of tuple emission (and not on a separate thread later) for
+        ;; topologies to be tracked correctly. This is because "transferred" *must*
+        ;; be incremented before "processing".
+        executor/mk-executor-transfer-fn
+        (let [old# executor/mk-executor-transfer-fn]
+          (fn [& args#]
+            (let [transferrer# (apply old# args#)]
+              (fn [& args2#]
+                ;; (log-message "Transferring: " transfer-args#)
+                (increment-global! id# "transferred" 1)
+                (apply transferrer# args2#)))))]
        (with-local-cluster [~cluster-sym ~@cluster-args]
-         (let [~cluster-sym (assoc-track-id ~cluster-sym id#)]
-           ~@body)
-         ))
-     (RegisteredGlobalState/clearState id#)
-     ))
+                           (let [~cluster-sym (assoc-track-id ~cluster-sym id#)]
+                             ~@body)))
+     (RegisteredGlobalState/clearState id#)))
 
 (defn tracked-wait
   "Waits until topology is idle and 'amt' more tuples have been emitted by spouts."
   ([tracked-topology]
-     (tracked-wait tracked-topology 1))
+   (tracked-wait tracked-topology 1))
   ([tracked-topology amt]
-      (let [target (+ amt @(:last-spout-emit tracked-topology))
-            track-id (-> tracked-topology :cluster ::track-id)
-            waiting? (fn []
-                       (or (not= target (global-amt track-id "spout-emitted"))
-                           (not= (global-amt track-id "transferred")                                 
-                                 (global-amt track-id "processed"))
-                           ))]
-        (while-timeout TEST-TIMEOUT-MS (waiting?)
-          ;; (println "Spout emitted: " (global-amt track-id "spout-emitted"))
-          ;; (println "Processed: " (global-amt track-id "processed"))
-          ;; (println "Transferred: " (global-amt track-id "transferred"))
-          (Thread/sleep 500))
-        (reset! (:last-spout-emit tracked-topology) target)
-        )))
-
-(defnk test-tuple [values
-                   :stream Utils/DEFAULT_STREAM_ID
-                   :component "component"
-                   :fields nil]
+   (let [target (+ amt @(:last-spout-emit tracked-topology))
+         track-id (-> tracked-topology :cluster ::track-id)
+         waiting? (fn []
+                    (or (not= target (global-amt track-id "spout-emitted"))
+                        (not= (global-amt track-id "transferred")
+                              (global-amt track-id "processed"))
+                        ))]
+     (while-timeout TEST-TIMEOUT-MS (waiting?)
+                    ;; (println "Spout emitted: " (global-amt track-id "spout-emitted"))
+                    ;; (println "Processed: " (global-amt track-id "processed"))
+                    ;; (println "Transferred: " (global-amt track-id "transferred"))
+                    (Thread/sleep 500))
+     (reset! (:last-spout-emit tracked-topology) target))))
+
+(defnk test-tuple
+  [values
+   :stream Utils/DEFAULT_STREAM_ID
+   :component "component"
+   :fields nil]
   (let [fields (or fields
                    (->> (iterate inc 1)
                         (take (count values))
@@ -618,5 +618,4 @@
                   (HashMap.)
                   (HashMap.)
                   (atom false))]
-    (TupleImpl. context values 1 stream)
-    ))
+    (TupleImpl. context values 1 stream)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/testing4j.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/testing4j.clj b/storm-core/src/clj/backtype/storm/testing4j.clj
index 5d44604..ff939a9 100644
--- a/storm-core/src/clj/backtype/storm/testing4j.clj
+++ b/storm-core/src/clj/backtype/storm/testing4j.clj
@@ -51,7 +51,6 @@
              ^:static [testTuple [java.util.List] backtype.storm.tuple.Tuple]
              ^:static [testTuple [java.util.List backtype.storm.testing.MkTupleParam] backtype.storm.tuple.Tuple]]))
 
-
 (defn -completeTopology
   ([^ILocalCluster cluster ^StormTopology topology ^CompleteTopologyParam completeTopologyParam]
      (let [mocked-sources (or (-> completeTopologyParam .getMockedSources .getData) {})
@@ -66,11 +65,13 @@
   ([^ILocalCluster cluster ^StormTopology topology]
      (-completeTopology cluster topology (CompleteTopologyParam.))))
 
-(defn -withSimulatedTime [^Runnable code]
+(defn -withSimulatedTime
+  [^Runnable code]
   (with-simulated-time
     (.run code)))
 
-(defmacro with-cluster [cluster-type mkClusterParam code]
+(defmacro with-cluster
+  [cluster-type mkClusterParam code]
   `(let [supervisors# (or (.getSupervisors ~mkClusterParam) 2)
          ports-per-supervisor# (or (.getPortsPerSupervisor ~mkClusterParam) 3)
          daemon-conf# (or (.getDaemonConf ~mkClusterParam) {})]
@@ -98,7 +99,8 @@
   ([^TestJob code]
      (-withTrackedCluster (MkClusterParam.) code)))
 
-(defn- find-tuples [^List fixed-tuples ^String stream]
+(defn- find-tuples
+  [^List fixed-tuples ^String stream]
   (let [ret (ArrayList.)]
     (doseq [fixed-tuple fixed-tuples]
       (if (= (.stream fixed-tuple) stream)
@@ -107,48 +109,52 @@
 
 (defn -readTuples
   ([^Map result ^String componentId ^String streamId]
-     (let [stream-result (.get result componentId)
-           ret (if stream-result
-                 (find-tuples stream-result streamId)
-                 [])]
-       ret))
+   (let [stream-result (.get result componentId)
+         ret (if stream-result
+               (find-tuples stream-result streamId)
+               [])]
+     ret))
   ([^Map result ^String componentId]
-     (-readTuples result componentId Utils/DEFAULT_STREAM_ID)))
+   (-readTuples result componentId Utils/DEFAULT_STREAM_ID)))
 
-(defn -mkTrackedTopology [^ILocalCluster trackedCluster ^StormTopology topology]
+(defn -mkTrackedTopology
+  [^ILocalCluster trackedCluster ^StormTopology topology]
   (-> (mk-tracked-topology (.getState trackedCluster) topology)
       (TrackedTopology.)))
 
 (defn -trackedWait
   ([^TrackedTopology trackedTopology ^Integer amt]
-     (tracked-wait trackedTopology amt))
+   (tracked-wait trackedTopology amt))
   ([^TrackedTopology trackedTopology]
-     (-trackedWait trackedTopology 1)))
+   (-trackedWait trackedTopology 1)))
 
 (defn -advanceClusterTime
   ([^ILocalCluster cluster ^Integer secs ^Integer step]
-     (advance-cluster-time (.getState cluster) secs step))
+   (advance-cluster-time (.getState cluster) secs step))
   ([^ILocalCluster cluster ^Integer secs]
-      (-advanceClusterTime cluster secs 1)))
+   (-advanceClusterTime cluster secs 1)))
 
-(defn- multiseteq [^Object obj1 ^Object obj2]
+(defn- multiseteq
+  [^Object obj1 ^Object obj2]
   (let [obj1 (clojurify-structure obj1)
         obj2 (clojurify-structure obj2)]
     (ms= obj1 obj2)))
 
-(defn -multiseteq [^Collection coll1 ^Collection coll2]
-     (multiseteq coll1 coll2))
+(defn -multiseteq
+  [^Collection coll1 ^Collection coll2]
+  (multiseteq coll1 coll2))
 
-(defn -multiseteq [^Map coll1 ^Map coll2]
+(defn -multiseteq
+  [^Map coll1 ^Map coll2]
   (multiseteq coll1 coll2))
 
 (defn -testTuple
   ([^List values]
-     (-testTuple values nil))
+   (-testTuple values nil))
   ([^List values ^MkTupleParam param]
-     (if (nil? param)
-       (test-tuple values)
-       (let [stream (or (.getStream param) Utils/DEFAULT_STREAM_ID)
-             component (or (.getComponent param) "component")
-             fields (.getFields param)]
-         (test-tuple values :stream stream :component component :fields fields)))))
\ No newline at end of file
+   (if (nil? param)
+     (test-tuple values)
+     (let [stream (or (.getStream param) Utils/DEFAULT_STREAM_ID)
+           component (or (.getComponent param) "component")
+           fields (.getFields param)]
+       (test-tuple values :stream stream :component component :fields fields)))))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/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 ec8ce5d..f778d23 100644
--- a/storm-core/src/clj/backtype/storm/thrift.clj
+++ b/storm-core/src/clj/backtype/storm/thrift.clj
@@ -13,11 +13,14 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.thrift
   (:import [java.util HashMap])
-  (:import [backtype.storm.generated JavaObject Grouping Nimbus StormTopology StormTopology$_Fields 
-    Bolt Nimbus$Client Nimbus$Iface ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
-    GlobalStreamId ComponentObject ComponentObject$_Fields ShellComponent])
+  (:import [backtype.storm.generated JavaObject Grouping Nimbus StormTopology
+            StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface
+            ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
+            GlobalStreamId ComponentObject ComponentObject$_Fields
+            ShellComponent])
   (:import [backtype.storm.utils Utils])
   (:import [backtype.storm Constants])
   (:import [backtype.storm.grouping CustomStreamGrouping])
@@ -25,14 +28,13 @@
   (:import [backtype.storm.clojure RichShellBolt RichShellSpout])
   (:import [org.apache.thrift.protocol TBinaryProtocol TProtocol])
   (:import [org.apache.thrift.transport TTransport TFramedTransport TSocket])
-  (:use [backtype.storm util config log])
-  )
+  (:use [backtype.storm util config log]))
 
-(defn instantiate-java-object [^JavaObject obj]
+(defn instantiate-java-object
+  [^JavaObject obj]
   (let [name (symbol (.get_full_class_name obj))
         args (map (memfn getFieldValue) (.get_args_list obj))]
-    (eval `(new ~name ~@args))
-    ))
+    (eval `(new ~name ~@args))))
 
 (def grouping-constants
   {Grouping$_Fields/FIELDS :fields
@@ -42,144 +44,184 @@
    Grouping$_Fields/CUSTOM_SERIALIZED :custom-serialized
    Grouping$_Fields/CUSTOM_OBJECT :custom-object
    Grouping$_Fields/DIRECT :direct
-   Grouping$_Fields/LOCAL_OR_SHUFFLE :local-or-shuffle
-  })
+   Grouping$_Fields/LOCAL_OR_SHUFFLE :local-or-shuffle})
 
-(defn grouping-type [^Grouping grouping]
+(defn grouping-type
+  [^Grouping grouping]
   (grouping-constants (.getSetField grouping)))
 
-(defn field-grouping [^Grouping grouping]
+(defn field-grouping
+  [^Grouping grouping]
   (when-not (= (grouping-type grouping) :fields)
     (throw (IllegalArgumentException. "Tried to get grouping fields from non fields grouping")))
   (.get_fields grouping))
 
-(defn global-grouping? [^Grouping grouping]
+(defn global-grouping?
+  [^Grouping grouping]
   (and (= :fields (grouping-type grouping))
-       (empty? (field-grouping grouping))
-       ))
+       (empty? (field-grouping grouping))))
 
-(defn parallelism-hint [^ComponentCommon component-common]
+(defn parallelism-hint
+  [^ComponentCommon component-common]
   (let [phint (.get_parallelism_hint component-common)]
-    (if-not (.is_set_parallelism_hint component-common) 1 phint)
-    ))
+    (if-not (.is_set_parallelism_hint component-common) 1 phint)))
 
-(defn nimbus-client-and-conn [host port]
+(defn nimbus-client-and-conn
+  [host port]
   (log-message "Connecting to Nimbus at " host ":" port)
   (let [transport (TFramedTransport. (TSocket. host port))
         prot (TBinaryProtocol. transport)
         client (Nimbus$Client. prot)]
-        (.open transport)
-        [client transport] ))
-
-(defmacro with-nimbus-connection [[client-sym host port] & body]
-  `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)]
-      (try
-        ~@body
-      (finally (.close conn#)))
-      ))
-
-(defmacro with-configured-nimbus-connection [client-sym & body]
+    (.open transport)
+    [client transport]))
+
+(defmacro with-nimbus-connection
+  [[client-sym host port] & body]
+  `(let [[^Nimbus$Client ~client-sym ^TTransport conn#]
+         (nimbus-client-and-conn ~host ~port)]
+     (try
+       ~@body
+       (finally (.close conn#)))))
+
+(defmacro with-configured-nimbus-connection
+  [client-sym & body]
   `(let [conf# (read-storm-config)
          host# (conf# NIMBUS-HOST)
          port# (conf# NIMBUS-THRIFT-PORT)]
      (with-nimbus-connection [~client-sym host# port#]
        ~@body )))
 
-(defn direct-output-fields [fields]
+(defn direct-output-fields
+  [fields]
   (StreamInfo. fields true))
 
-(defn output-fields [fields]
+(defn output-fields
+  [fields]
   (StreamInfo. fields false))
 
-(defn mk-output-spec [output-spec]
-  (let [output-spec (if (map? output-spec) output-spec {Utils/DEFAULT_STREAM_ID output-spec})]
+(defn mk-output-spec
+  [output-spec]
+  (let [output-spec (if (map? output-spec)
+                      output-spec
+                      {Utils/DEFAULT_STREAM_ID output-spec})]
     (map-val
       (fn [out]
         (if (instance? StreamInfo out)
           out
-          (StreamInfo. out false)
-          ))
-      output-spec
-      )))
+          (StreamInfo. out false)))
+      output-spec)))
 
-(defnk mk-plain-component-common [inputs output-spec parallelism-hint :conf nil]
+(defnk mk-plain-component-common
+  [inputs output-spec parallelism-hint :conf nil]
   (let [ret (ComponentCommon. (HashMap. inputs) (HashMap. (mk-output-spec output-spec)))]
     (when parallelism-hint
       (.set_parallelism_hint ret parallelism-hint))
     (when conf
       (.set_json_conf ret (to-json conf)))
-    ret
-    ))
+    ret))
 
-(defnk mk-spout-spec* [spout outputs :p nil :conf nil]
+(defnk mk-spout-spec*
+  [spout outputs :p nil :conf nil]
   (SpoutSpec. (ComponentObject/serialized_java (Utils/serialize spout))
               (mk-plain-component-common {} outputs p :conf conf)))
 
-(defn mk-shuffle-grouping []
+(defn mk-shuffle-grouping
+  []
   (Grouping/shuffle (NullStruct.)))
 
-(defn mk-local-or-shuffle-grouping []
+(defn mk-local-or-shuffle-grouping
+  []
   (Grouping/local_or_shuffle (NullStruct.)))
 
-(defn mk-fields-grouping [fields]
+(defn mk-fields-grouping
+  [fields]
   (Grouping/fields fields))
 
-(defn mk-global-grouping []
+(defn mk-global-grouping
+  []
   (mk-fields-grouping []))
 
-(defn mk-direct-grouping []
+(defn mk-direct-grouping
+  []
   (Grouping/direct (NullStruct.)))
 
-(defn mk-all-grouping []
+(defn mk-all-grouping
+  []
   (Grouping/all (NullStruct.)))
 
-(defn mk-none-grouping []
+(defn mk-none-grouping
+  []
   (Grouping/none (NullStruct.)))
 
-(defn deserialized-component-object [^ComponentObject obj]
+(defn deserialized-component-object
+  [^ComponentObject obj]
   (when (not= (.getSetField obj) ComponentObject$_Fields/SERIALIZED_JAVA)
     (throw (RuntimeException. "Cannot deserialize non-java-serialized object")))
-  (Utils/deserialize (.get_serialized_java obj))
-  )
+  (Utils/deserialize (.get_serialized_java obj)))
 
-(defn serialize-component-object [obj]
+(defn serialize-component-object
+  [obj]
   (ComponentObject/serialized_java (Utils/serialize obj)))
 
-(defn- mk-grouping [grouping-spec]
-  (cond (nil? grouping-spec) (mk-none-grouping)
-        (instance? Grouping grouping-spec) grouping-spec
-        (instance? CustomStreamGrouping grouping-spec) (Grouping/custom_serialized (Utils/serialize grouping-spec))
-        (instance? JavaObject grouping-spec) (Grouping/custom_object grouping-spec)
-        (sequential? grouping-spec) (mk-fields-grouping grouping-spec)
-        (= grouping-spec :shuffle) (mk-shuffle-grouping)
-        (= grouping-spec :local-or-shuffle) (mk-local-or-shuffle-grouping)
-        (= grouping-spec :none) (mk-none-grouping)
-        (= grouping-spec :all) (mk-all-grouping)
-        (= grouping-spec :global) (mk-global-grouping)
-        (= grouping-spec :direct) (mk-direct-grouping)
-        true (throw (IllegalArgumentException. (str grouping-spec " is not a valid grouping")))
-        ))
-
-(defn- mk-inputs [inputs]
-  (into {}
-    (for [[stream-id grouping-spec] inputs]
-      [(if (sequential? stream-id)
-         (GlobalStreamId. (first stream-id) (second stream-id))
-         (GlobalStreamId. stream-id Utils/DEFAULT_STREAM_ID))
-       (mk-grouping grouping-spec)]
-      )))
-
-(defnk mk-bolt-spec* [inputs bolt outputs :p nil :conf nil]
+(defn- mk-grouping
+  [grouping-spec]
+  (cond (nil? grouping-spec)
+        (mk-none-grouping)
+
+        (instance? Grouping grouping-spec)
+        grouping-spec
+
+        (instance? CustomStreamGrouping grouping-spec)
+        (Grouping/custom_serialized (Utils/serialize grouping-spec))
+
+        (instance? JavaObject grouping-spec)
+        (Grouping/custom_object grouping-spec)
+
+        (sequential? grouping-spec)
+        (mk-fields-grouping grouping-spec)
+
+        (= grouping-spec :shuffle)
+        (mk-shuffle-grouping)
+
+        (= grouping-spec :local-or-shuffle)
+        (mk-local-or-shuffle-grouping)
+        (= grouping-spec :none)
+        (mk-none-grouping)
+
+        (= grouping-spec :all)
+        (mk-all-grouping)
+
+        (= grouping-spec :global)
+        (mk-global-grouping)
+
+        (= grouping-spec :direct)
+        (mk-direct-grouping)
+
+        true
+        (throw (IllegalArgumentException.
+                 (str grouping-spec " is not a valid grouping")))))
+
+(defn- mk-inputs
+  [inputs]
+  (into {} (for [[stream-id grouping-spec] inputs]
+             [(if (sequential? stream-id)
+                (GlobalStreamId. (first stream-id) (second stream-id))
+                (GlobalStreamId. stream-id Utils/DEFAULT_STREAM_ID))
+              (mk-grouping grouping-spec)])))
+
+(defnk mk-bolt-spec*
+  [inputs bolt outputs :p nil :conf nil]
   (let [common (mk-plain-component-common (mk-inputs inputs) outputs p :conf conf)]
     (Bolt. (ComponentObject/serialized_java (Utils/serialize bolt))
-           common )))
+           common)))
 
-(defnk mk-spout-spec [spout :parallelism-hint nil :p nil :conf nil]
+(defnk mk-spout-spec
+  [spout :parallelism-hint nil :p nil :conf nil]
   (let [parallelism-hint (if p p parallelism-hint)]
-    {:obj spout :p parallelism-hint :conf conf}
-    ))
+    {:obj spout :p parallelism-hint :conf conf}))
 
-(defn- shell-component-params [command script-or-output-spec kwargs]
+(defn- shell-component-params
+  [command script-or-output-spec kwargs]
   (if (string? script-or-output-spec)
     [(into-array String [command script-or-output-spec])
      (first kwargs)
@@ -188,41 +230,46 @@
      script-or-output-spec
      kwargs]))
 
-(defnk mk-bolt-spec [inputs bolt :parallelism-hint nil :p nil :conf nil]
+(defnk mk-bolt-spec
+  [inputs bolt :parallelism-hint nil :p nil :conf nil]
   (let [parallelism-hint (if p p parallelism-hint)]
-    {:obj bolt :inputs inputs :p parallelism-hint :conf conf}
-    ))
+    {:obj bolt :inputs inputs :p parallelism-hint :conf conf}))
 
-(defn mk-shell-bolt-spec [inputs command script-or-output-spec & kwargs]
+(defn mk-shell-bolt-spec
+  [inputs command script-or-output-spec & kwargs]
   (let [[command output-spec kwargs]
         (shell-component-params command script-or-output-spec kwargs)]
-    (apply mk-bolt-spec inputs (RichShellBolt. command (mk-output-spec output-spec)) kwargs)))
+    (apply mk-bolt-spec inputs
+           (RichShellBolt. command (mk-output-spec output-spec)) kwargs)))
 
-(defn mk-shell-spout-spec [command script-or-output-spec & kwargs]
+(defn mk-shell-spout-spec
+  [command script-or-output-spec & kwargs]
   (let [[command output-spec kwargs]
         (shell-component-params command script-or-output-spec kwargs)]
-   (apply mk-spout-spec (RichShellSpout. command (mk-output-spec output-spec)) kwargs)))
+    (apply mk-spout-spec
+           (RichShellSpout. command (mk-output-spec output-spec)) kwargs)))
 
-(defn- add-inputs [declarer inputs]
+(defn- add-inputs
+  [declarer inputs]
   (doseq [[id grouping] (mk-inputs inputs)]
-    (.grouping declarer id grouping)
-    ))
+    (.grouping declarer id grouping)))
 
 (defn mk-topology
   ([spout-map bolt-map]
-    (let [builder (TopologyBuilder.)]
-      (doseq [[name {spout :obj p :p conf :conf}] spout-map]
-        (-> builder (.setSpout name spout (if-not (nil? p) (int p) p)) (.addConfigurations conf)))
-      (doseq [[name {bolt :obj p :p conf :conf inputs :inputs}] bolt-map]
-        (-> builder (.setBolt name bolt (if-not (nil? p) (int p) p)) (.addConfigurations conf) (add-inputs inputs)))
-      (.createTopology builder)
-      ))
+   (let [builder (TopologyBuilder.)]
+     (doseq [[name {spout :obj p :p conf :conf}] spout-map]
+       (-> builder (.setSpout name spout (if-not (nil? p) (int p) p)) (.addConfigurations conf)))
+     (doseq [[name {bolt :obj p :p conf :conf inputs :inputs}] bolt-map]
+       (-> builder (.setBolt name bolt (if-not (nil? p) (int p) p)) (.addConfigurations conf) (add-inputs inputs)))
+     (.createTopology builder)))
   ([spout-map bolt-map state-spout-map]
-     (mk-topology spout-map bolt-map)))
+   (mk-topology spout-map bolt-map)))
 
 ;; clojurify-structure is needed or else every element becomes the same after successive calls
 ;; don't know why this happens
-(def STORM-TOPOLOGY-FIELDS (-> StormTopology/metaDataMap clojurify-structure keys))
+(def STORM-TOPOLOGY-FIELDS
+  (-> StormTopology/metaDataMap clojurify-structure keys))
 
-(def SPOUT-FIELDS [StormTopology$_Fields/SPOUTS
-                   StormTopology$_Fields/STATE_SPOUTS])
+(def SPOUT-FIELDS
+  [StormTopology$_Fields/SPOUTS
+   StormTopology$_Fields/STATE_SPOUTS])

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/timer.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/timer.clj b/storm-core/src/clj/backtype/storm/timer.clj
index 78b1f1c..b105698 100644
--- a/storm-core/src/clj/backtype/storm/timer.clj
+++ b/storm-core/src/clj/backtype/storm/timer.clj
@@ -13,57 +13,55 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.timer
   (:import [backtype.storm.utils Time])
   (:import [java.util PriorityQueue Comparator])
   (:import [java.util.concurrent Semaphore])
-  (:use [backtype.storm util log])
-  )
+  (:use [backtype.storm util log]))
 
-;; The timer defined in this file is very similar to java.util.Timer, except it integrates with
-;; Storm's time simulation capabilities. This lets us test code that does asynchronous work on the timer thread
+;; The timer defined in this file is very similar to java.util.Timer, except
+;; it integrates with Storm's time simulation capabilities. This lets us test
+;; code that does asynchronous work on the timer thread
 
-(defnk mk-timer [:kill-fn (fn [& _] )]
+(defnk mk-timer
+  [:kill-fn (fn [& _] )]
   (let [queue (PriorityQueue. 10
                               (reify Comparator
                                 (compare [this o1 o2]
-                                  (- (first o1) (first o2))
-                                  )
+                                         (- (first o1) (first o2)))
                                 (equals [this obj]
-                                  true
-                                  )))
+                                        true)))
         active (atom true)
         lock (Object.)
         notifier (Semaphore. 0)
         timer-thread (Thread.
-                      (fn []
-                        (while @active
-                          (try
-                            (let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
-                              (if (and elem (>= (current-time-millis) time-millis))
-                                ;; imperative to not run the function inside the timer lock
-                                ;; otherwise, it's possible to deadlock if function deals with other locks
-                                ;; (like the submit lock)
-                                (let [afn (locking lock (second (.poll queue)))]
-                                  (afn))
-                                (if time-millis ;; if any events are scheduled
-                                  ;; sleep until event generation
-                                  ;; note that if any recurring events are scheduled then we will always go through
-                                  ;; this branch, sleeping only the exact necessary amount of time
-                                  (Time/sleep (- time-millis (current-time-millis)))
-                                  ;; else poll to see if any new event was scheduled
-                                  ;; this is in essence the response time for detecting any new event schedulings when
-                                  ;; there are no scheduled events
-                                  (Time/sleep 1000))
-                                ))
-                            (catch Throwable t
-                              ;; because the interrupted exception can be wrapped in a runtimeexception
-                              (when-not (exception-cause? InterruptedException t)
-                                (kill-fn t)
-                                (reset! active false)
-                                (throw t))
-                              )))
-                        (.release notifier)))]
+                       (fn []
+                         (while @active
+                           (try
+                             (let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
+                               (if (and elem (>= (current-time-millis) time-millis))
+                                 ;; imperative to not run the function inside the timer lock
+                                 ;; otherwise, it's possible to deadlock if function deals with other locks
+                                 ;; (like the submit lock)
+                                 (let [afn (locking lock (second (.poll queue)))]
+                                   (afn))
+                                 (if time-millis ;; if any events are scheduled
+                                   ;; sleep until event generation
+                                   ;; note that if any recurring events are scheduled then we will always go through
+                                   ;; this branch, sleeping only the exact necessary amount of time
+                                   (Time/sleep (- time-millis (current-time-millis)))
+                                   ;; else poll to see if any new event was scheduled
+                                   ;; this is in essence the response time for detecting any new event schedulings when
+                                   ;; there are no scheduled events
+                                   (Time/sleep 1000))))
+                             (catch Throwable t
+                               ;; because the interrupted exception can be wrapped in a runtimeexception
+                               (when-not (exception-cause? InterruptedException t)
+                                 (kill-fn t)
+                                 (reset! active false)
+                                 (throw t)))))
+                         (.release notifier)))]
     (.setDaemon timer-thread true)
     (.setPriority timer-thread Thread/MAX_PRIORITY)
     (.start timer-thread)
@@ -73,32 +71,36 @@
      :lock lock
      :cancel-notifier notifier}))
 
-(defn- check-active! [timer]
+(defn- check-active!
+  [timer]
   (when-not @(:active timer)
     (throw (IllegalStateException. "Timer is not active"))))
 
-(defnk schedule [timer delay-secs afn :check-active true]
+(defnk schedule
+  [timer delay-secs afn :check-active true]
   (when check-active (check-active! timer))
   (let [id (uuid)
         ^PriorityQueue queue (:queue timer)]
     (locking (:lock timer)
-      (.add queue [(+ (current-time-millis) (secs-to-millis-long delay-secs)) afn id])
-      )))
+      (.add queue [(+ (current-time-millis) (secs-to-millis-long delay-secs)) afn id]))))
 
-(defn schedule-recurring [timer delay-secs recur-secs afn]
+(defn schedule-recurring
+  [timer delay-secs recur-secs afn]
   (schedule timer
             delay-secs
             (fn this []
               (afn)
-              (schedule timer recur-secs this :check-active false)) ; this avoids a race condition with cancel-timer
-            ))
+              ; this avoids a race condition with cancel-timer
+              (schedule timer recur-secs this :check-active false))))
 
-(defn cancel-timer [timer]
+(defn cancel-timer
+  [timer]
   (check-active! timer)
   (locking (:lock timer)
     (reset! (:active timer) false)
     (.interrupt (:timer-thread timer)))
   (.acquire (:cancel-notifier timer)))
 
-(defn timer-waiting? [timer]
+(defn timer-waiting?
+  [timer]
   (Time/isThreadWaiting (:timer-thread timer)))

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2278fc96/storm-core/src/clj/backtype/storm/tuple.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/tuple.clj b/storm-core/src/clj/backtype/storm/tuple.clj
index eaec818..29df727 100644
--- a/storm-core/src/clj/backtype/storm/tuple.clj
+++ b/storm-core/src/clj/backtype/storm/tuple.clj
@@ -13,11 +13,12 @@
 ;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
+
 (ns backtype.storm.tuple
-  (:use [backtype.storm bootstrap])
-  )
+  (:use [backtype.storm bootstrap]))
 
 (bootstrap)
 
-(defn list-hash-code [^List alist]
+(defn list-hash-code
+  [^List alist]
   (.hashCode alist))


[08/13] Merge branch 'master' into idiomatic-clojure-01

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java
index 6996b49..8d2d221 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java
@@ -21,52 +21,57 @@ import backtype.storm.Config;
 import backtype.storm.messaging.IConnection;
 import backtype.storm.messaging.TaskMessage;
 import backtype.storm.utils.Utils;
-
 import org.jboss.netty.bootstrap.ClientBootstrap;
 import org.jboss.netty.channel.Channel;
 import org.jboss.netty.channel.ChannelFactory;
-import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
 import org.jboss.netty.channel.ChannelFuture;
 import org.jboss.netty.channel.ChannelFutureListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
-import java.util.Timer;
-import java.util.TimerTask;
 import java.util.Random;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
-class Client implements IConnection {
+public class Client implements IConnection {
     private static final Logger LOG = LoggerFactory.getLogger(Client.class);
-    private static final Timer TIMER = new Timer("netty-client-timer", true);
-
+    private static final String PREFIX = "Netty-Client-";
     private final int max_retries;
     private final long base_sleep_ms;
     private final long max_sleep_ms;
-    private LinkedBlockingQueue<Object> message_queue; //entry should either be TaskMessage or ControlMessage
     private AtomicReference<Channel> channelRef;
     private final ClientBootstrap bootstrap;
-    InetSocketAddress remote_addr;
-    private AtomicInteger retries;
+    private InetSocketAddress remote_addr;
+    
     private final Random random = new Random();
     private final ChannelFactory factory;
     private final int buffer_size;
-    private final AtomicBoolean being_closed;
-    private boolean wait_for_requests;
+    private boolean closing;
+
+    private int messageBatchSize;
+    
+    private AtomicLong pendings;
+
+    MessageBatch messageBatch = null;
+    private AtomicLong flushCheckTimer;
+    private int flushCheckInterval;
+    private ScheduledExecutorService scheduler;
 
     @SuppressWarnings("rawtypes")
-    Client(Map storm_conf, ChannelFactory factory, String host, int port) {
+    Client(Map storm_conf, ChannelFactory factory, 
+            ScheduledExecutorService scheduler, String host, int port) {
         this.factory = factory;
-        message_queue = new LinkedBlockingQueue<Object>();
-        retries = new AtomicInteger(0);
+        this.scheduler = scheduler;
         channelRef = new AtomicReference<Channel>(null);
-        being_closed = new AtomicBoolean(false);
-        wait_for_requests = false;
+        closing = false;
+        pendings = new AtomicLong(0);
+        flushCheckTimer = new AtomicLong(Long.MAX_VALUE);
 
         // Configure
         buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
@@ -74,6 +79,13 @@ class Client implements IConnection {
         base_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS));
         max_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS));
 
+        this.messageBatchSize = Utils.getInt(storm_conf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144);
+        
+        flushCheckInterval = Utils.getInt(storm_conf.get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); // default 10 ms
+
+        LOG.info("New Netty Client, connect to " + host + ", " + port
+                + ", config: " + ", buffer_size: " + buffer_size);
+
         bootstrap = new ClientBootstrap(factory);
         bootstrap.setOption("tcpNoDelay", true);
         bootstrap.setOption("sendBufferSize", buffer_size);
@@ -84,43 +96,88 @@ class Client implements IConnection {
 
         // Start the connection attempt.
         remote_addr = new InetSocketAddress(host, port);
-        bootstrap.connect(remote_addr);
+        
+        // setup the connection asyncly now
+        scheduler.execute(new Runnable() {
+            @Override
+            public void run() {   
+                connect();
+            }
+        });
+        
+        Runnable flusher = new Runnable() {
+            @Override
+            public void run() {
+
+                if(!closing) {
+                    long flushCheckTime = flushCheckTimer.get();
+                    long now = System.currentTimeMillis();
+                    if (now > flushCheckTime) {
+                        Channel channel = channelRef.get();
+                        if (null != channel && channel.isWritable()) {
+                            flush(channel);
+                        }
+                    }
+                }
+                
+            }
+        };
+        
+        long initialDelay = Math.min(30L * 1000, max_sleep_ms * max_retries); //max wait for 30s
+        scheduler.scheduleWithFixedDelay(flusher, initialDelay, flushCheckInterval, TimeUnit.MILLISECONDS);
     }
 
     /**
      * We will retry connection with exponential back-off policy
      */
-    void reconnect() {
-        close_n_release();
-
-        //reconnect only if it's not being closed
-        if (being_closed.get()) return;
-
-        final int tried_count = retries.incrementAndGet();
-        if (tried_count <= max_retries) {
-            long sleep = getSleepTimeMs();
-            LOG.info("Waiting {} ms before trying connection to {}", sleep, remote_addr);
-            TIMER.schedule(new TimerTask() {
-                @Override
-                public void run() { 
-                    LOG.info("Reconnect ... [{}] to {}", tried_count, remote_addr);
-                    bootstrap.connect(remote_addr);
-                }}, sleep);
-        } else {
-            LOG.warn(remote_addr+" is not reachable. We will close this client.");
-            close();
+    private synchronized void connect() {
+        try {
+            if (channelRef.get() != null) {
+                return;
+            }
+            
+            Channel channel = null;
+
+            int tried = 0;
+            while (tried <= max_retries) {
+
+                LOG.info("Reconnect started for {}... [{}]", name(), tried);
+                LOG.debug("connection started...");
+
+                ChannelFuture future = bootstrap.connect(remote_addr);
+                future.awaitUninterruptibly();
+                Channel current = future.getChannel();
+                if (!future.isSuccess()) {
+                    if (null != current) {
+                        current.close();
+                    }
+                } else {
+                    channel = current;
+                    break;
+                }
+                Thread.sleep(getSleepTimeMs(tried));
+                tried++;  
+            }
+            if (null != channel) {
+                LOG.info("connection established to a remote host " + name() + ", " + channel.toString());
+                channelRef.set(channel);
+            } else {
+                close();
+                throw new RuntimeException("Remote address is not reachable. We will close this client " + name());
+            }
+        } catch (InterruptedException e) {
+            throw new RuntimeException("connection failed " + name(), e);
         }
     }
 
     /**
      * # of milliseconds to wait per exponential back-off policy
      */
-    private long getSleepTimeMs()
-    {
-        if (retries.get() > 30) {
+    private long getSleepTimeMs(int retries) {
+        if (retries > 30) {
            return max_sleep_ms;
         }
-        int backoff = 1 << retries.get();
+        int backoff = 1 << retries;
         long sleepMs = base_sleep_ms * Math.max(1, random.nextInt(backoff));
         if ( sleepMs > max_sleep_ms )
             sleepMs = max_sleep_ms;
@@ -128,133 +185,114 @@ class Client implements IConnection {
     }
 
     /**
-     * Enqueue a task message to be sent to server
+     * Enqueue task messages to be sent to server
      */
-    public void send(int task, byte[] message) {
-        //throw exception if the client is being closed
-        if (being_closed.get()) {
+    synchronized public void send(Iterator<TaskMessage> msgs) {
+
+        // throw exception if the client is being closed
+        if (closing) {
             throw new RuntimeException("Client is being closed, and does not take requests any more");
         }
-
-        try {
-            message_queue.put(new TaskMessage(task, message));
-
-            //resume delivery if it is waiting for requests
-            tryDeliverMessages(true);
-        } catch (InterruptedException e) {
-            throw new RuntimeException(e);
+        
+        if (null == msgs || !msgs.hasNext()) {
+            return;
         }
-    }
-
-    /**
-     * Retrieve messages from queue, and delivery to server if any
-     */
-    synchronized void tryDeliverMessages(boolean only_if_waiting) throws InterruptedException {
-        //just skip if delivery only if waiting, and we are not waiting currently
-        if (only_if_waiting && !wait_for_requests)  return;
 
-        //make sure that channel was not closed
         Channel channel = channelRef.get();
-        if (channel == null)  return;
-        if (!channel.isOpen()) {
-            LOG.info("Channel to {} is no longer open.",remote_addr);
-            //The channel is not open yet. Reconnect?
-            reconnect();
-            return;
+        if (null == channel) {
+            connect();
+            channel = channelRef.get();
         }
 
-        final MessageBatch requests = tryTakeMessages();
-        if (requests==null) {
-            wait_for_requests = true;
-            return;
-        }
+        while (msgs.hasNext()) {
+            TaskMessage message = msgs.next();
+            if (null == messageBatch) {
+                messageBatch = new MessageBatch(messageBatchSize);
+            }
 
-        //if channel is being closed and we have no outstanding messages,  let's close the channel
-        if (requests.isEmpty() && being_closed.get()) {
-            close_n_release();
-            return;
+            messageBatch.add(message);
+            if (messageBatch.isFull()) {
+                MessageBatch toBeFlushed = messageBatch;
+                flushRequest(channel, toBeFlushed);
+                messageBatch = null;
+            }
         }
 
-        //we are busily delivering messages, and will check queue upon response.
-        //When send() is called by senders, we should not thus call tryDeliverMessages().
-        wait_for_requests = false;
-
-        //write request into socket channel
-        ChannelFuture future = channel.write(requests);
-        future.addListener(new ChannelFutureListener() {
-            public void operationComplete(ChannelFuture future)
-                    throws Exception {
-                if (!future.isSuccess()) {
-                    LOG.info("failed to send "+requests.size()+" requests to "+remote_addr, future.getCause());
-                    reconnect();
-                } else {
-                    LOG.debug("{} request(s) sent", requests.size());
-
-                    //Now that our requests have been sent, channel could be closed if needed
-                    if (being_closed.get())
-                        close_n_release();
-                }
+        if (null != messageBatch && !messageBatch.isEmpty()) {
+            if (channel.isWritable()) {
+                flushCheckTimer.set(Long.MAX_VALUE);
+                
+                // Flush as fast as we can to reduce the latency
+                MessageBatch toBeFlushed = messageBatch;
+                messageBatch = null;
+                flushRequest(channel, toBeFlushed);
+                
+            } else {
+                // when channel is NOT writable, it means the internal netty buffer is full. 
+                // In this case, we can try to buffer up more incoming messages.
+                flushCheckTimer.set(System.currentTimeMillis() + flushCheckInterval);
             }
-        });
+        }
+
     }
 
-    /**
-     * Take all enqueued messages from queue
-     * @return  batch of messages
-     * @throws InterruptedException
-     *
-     * synchronized ... ensure that messages are delivered in the same order
-     * as they are added into queue
-     */
-    private MessageBatch tryTakeMessages() throws InterruptedException {
-        //1st message
-        Object msg = message_queue.poll();
-        if (msg == null) return null;
-
-        MessageBatch batch = new MessageBatch(buffer_size);
-        //we will discard any message after CLOSE
-        if (msg == ControlMessage.CLOSE_MESSAGE) {
-            LOG.info("Connection to {} is being closed", remote_addr);
-            being_closed.set(true);
-            return batch;
+    public String name() {
+        if (null != remote_addr) {
+            return PREFIX + remote_addr.toString();
         }
+        return "";
+    }
 
-        batch.add((TaskMessage)msg);
-        while (!batch.isFull() && ((msg = message_queue.peek())!=null)) {
-            //Is it a CLOSE message?
-            if (msg == ControlMessage.CLOSE_MESSAGE) {
-                message_queue.take();
-                LOG.info("Connection to {} is being closed", remote_addr);
-                being_closed.set(true);
-                break;
+    private synchronized void flush(Channel channel) {
+        if (!closing) {
+            if (null != messageBatch && !messageBatch.isEmpty()) {
+                MessageBatch toBeFlushed = messageBatch;
+                flushCheckTimer.set(Long.MAX_VALUE);
+                flushRequest(channel, toBeFlushed);
+                messageBatch = null;
             }
-
-            //try to add this msg into batch
-            if (!batch.tryAdd((TaskMessage) msg))
-                break;
-
-            //remove this message
-            message_queue.take();
         }
-
-        return batch;
     }
-
+    
     /**
      * gracefully close this client.
-     *
-     * We will send all existing requests, and then invoke close_n_release() method
+     * 
+     * We will send all existing requests, and then invoke close_n_release()
+     * method
      */
-    public void close() {
-        //enqueue a CLOSE message so that shutdown() will be invoked
-        try {
-            message_queue.put(ControlMessage.CLOSE_MESSAGE);
-
-            //resume delivery if it is waiting for requests
-            tryDeliverMessages(true);
-        } catch (InterruptedException e) {
-            LOG.info("Interrupted Connection to {} is being closed", remote_addr);
-            being_closed.set(true);
+    public synchronized void close() {
+        if (!closing) {
+            closing = true;
+            LOG.info("Closing Netty Client " + name());
+            
+            if (null != messageBatch && !messageBatch.isEmpty()) {
+                MessageBatch toBeFlushed = messageBatch;
+                Channel channel = channelRef.get();
+                if (channel != null) {
+                    flushRequest(channel, toBeFlushed);
+                }
+                messageBatch = null;
+            }
+        
+            //wait for pendings to exit
+            final long timeoutMilliSeconds = 600 * 1000; //600 seconds
+            final long start = System.currentTimeMillis();
+            
+            LOG.info("Waiting for pending batchs to be sent with "+ name() + "..., timeout: {}ms, pendings: {}", timeoutMilliSeconds, pendings.get());
+            
+            while(pendings.get() != 0) {
+                try {
+                    long delta = System.currentTimeMillis() - start;
+                    if (delta > timeoutMilliSeconds) {
+                        LOG.error("Timeout when sending pending batchs with {}..., there are still {} pending batchs not sent", name(), pendings.get());
+                        break;
+                    }
+                    Thread.sleep(1000); //sleep 1s
+                } catch (InterruptedException e) {
+                    break;
+                } 
+            }
+            
             close_n_release();
         }
     }
@@ -262,27 +300,51 @@ class Client implements IConnection {
     /**
      * close_n_release() is invoked after all messages have been sent.
      */
-    synchronized void close_n_release() {
+    private void close_n_release() {
         if (channelRef.get() != null) {
             channelRef.get().close();
             LOG.debug("channel {} closed",remote_addr);
-            setChannel(null);
         }
     }
 
-    public TaskMessage recv(int flags) {
+    @Override
+    public Iterator<TaskMessage> recv(int flags, int clientId) {
         throw new RuntimeException("Client connection should not receive any messages");
     }
 
-    void setChannel(Channel channel) {
-        if (channel != null && channel.isOpen()) {
-            //Assume the most recent connection attempt was successful.
-            retries.set(0);
-        }
-        channelRef.set(channel);
-        //reset retries
-        if (channel != null)
-            retries.set(0);
+    @Override
+    public void send(int taskId, byte[] payload) {
+        TaskMessage msg = new TaskMessage(taskId, payload);
+        List<TaskMessage> wrapper = new ArrayList<TaskMessage>(1);
+        wrapper.add(msg);
+        send(wrapper.iterator());
     }
 
-}
+    private void flushRequest(Channel channel, final MessageBatch requests) {
+        if (requests == null)
+            return;
+
+        pendings.incrementAndGet();
+        ChannelFuture future = channel.write(requests);
+        future.addListener(new ChannelFutureListener() {
+            public void operationComplete(ChannelFuture future)
+                    throws Exception {
+
+                pendings.decrementAndGet();
+                if (!future.isSuccess()) {
+                    LOG.info(
+                            "failed to send requests to " + remote_addr.toString() + ": ", future.getCause());
+
+                    Channel channel = future.getChannel();
+
+                    if (null != channel) {
+                        channel.close();
+                        channelRef.compareAndSet(channel, null);
+                    }
+                } else {
+                    LOG.debug("{} request(s) sent", requests.size());
+                }
+            }
+        });
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java
index 80b4443..f592aff 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java
@@ -18,8 +18,12 @@
 package backtype.storm.messaging.netty;
 
 import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
 import java.util.Map;
 import java.util.Vector;
 
@@ -28,14 +32,16 @@ import backtype.storm.messaging.IConnection;
 import backtype.storm.messaging.IContext;
 import backtype.storm.utils.Utils;
 
-import java.util.Map;
-import java.util.Vector;
-
 public class Context implements IContext {
+    private static final Logger LOG = LoggerFactory.getLogger(Context.class);
+        
     @SuppressWarnings("rawtypes")
     private Map storm_conf;
     private volatile Vector<IConnection> connections;
     private NioClientSocketChannelFactory clientChannelFactory;
+    
+    private ScheduledExecutorService clientScheduleService;
+    private final int MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE = 10;
 
     /**
      * initialization per Storm configuration 
@@ -47,13 +53,19 @@ public class Context implements IContext {
 
         //each context will have a single client channel factory
         int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS));
+		ThreadFactory bossFactory = new NettyRenameThreadFactory("client" + "-boss");
+        ThreadFactory workerFactory = new NettyRenameThreadFactory("client" + "-worker");
         if (maxWorkers > 0) {
-            clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(),
-                    Executors.newCachedThreadPool(), maxWorkers);
+            clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
+                    Executors.newCachedThreadPool(workerFactory), maxWorkers);
         } else {
-            clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(),
-                    Executors.newCachedThreadPool());
+            clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
+                    Executors.newCachedThreadPool(workerFactory));
         }
+        
+        int otherWorkers = Utils.getInt(storm_conf.get(Config.TOPOLOGY_WORKERS), 1) - 1;
+        int poolSize = Math.min(Math.max(1, otherWorkers), MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE);
+        clientScheduleService = Executors.newScheduledThreadPool(poolSize, new NettyRenameThreadFactory("client-schedule-service"));
     }
 
     /**
@@ -69,7 +81,8 @@ public class Context implements IContext {
      * establish a connection to a remote server
      */
     public IConnection connect(String storm_id, String host, int port) {        
-        IConnection client =  new Client(storm_conf, clientChannelFactory, host, port);
+        IConnection client =  new Client(storm_conf, clientChannelFactory, 
+                clientScheduleService, host, port);
         connections.add(client);
         return client;
     }
@@ -78,12 +91,22 @@ public class Context implements IContext {
      * terminate this context
      */
     public void term() {
+        clientScheduleService.shutdown();        
+        
         for (IConnection conn : connections) {
             conn.close();
         }
+        
+        try {
+            clientScheduleService.awaitTermination(30, TimeUnit.SECONDS);
+        } catch (InterruptedException e) {
+            LOG.error("Error when shutting down client scheduler", e);
+        }
+        
         connections = null;
 
         //we need to release resources associated with client channel factory
         clientChannelFactory.releaseExternalResources();
+
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java b/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java
index a552cf7..b7335b3 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java
@@ -17,6 +17,8 @@
  */
 package backtype.storm.messaging.netty;
 
+import java.io.IOException;
+
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBufferOutputStream;
 import org.jboss.netty.buffer.ChannelBuffers;
@@ -54,14 +56,14 @@ enum ControlMessage {
      * encode the current Control Message into a channel buffer
      * @throws Exception
      */
-    ChannelBuffer buffer() throws Exception {
+    ChannelBuffer buffer() throws IOException {
         ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encodeLength()));      
         write(bout);
         bout.close();
         return bout.buffer();
     }
 
-    void write(ChannelBufferOutputStream bout) throws Exception {
+    void write(ChannelBufferOutputStream bout) throws IOException {
         bout.writeShort(code);        
     } 
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java
index cd8d4e3..63c861a 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java
@@ -44,6 +44,11 @@ class MessageBatch {
         encoded_length += msgEncodeLength(msg);
     }
 
+
+    TaskMessage get(int index) {
+        return msgs.get(index);
+    }
+
     /**
      * try to add a TaskMessage to a batch
      * @param taskMsg

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java
index 3365e58..72c3cf7 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java
@@ -17,6 +17,9 @@
  */
 package backtype.storm.messaging.netty;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import backtype.storm.messaging.TaskMessage;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.channel.Channel;
@@ -34,52 +37,79 @@ public class MessageDecoder extends FrameDecoder {
      */
     protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buf) throws Exception {
         // Make sure that we have received at least a short 
-        if (buf.readableBytes() < 2) {
+        long available = buf.readableBytes();
+        if (available < 2) {
             //need more data
             return null;
         }
 
-        // Mark the current buffer position before reading task/len field
-        // because the whole frame might not be in the buffer yet.
-        // We will reset the buffer position to the marked position if
-        // there's not enough bytes in the buffer.
-        buf.markReaderIndex();
-
-        //read the short field
-        short code = buf.readShort();
-        
-        //case 1: Control message
-        ControlMessage ctrl_msg = ControlMessage.mkMessage(code);
-        if (ctrl_msg != null) return ctrl_msg;
-        
-        //case 2: task Message
-        short task = code;
-        
-        // Make sure that we have received at least an integer (length) 
-        if (buf.readableBytes() < 4) {
-            //need more data
-            buf.resetReaderIndex();
-            return null;
-        }
+        List<Object> ret = new ArrayList<Object>();
+
+        // Use while loop, try to decode as more messages as possible in single call
+        while (available >= 2) {
+
+            // Mark the current buffer position before reading task/len field
+            // because the whole frame might not be in the buffer yet.
+            // We will reset the buffer position to the marked position if
+            // there's not enough bytes in the buffer.
+            buf.markReaderIndex();
+
+            // read the short field
+            short code = buf.readShort();
+            available -= 2;
+
+            // case 1: Control message
+            ControlMessage ctrl_msg = ControlMessage.mkMessage(code);
+            if (ctrl_msg != null) {
+
+                if (ctrl_msg == ControlMessage.EOB_MESSAGE) {
+                    continue;
+                } else {
+                    return ctrl_msg;
+                }
+            }
+
+            // case 2: task Message
+            short task = code;
 
-        // Read the length field.
-        int length = buf.readInt();
-        if (length<=0) {
-            return new TaskMessage(task, null);
+            // Make sure that we have received at least an integer (length)
+            if (available < 4) {
+                // need more data
+                buf.resetReaderIndex();
+                break;
+            }
+
+            // Read the length field.
+            int length = buf.readInt();
+
+            available -= 4;
+
+            if (length <= 0) {
+                ret.add(new TaskMessage(task, null));
+                break;
+            }
+
+            // Make sure if there's enough bytes in the buffer.
+            if (available < length) {
+                // The whole bytes were not received yet - return null.
+                buf.resetReaderIndex();
+                break;
+            }
+            available -= length;
+
+            // There's enough bytes in the buffer. Read it.
+            ChannelBuffer payload = buf.readBytes(length);
+
+
+            // Successfully decoded a frame.
+            // Return a TaskMessage object
+            ret.add(new TaskMessage(task, payload.array()));
         }
-        
-        // Make sure if there's enough bytes in the buffer.
-        if (buf.readableBytes() < length) {
-            // The whole bytes were not received yet - return null.
-            buf.resetReaderIndex();
+
+        if (ret.size() == 0) {
             return null;
+        } else {
+            return ret;
         }
-
-        // There's enough bytes in the buffer. Read it.
-        ChannelBuffer payload = buf.readBytes(length);
-
-        // Successfully decoded a frame.
-        // Return a TaskMessage object
-        return new TaskMessage(task,payload.array());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
new file mode 100644
index 0000000..ea3f249
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
@@ -0,0 +1,35 @@
+package backtype.storm.messaging.netty;
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.jboss.netty.util.ThreadNameDeterminer;
+import org.jboss.netty.util.ThreadRenamingRunnable;
+
+public class NettyRenameThreadFactory  implements ThreadFactory {
+    
+    static {
+      //Rename Netty threads
+      ThreadRenamingRunnable.setThreadNameDeterminer(ThreadNameDeterminer.CURRENT);
+    }
+  
+    final ThreadGroup group;
+    final AtomicInteger index = new AtomicInteger(1);
+    final String name;
+
+    NettyRenameThreadFactory(String name) {
+        SecurityManager s = System.getSecurityManager();
+        group = (s != null)? s.getThreadGroup() :
+                             Thread.currentThread().getThreadGroup();
+        this.name = name;
+    }
+
+    public Thread newThread(Runnable r) {
+        Thread t = new Thread(group, r, name + "-" + index.getAndIncrement(), 0);
+        if (t.isDaemon())
+            t.setDaemon(false);
+        if (t.getPriority() != Thread.NORM_PRIORITY)
+            t.setPriority(Thread.NORM_PRIORITY);
+        return t;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java
index 83e4187..20a147d 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java
@@ -31,35 +31,69 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
 
 class Server implements IConnection {
     private static final Logger LOG = LoggerFactory.getLogger(Server.class);
     @SuppressWarnings("rawtypes")
     Map storm_conf;
     int port;
-    private LinkedBlockingQueue<TaskMessage> message_queue;
+    
+    // Create multiple queues for incoming messages. The size equals the number of receiver threads.
+    // For message which is sent to same task, it will be stored in the same queue to preserve the message order.
+    private LinkedBlockingQueue<ArrayList<TaskMessage>>[] message_queue;
+    
     volatile ChannelGroup allChannels = new DefaultChannelGroup("storm-server");
     final ChannelFactory factory;
     final ServerBootstrap bootstrap;
-
+    
+    private int queueCount;
+    HashMap<Integer, Integer> taskToQueueId = null;
+    int roundRobinQueueId;
+	
+    boolean closing = false;
+    List<TaskMessage> closeMessage = Arrays.asList(new TaskMessage(-1, null));
+    
+    
     @SuppressWarnings("rawtypes")
     Server(Map storm_conf, int port) {
         this.storm_conf = storm_conf;
         this.port = port;
-        message_queue = new LinkedBlockingQueue<TaskMessage>();
-
+        
+        queueCount = Utils.getInt(storm_conf.get(Config.WORKER_RECEIVER_THREAD_COUNT), 1);
+        roundRobinQueueId = 0;
+        taskToQueueId = new HashMap<Integer, Integer>();
+    
+        message_queue = new LinkedBlockingQueue[queueCount];
+        for (int i = 0; i < queueCount; i++) {
+            message_queue[i] = new LinkedBlockingQueue<ArrayList<TaskMessage>>();
+        }
+        
         // Configure the server.
         int buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
         int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS));
 
+        ThreadFactory bossFactory = new NettyRenameThreadFactory(name() + "-boss");
+        ThreadFactory workerFactory = new NettyRenameThreadFactory(name() + "-worker");
+        
         if (maxWorkers > 0) {
-            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), maxWorkers);
+            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), 
+                Executors.newCachedThreadPool(workerFactory), maxWorkers);
         } else {
-            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool());
+            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), 
+                Executors.newCachedThreadPool(workerFactory));
         }
+        
+        LOG.info("Create Netty Server " + name() + ", buffer_size: " + buffer_size + ", maxWorkers: " + maxWorkers);
+        
         bootstrap = new ServerBootstrap(factory);
         bootstrap.setOption("child.tcpNoDelay", true);
         bootstrap.setOption("child.receiveBufferSize", buffer_size);
@@ -72,36 +106,101 @@ class Server implements IConnection {
         Channel channel = bootstrap.bind(new InetSocketAddress(port));
         allChannels.add(channel);
     }
+    
+    private ArrayList<TaskMessage>[] groupMessages(List<TaskMessage> msgs) {
+      ArrayList<TaskMessage> messageGroups[] = new ArrayList[queueCount];
+      
+      for (int i = 0; i < msgs.size(); i++) {
+        TaskMessage message = msgs.get(i);
+        int task = message.task();
+        
+        if (task == -1) {
+          closing = true;
+          return null;
+        }
+        
+        Integer queueId = getMessageQueueId(task);
+        
+        if (null == messageGroups[queueId]) {
+          messageGroups[queueId] = new ArrayList<TaskMessage>();
+        }
+        messageGroups[queueId].add(message);
+      }
+      return messageGroups;
+    }
+    
+    private Integer getMessageQueueId(int task) {
+      // try to construct the map from taskId -> queueId in round robin manner.
+      
+      Integer queueId = taskToQueueId.get(task);
+      if (null == queueId) {
+        synchronized(taskToQueueId) {
+          //assgin task to queue in round-robin manner
+          if (null == taskToQueueId.get(task)) {
+            queueId = roundRobinQueueId++;
+            
+            taskToQueueId.put(task, queueId);
+            if (roundRobinQueueId == queueCount) {
+              roundRobinQueueId = 0;
+            }
+          }
+        }
+      }
+      return queueId;
+    }
 
     /**
      * enqueue a received message 
      * @param message
      * @throws InterruptedException
      */
-    protected void enqueue(TaskMessage message) throws InterruptedException {
-        message_queue.put(message);
-        LOG.debug("message received with task: {}, payload size: {}", message.task(), message.message().length);
+    protected void enqueue(List<TaskMessage> msgs) throws InterruptedException {
+      
+      if (null == msgs || msgs.size() == 0 || closing) {
+        return;
+      }
+      
+      ArrayList<TaskMessage> messageGroups[] = groupMessages(msgs);
+      
+      if (null == messageGroups || closing) {
+        return;
+      }
+      
+      for (int receiverId = 0; receiverId < messageGroups.length; receiverId++) {
+        ArrayList<TaskMessage> msgGroup = messageGroups[receiverId];
+        if (null != msgGroup) {
+          message_queue[receiverId].put(msgGroup);
+        }
+      }
     }
     
-    /**
-     * fetch a message from message queue synchronously (flags != 1) or asynchronously (flags==1)
-     */
-    public TaskMessage recv(int flags)  {
-        if ((flags & 0x01) == 0x01) { 
+    public Iterator<TaskMessage> recv(int flags, int receiverId)  {
+      if (closing) {
+        return closeMessage.iterator();
+      }
+      
+      ArrayList<TaskMessage> ret = null; 
+      int queueId = receiverId % queueCount;
+      if ((flags & 0x01) == 0x01) { 
             //non-blocking
-            return message_queue.poll();
+            ret = message_queue[queueId].poll();
         } else {
             try {
-                TaskMessage request = message_queue.take();
+                ArrayList<TaskMessage> request = message_queue[queueId].take();
                 LOG.debug("request to be processed: {}", request);
-                return request;
+                ret = request;
             } catch (InterruptedException e) {
                 LOG.info("exception within msg receiving", e);
-                return null;
+                ret = null;
             }
         }
+      
+      if (null != ret) {
+        return ret.iterator();
+      }
+      return null;
     }
-
+   
     /**
      * register a newly created channel
      * @param channel
@@ -133,4 +232,12 @@ class Server implements IConnection {
     public void send(int task, byte[] message) {
         throw new RuntimeException("Server connection should not send any messages");
     }
+    
+    public void send(Iterator<TaskMessage> msgs) {
+      throw new RuntimeException("Server connection should not send any messages");
+    }
+	
+    public String name() {
+      return "Netty-server-localhost-" + port;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientErrorHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientErrorHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientErrorHandler.java
new file mode 100644
index 0000000..ae317aa
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientErrorHandler.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import java.net.ConnectException;
+
+import org.jboss.netty.channel.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StormClientErrorHandler extends SimpleChannelUpstreamHandler  {
+    private static final Logger LOG = LoggerFactory.getLogger(StormClientErrorHandler.class);
+    private String name;
+    
+    StormClientErrorHandler(String name) {
+        this.name = name;
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) {
+        Throwable cause = event.getCause();
+        if (!(cause instanceof ConnectException)) {
+            LOG.info("Connection failed " + name, cause);
+        } 
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java
deleted file mode 100644
index 43a8c39..0000000
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package backtype.storm.messaging.netty;
-
-import java.net.ConnectException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelStateEvent;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.ConnectException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class StormClientHandler extends SimpleChannelUpstreamHandler  {
-    private static final Logger LOG = LoggerFactory.getLogger(StormClientHandler.class);
-    private Client client;
-    long start_time;
-    
-    StormClientHandler(Client client) {
-        this.client = client;
-        start_time = System.currentTimeMillis();
-    }
-
-    @Override
-    public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent event) {
-        //register the newly established channel
-        Channel channel = event.getChannel();
-        client.setChannel(channel);
-        LOG.info("connection established from "+channel.getLocalAddress()+" to "+channel.getRemoteAddress());
-        
-        //send next batch of requests if any
-        try {
-            client.tryDeliverMessages(false);
-        } catch (Exception ex) {
-            LOG.info("exception when sending messages:", ex.getMessage());
-            client.reconnect();
-        }
-    }
-
-    @Override
-    public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) {
-        LOG.debug("send/recv time (ms): {}", (System.currentTimeMillis() - start_time));
-        
-        //examine the response message from server
-        ControlMessage msg = (ControlMessage)event.getMessage();
-        if (msg==ControlMessage.FAILURE_RESPONSE)
-            LOG.info("failure response:{}", msg);
-
-        //send next batch of requests if any
-        try {
-            client.tryDeliverMessages(false);
-        } catch (Exception ex) {
-            LOG.info("exception when sending messages:", ex.getMessage());
-            client.reconnect();
-        }
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) {
-        Throwable cause = event.getCause();
-        if (!(cause instanceof ConnectException)) {
-            LOG.info("Connection to "+client.remote_addr+" failed:", cause);
-        }
-        client.reconnect();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java
index 6bad8e3..e6e8b3d 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java
@@ -37,7 +37,7 @@ class StormClientPipelineFactory implements ChannelPipelineFactory {
         // Encoder
         pipeline.addLast("encoder", new MessageEncoder());
         // business logic.
-        pipeline.addLast("handler", new StormClientHandler(client));
+        pipeline.addLast("handler", new StormClientErrorHandler(client.name()));
 
         return pipeline;
     }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java
index 093fb61..bf9b79e 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java
@@ -18,10 +18,14 @@
 package backtype.storm.messaging.netty;
 
 import backtype.storm.messaging.TaskMessage;
-import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
+import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
 class StormServerHandler extends SimpleChannelUpstreamHandler  {
@@ -41,30 +45,22 @@ class StormServerHandler extends SimpleChannelUpstreamHandler  {
     
     @Override
     public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
-        Object msg = e.getMessage();  
-        if (msg == null) return;
-
-        //end of batch?
-        if (msg==ControlMessage.EOB_MESSAGE) {
-            Channel channel = ctx.getChannel();
-            LOG.debug("Send back response ...");
-            if (failure_count.get()==0)
-                channel.write(ControlMessage.OK_RESPONSE);
-            else channel.write(ControlMessage.FAILURE_RESPONSE);
-            return;
-        }
-        
-        //enqueue the received message for processing
-        try {
-            server.enqueue((TaskMessage)msg);
-        } catch (InterruptedException e1) {
-            LOG.info("failed to enqueue a request message", e);
-            failure_count.incrementAndGet();
-        }
+      List<TaskMessage> msgs = (List<TaskMessage>) e.getMessage();
+      if (msgs == null) {
+        return;
+      }
+      
+      try {
+        server.enqueue(msgs);
+      } catch (InterruptedException e1) {
+        LOG.info("failed to enqueue a request message", e);
+        failure_count.incrementAndGet();
+      }
     }
 
     @Override
     public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
+        LOG.error("server errors in handling the request", e.getCause());
         server.closeChannel(e.getChannel());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/testing/TestEventLogSpout.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/TestEventLogSpout.java b/storm-core/src/jvm/backtype/storm/testing/TestEventLogSpout.java
new file mode 100644
index 0000000..1570aeb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/testing/TestEventLogSpout.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.testing;
+
+import static backtype.storm.utils.Utils.get;
+import backtype.storm.topology.OutputFieldsDeclarer;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+
+public class TestEventLogSpout extends BaseRichSpout {
+    public static Logger LOG = LoggerFactory.getLogger(TestEventLogSpout.class);
+    
+    private static final Map<String, Integer> acked = new HashMap<String, Integer>();
+    private static final Map<String, Integer> failed = new HashMap<String, Integer>();
+    
+    private String uid;
+    private long totalCount;
+    
+    SpoutOutputCollector _collector;
+    private long eventId = 0;
+    private long myCount;
+    private int source;
+    
+    public static int getNumAcked(String stormId) {
+        synchronized(acked) {
+            return get(acked, stormId, 0);
+        }
+    }
+
+    public static int getNumFailed(String stormId) {
+        synchronized(failed) {
+            return get(failed, stormId, 0);
+        }
+    }
+    
+    public TestEventLogSpout(long totalCount) {
+        this.uid = UUID.randomUUID().toString();
+        
+        synchronized(acked) {
+            acked.put(uid, 0);
+        }
+        synchronized(failed) {
+            failed.put(uid, 0);
+        }
+        
+        this.totalCount = totalCount;
+    }
+        
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        _collector = collector;
+        this.source = context.getThisTaskId();
+        long taskCount = context.getComponentTasks(context.getThisComponentId()).size();
+        myCount = totalCount / taskCount;
+    }
+    
+    public void close() {
+        
+    }
+    
+    public void cleanup() {
+        synchronized(acked) {            
+            acked.remove(uid);
+        } 
+        synchronized(failed) {            
+            failed.remove(uid);
+        }
+    }
+    
+    public boolean completed() {
+        
+        int ackedAmt;
+        int failedAmt;
+        
+        synchronized(acked) {
+            ackedAmt = acked.get(uid);
+        }
+        synchronized(failed) {
+            failedAmt = failed.get(uid);
+        }
+        int totalEmitted = ackedAmt + failedAmt;
+        
+        if (totalEmitted >= totalCount) {
+            return true;
+        }
+        return false;
+    }
+        
+    public void nextTuple() {
+        if (eventId < myCount) { 
+            eventId++;
+            _collector.emit(new Values(source, eventId), eventId);
+        }        
+    }
+    
+    public void ack(Object msgId) {
+        synchronized(acked) {
+            int curr = get(acked, uid, 0);
+            acked.put(uid, curr+1);
+        }
+    }
+
+    public void fail(Object msgId) {
+        synchronized(failed) {
+            int curr = get(failed, uid, 0);
+            failed.put(uid, curr+1);
+        }
+    }
+    
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("source", "eventId"));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/testing/TestEventOrderCheckBolt.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/TestEventOrderCheckBolt.java b/storm-core/src/jvm/backtype/storm/testing/TestEventOrderCheckBolt.java
new file mode 100644
index 0000000..1f80362
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/testing/TestEventOrderCheckBolt.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.testing;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+
+public class TestEventOrderCheckBolt extends BaseRichBolt {
+    public static Logger LOG = LoggerFactory.getLogger(TestEventOrderCheckBolt.class);
+    
+    private int _count;
+    OutputCollector _collector;
+    Map<Integer, Long> recentEventId = new HashMap<Integer, Long>();
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+        _count = 0;
+    }
+
+    public void execute(Tuple input) {
+        Integer sourceId = input.getInteger(0);
+        Long eventId = input.getLong(1);
+        Long recentEvent = recentEventId.get(sourceId);
+
+        if (null != recentEvent && eventId <= recentEvent) {
+            String error = "Error: event id is not in strict order! event source Id: "
+                    + sourceId + ", last event Id: " + recentEvent + ", current event Id: " + eventId;
+
+            _collector.emit(input, new Values(error));
+        }
+        recentEventId.put(sourceId, eventId);
+
+        _collector.ack(input);
+    }
+
+    public void cleanup() {
+
+    }
+
+    public Fields getOutputFields() {
+        return new Fields("error");
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("error"));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
index 56751c6..8c5b466 100644
--- a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
@@ -51,8 +51,11 @@ public class DisruptorQueue implements IStatefulObject {
     // TODO: consider having a threadlocal cache of this variable to speed up reads?
     volatile boolean consumerStartedFlag = false;
     ConcurrentLinkedQueue<Object> _cache = new ConcurrentLinkedQueue();
+    private static String PREFIX = "disruptor-";
+    private String _queueName = "";
     
-    public DisruptorQueue(ClaimStrategy claim, WaitStrategy wait) {
+    public DisruptorQueue(String queueName, ClaimStrategy claim, WaitStrategy wait) {
+         this._queueName = PREFIX + queueName;
         _buffer = new RingBuffer<MutableObject>(new ObjectEventFactory(), claim, wait);
         _consumer = new Sequence();
         _barrier = _buffer.newBarrier();
@@ -62,6 +65,10 @@ public class DisruptorQueue implements IStatefulObject {
         }
     }
     
+    public String getName() {
+      return _queueName;
+    }
+    
     public void consumeBatch(EventHandler<Object> handler) {
         consumeBatchToCursor(_barrier.getCursor(), handler);
     }

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java b/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
new file mode 100644
index 0000000..0e53632
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import backtype.storm.messaging.IConnection;
+import backtype.storm.messaging.TaskMessage;
+
+public class TransferDrainer {
+
+  private HashMap<String, ArrayList<ArrayList<TaskMessage>>> bundles = new HashMap();
+  
+  public void add(HashMap<String, ArrayList<TaskMessage>> workerTupleSetMap) {
+    for (String key : workerTupleSetMap.keySet()) {
+      
+      ArrayList<ArrayList<TaskMessage>> bundle = bundles.get(key);
+      if (null == bundle) {
+        bundle = new ArrayList<ArrayList<TaskMessage>>();
+        bundles.put(key, bundle);
+      }
+      
+      ArrayList tupleSet = workerTupleSetMap.get(key);
+      if (null != tupleSet && tupleSet.size() > 0) {
+        bundle.add(tupleSet);
+      }
+    } 
+  }
+  
+  public void send(HashMap<String, IConnection> connections) {
+    for (String hostPort : bundles.keySet()) {
+      IConnection connection = connections.get(hostPort);
+      if (null != connection) { 
+        ArrayList<ArrayList<TaskMessage>> bundle = bundles.get(hostPort);
+        Iterator<TaskMessage> iter = getBundleIterator(bundle);
+        if (null != iter && iter.hasNext()) {
+          connection.send(iter);
+        }
+      }
+    } 
+  }
+  
+  private Iterator<TaskMessage> getBundleIterator(final ArrayList<ArrayList<TaskMessage>> bundle) {
+    
+    if (null == bundle) {
+      return null;
+    }
+    
+    return new Iterator<TaskMessage> () {
+      
+      private int offset = 0;
+      private int size = 0;
+      {
+        for (ArrayList<TaskMessage> list : bundle) {
+            size += list.size();
+        }
+      }
+      
+      private int bundleOffset = 0;
+      private Iterator<TaskMessage> iter = bundle.get(bundleOffset).iterator();
+      
+      @Override
+      public boolean hasNext() {
+        if (offset < size) {
+          return true;
+        }
+        return false;
+      }
+
+      @Override
+      public TaskMessage next() {
+        TaskMessage msg = null;
+        if (iter.hasNext()) {
+          msg = iter.next(); 
+        } else {
+          bundleOffset++;
+          iter = bundle.get(bundleOffset).iterator();
+          msg = iter.next();
+        }
+        if (null != msg) {
+          offset++;
+        }
+        return msg;
+      }
+
+      @Override
+      public void remove() {
+        throw new RuntimeException("not supported");
+      }
+    };
+  }
+  
+  public void clear() {
+    bundles.clear();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/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 a1fed96..6a0a447 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -25,6 +25,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
+import java.io.PrintStream;
 import java.net.URL;
 import java.net.URLDecoder;
 import java.nio.ByteBuffer;
@@ -37,6 +38,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Scanner;
 import java.util.TreeMap;
 import java.util.UUID;
 
@@ -301,15 +303,39 @@ public class Utils {
     }
     
     public static Integer getInt(Object o) {
-        if(o instanceof Long) {
-            return ((Long) o ).intValue();
-        } else if (o instanceof Integer) {
-            return (Integer) o;
-        } else if (o instanceof Short) {
-            return ((Short) o).intValue();
-        } else {
-            throw new IllegalArgumentException("Don't know how to convert " + o + " + to int");
-        }
+      Integer result = getInt(o, null);
+      if (null == result) {
+        throw new IllegalArgumentException("Don't know how to convert null + to int");
+      }
+      return result;
+    }
+    
+    public static Integer getInt(Object o, Integer defaultValue) {
+      if (null == o) {
+        return defaultValue;
+      }
+      
+      if(o instanceof Long) {
+          return ((Long) o ).intValue();
+      } else if (o instanceof Integer) {
+          return (Integer) o;
+      } else if (o instanceof Short) {
+          return ((Short) o).intValue();
+      } else {
+          throw new IllegalArgumentException("Don't know how to convert " + o + " + to int");
+      }
+    }
+
+    public static boolean getBoolean(Object o, boolean defaultValue) {
+      if (null == o) {
+        return defaultValue;
+      }
+      
+      if(o instanceof Boolean) {
+          return (Boolean) o;
+      } else {
+          throw new IllegalArgumentException("Don't know how to convert " + o + " + to boolean");
+      }
     }
     
     public static long secureRandomLong() {
@@ -373,7 +399,7 @@ public class Utils {
         ret.start();
         return ret;
     }
-
+    
     public static CuratorFramework newCuratorStarted(Map conf, List<String> servers, Object port) {
         CuratorFramework ret = newCurator(conf, servers, port);
         ret.start();

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/component.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html
new file mode 100644
index 0000000..90ca630
--- /dev/null
+++ b/storm-core/src/ui/public/component.html
@@ -0,0 +1,88 @@
+<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01//EN" "http://www.w3.org/TR/html4/strict.dtd">
+<html><head>
+<title>Storm UI</title>
+<link href="/css/bootstrap-1.4.0.css" rel="stylesheet" type="text/css">
+<link href="/css/style.css" rel="stylesheet" type="text/css">
+<script src="/js/jquery-1.6.2.min.js" type="text/javascript"></script>
+<script src="/js/jquery.tablesorter.min.js" type="text/javascript"></script>
+<script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
+<script src="/js/jquery.mustache.js" type="text/javascript"></script>
+<script src="/js/purl.js" type="text/javascript"></script>
+<script src="/js/bootstrap-twipsy.js" type="text/javascript"></script>
+<script src="/js/script.js" type="text/javascript"></script>
+</head>
+<body>
+<h1><a href="/">Storm UI</a></h1>
+<div id="component-summary">
+</div>
+<div id="component-stats-detail">
+</div>
+<div id="component-input-stats">
+</div>
+<div id="component-output-stats">
+</div>
+<div id="component-executor-stats">
+</div>
+<div id="component-errors">
+</div>
+<div id="json-response-error">
+</div>
+<p id="toggle-switch" style="display: block;" class="js-only"></p>
+<script>
+$(document).ready(function() {
+    var componentId = $.url().param("id");
+    var topologyId = $.url().param("topology_id");
+    var window = $.url().param("window");
+    var sys = $.cookies.get("sys") || "false";
+    var url = "/api/v1/topology/"+topologyId+"/component/"+componentId+"?sys="+sys;
+    if(window) url += "&window="+window;
+    renderToggleSys($("#toggle-switch"));
+    $.ajaxSetup({
+        "error":function(jqXHR,textStatus,response) {
+            var errorJson = jQuery.parseJSON(jqXHR.responseText);
+            $.get("/templates/json-error-template.html", function(template) {
+                $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),errorJson));
+            });
+        }
+    });
+
+    $.getJSON(url,function(response,status,jqXHR) {
+        var componentSummary = $("#component-summary");
+        var componentStatsDetail = $("#component-stats-detail")
+        var inputStats = $("#component-input-stats");
+        var outputStats = $("#component-output-stats");
+        var executorStats = $("#component-executor-stats");
+        var componentErrors = $("#component-errors");
+        $.get("/templates/component-page-template.html", function(template) {
+            componentSummary.append(Mustache.render($(template).filter("#component-summary-template").html(),response));
+            if(response["componentType"] == "spout") {
+                componentStatsDetail.append(Mustache.render($(template).filter("#spout-stats-detail-template").html(),response));
+                $("#spout-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}});
+                outputStats.append(Mustache.render($(template).filter("#output-stats-template").html(),response));
+                $("#output-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}});
+                executorStats.append(Mustache.render($(template).filter("#executor-stats-template").html(),response));
+                $("#executor-stats-table").tablesorter({ sortList: [[0,0]], headers: {1: { sorter: "stormtimestr"}}});
+            } else {
+                componentStatsDetail.append(Mustache.render($(template).filter("#bolt-stats-template").html(),response));
+                $("#bolt-stats-table").tablesorter({ sortList: [[0,0]], headers: {0: { sorter: "stormtimestr"}}});
+                inputStats.append(Mustache.render($(template).filter("#bolt-input-stats-template").html(),response));
+                if (response["inputStats"].length > 0) {
+                    $("#bolt-input-stats-table").tablesorter({ sortList: [[0,0]], headers: {}});
+                }
+                outputStats.append(Mustache.render($(template).filter("#bolt-output-stats-template").html(),response));
+                $("#bolt-output-stats-table").tablesorter({ sortList: [[0,0]], headers: {}});
+                executorStats.append(Mustache.render($(template).filter("#bolt-executor-template").html(),response));
+                if(response["outputStats"].length > 0) {
+                    $("#bolt-executor-table").tablesorter({ sortList: [[0,0]], headers: {}});
+                }
+            }
+            componentErrors.append(Mustache.render($(template).filter("#component-errors-template").html(),response));
+            if(response["componentErrors"].length > 0) {
+                $("#component-errors-table").tablesorter({ sortList: [[0,0]], headers: {1: { sorter: "stormtimestr"}}});
+            }
+        });
+    });
+});
+</script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/55e1664d/storm-core/src/ui/public/index.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/index.html b/storm-core/src/ui/public/index.html
new file mode 100644
index 0000000..77af159
--- /dev/null
+++ b/storm-core/src/ui/public/index.html
@@ -0,0 +1,73 @@
+<html><head>
+<title>Storm UI</title>
+<link href="/css/bootstrap-1.4.0.css" rel="stylesheet" type="text/css">
+<link href="/css/style.css" rel="stylesheet" type="text/css">
+<script src="/js/jquery-1.6.2.min.js" type="text/javascript"></script>
+<script src="/js/jquery.tablesorter.min.js" type="text/javascript"></script>
+<script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
+<script src="/js/jquery.mustache.js" type="text/javascript"></script>
+<script src="/js/bootstrap-twipsy.js" type="text/javascript"></script>
+<script src="/js/script.js" type="text/javascript"></script>
+</head>
+<body>
+<h1><a href="/">Storm UI</a></h1>
+<h2>Cluster Summary</h2>
+<div id="cluster-summary">
+</div>
+<h2>Topology summary</h2>
+<div id="topology-summary">
+</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>
+$(document).ready(function() {
+    $.ajaxSetup({
+        "error":function(jqXHR,textStatus,response) {
+            var errorJson = jQuery.parseJSON(jqXHR.responseText);
+            $.get("/templates/json-error-template.html", function(template) {
+                $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),errorJson));
+            });
+        }
+    });
+    var template = $.get("/templates/index-page-template.html");
+    var clusterSummary = $("#cluster-summary");
+    var topologySummary = $("#topology-summary");
+    var supervisorSummary = $("#supervisor-summary");
+    var config = $("#nimbus-configuration");
+
+    $.getJSON("/api/v1/cluster/summary",function(response,status,jqXHR) {
+        $.get("/templates/index-page-template.html", function(template) {
+            clusterSummary.append(Mustache.render($(template).filter("#cluster-summary-template").html(),response));
+        });
+    });
+    $.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));
+          if(response["topologies"].length > 0) {
+              $("#topology-summary-table").tablesorter({ sortList: [[0,0]], headers: {3: { sorter: "stormtimestr"}}});
+              }
+      });
+    });
+    $.getJSON("/api/v1/supervisor/summary",function(response,status,jqXHR) {
+      $.get("/templates/index-page-template.html", function(template) {
+          supervisorSummary.append(Mustache.render($(template).filter("#supervisor-summary-template").html(),response));
+          if(response["supervisors"].length > 0) {
+              $("#supervisor-summary-table").tablesorter({ sortList: [[0,0]], headers: {3: { sorter: "stormtimestr"}}});
+          }
+      });
+    });
+    $.getJSON("/api/v1/cluster/configuration",function(response,status,jqXHR) {
+      var formattedResponse = formatConfigData(response);
+      $.get("/templates/index-page-template.html", function(template) {
+          config.append(Mustache.render($(template).filter("#configuration-template").html(),formattedResponse));
+          $("#nimbus-configuration-table").tablesorter({ sortList: [[0,0]], headers: {}});
+      });
+    });
+  });
+</script>
+</html>


[11/13] git commit: add Clojure style links to DEVELOPER.md

Posted by bo...@apache.org.
add Clojure style links to DEVELOPER.md

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

Branch: refs/heads/master
Commit: 2456f0aa8c3834ab562f32d645e2835402075ff2
Parents: 55e1664
Author: David James <da...@bluemontlabs.com>
Authored: Mon Jun 9 21:07:06 2014 -0700
Committer: David James <da...@bluemontlabs.com>
Committed: Mon Jun 9 21:08:02 2014 -0700

----------------------------------------------------------------------
 DEVELOPER.md | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/2456f0aa/DEVELOPER.md
----------------------------------------------------------------------
diff --git a/DEVELOPER.md b/DEVELOPER.md
index 0c8de00..7d1c09c 100644
--- a/DEVELOPER.md
+++ b/DEVELOPER.md
@@ -92,6 +92,14 @@ and
 [trivial issues](https://issues.apache.org/jira/secure/IssueNavigator.jspa?reset=true&jqlQuery=project+%3D+STORM+AND+resolution+%3D+Unresolved+AND+priority+%3D+Trivial+ORDER+BY+key+DESC&mode=hide))
 because they require learning about only an isolated portion of the codebase and are a relatively small amount of work.
 
+Please use idiomatic Clojure style, as explained in [this Clojure style guide][clj-SG]. Another useful reference is
+the [Clojure Library Coding Standards][clj-LCS]. Perhaps the most important is consistenly writing a clear docstring
+for functions, explaining the return value and arguments. As of this writing, the Storm codebase would benefit from
+various style improvements.
+
+[clj-SG]: https://github.com/bbatsov/clojure-style-guide
+[clj-LCS]: http://dev.clojure.org/display/community/Library+Coding+Standards
+
 Contributions to the Storm codebase should be sent as GitHub pull requests.  See section _Create a pull request_ below
 for details.  If there is any problem with the pull request we can iterate on it using the commenting features of
 GitHub.


[13/13] git commit: STORM-388 updated Changelog

Posted by bo...@apache.org.
STORM-388 updated Changelog


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

Branch: refs/heads/master
Commit: 0826b93323ccc33b97b45dc82dae2b7c10bfab5c
Parents: 862f673
Author: Robert (Bobby) Evans <bo...@apache.org>
Authored: Thu Jun 12 14:53:45 2014 -0500
Committer: Robert (Bobby) Evans <bo...@apache.org>
Committed: Thu Jun 12 14:53:45 2014 -0500

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


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/0826b933/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index fb1062c..ee2eae5 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,5 @@
 ## 0.9.3-incubating (unreleased)
- * STORM-338 Move towards idiomatic Clojure style 
+ * STORM-338: Move towards idiomatic Clojure style 
 
 ## 0.9.2-incubating
  * STORM-335: add drpc test for removing timed out requests from queue


[05/13] git commit: remove unintended :refer-clojure

Posted by bo...@apache.org.
remove unintended :refer-clojure

(This is meant for a separate pull request)

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

Branch: refs/heads/master
Commit: 3352dc298743eaffc7ade85aa175a1ef53ff3b7e
Parents: 2278fc9
Author: David James <da...@bluemontlabs.com>
Authored: Sat May 31 14:17:23 2014 -0400
Committer: David James <da...@bluemontlabs.com>
Committed: Sat May 31 14:17:23 2014 -0400

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/util.clj | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/3352dc29/storm-core/src/clj/backtype/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index 6e78cca..891d8eb 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -36,8 +36,7 @@
   (:require [clojure [set :as set]])
   (:require [clojure.java.io :as io])
   (:use [clojure walk])
-  (:use [backtype.storm log])
-  (:refer-clojure :exclude [some?]))
+  (:use [backtype.storm log]))
 
 (defn wrap-in-runtime
   "Wraps an exception in a RuntimeException if needed"


[12/13] git commit: Merge branch 'idiomatic-clojure-01' of https://github.com/bluemont/incubator-storm into STORM-338

Posted by bo...@apache.org.
Merge branch 'idiomatic-clojure-01' of https://github.com/bluemont/incubator-storm into STORM-338

STORM-338: Move towards idiomatic Clojure style

Conflicts:
	CHANGELOG.md
	storm-core/src/clj/backtype/storm/daemon/drpc.clj
	storm-core/src/clj/backtype/storm/disruptor.clj
	storm-core/src/clj/backtype/storm/timer.clj
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
	storm-core/src/ui/public/component.html
	storm-core/src/ui/public/index.html
	storm-core/src/ui/public/js/visualization.js
	storm-core/src/ui/public/templates/component-page-template.html
	storm-core/src/ui/public/templates/index-page-template.html
	storm-core/src/ui/public/templates/json-error-template.html
	storm-core/src/ui/public/templates/topology-page-template.html
	storm-core/src/ui/public/topology.html


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

Branch: refs/heads/master
Commit: 862f6735f2e4cf9734b395385609bec8f0b6041f
Parents: 2a60e99 2456f0a
Author: Robert (Bobby) Evans <bo...@apache.org>
Authored: Thu Jun 12 14:53:13 2014 -0500
Committer: Robert (Bobby) Evans <bo...@apache.org>
Committed: Thu Jun 12 14:53:13 2014 -0500

----------------------------------------------------------------------
 CHANGELOG.md                                    |   3 +
 DEVELOPER.md                                    |   8 +
 .../src/clj/backtype/storm/LocalCluster.clj     |  71 +-
 storm-core/src/clj/backtype/storm/LocalDRPC.clj |   5 +-
 storm-core/src/clj/backtype/storm/bootstrap.clj |   9 +-
 storm-core/src/clj/backtype/storm/clojure.clj   |  10 +-
 storm-core/src/clj/backtype/storm/cluster.clj   | 427 +++++------
 storm-core/src/clj/backtype/storm/config.clj    | 145 ++--
 .../src/clj/backtype/storm/daemon/drpc.clj      | 135 ++--
 storm-core/src/clj/backtype/storm/disruptor.clj |  80 +--
 storm-core/src/clj/backtype/storm/event.clj     |  49 +-
 storm-core/src/clj/backtype/storm/log.clj       |  22 +-
 .../clj/backtype/storm/process_simulator.clj    |  27 +-
 storm-core/src/clj/backtype/storm/stats.clj     | 289 ++++----
 storm-core/src/clj/backtype/storm/testing.clj   | 523 +++++++-------
 storm-core/src/clj/backtype/storm/testing4j.clj |  58 +-
 storm-core/src/clj/backtype/storm/thrift.clj    | 255 ++++---
 storm-core/src/clj/backtype/storm/timer.clj     | 106 +--
 storm-core/src/clj/backtype/storm/tuple.clj     |   7 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   | 559 ++++++++-------
 storm-core/src/clj/backtype/storm/util.clj      | 717 ++++++++++---------
 storm-core/src/clj/backtype/storm/zookeeper.clj | 158 ++--
 .../netty/NettyRenameThreadFactory.java         |   2 +-
 23 files changed, 1947 insertions(+), 1718 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/862f6735/CHANGELOG.md
----------------------------------------------------------------------
diff --cc CHANGELOG.md
index 1666729,0fa35e7..fb1062c
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@@ -1,4 -1,4 +1,7 @@@
 -## 0.9.2-incubating (unreleased)
++## 0.9.3-incubating (unreleased)
++ * STORM-338 Move towards idiomatic Clojure style 
++
 +## 0.9.2-incubating
   * STORM-335: add drpc test for removing timed out requests from queue
   * STORM-69: Storm UI Visualizations for Topologies
   * STORM-297: Performance scaling with CPU

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/862f6735/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index 5f2bcba,d8ad29a..68bc4c0
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -57,7 -62,13 +62,10 @@@
        (trim (slurp release-path))
        "Unknown")))
  
- (defn component-type [^StormTopology topology id]
 -;; TODO: What is the desired behavior if a particular component id
 -;; corresponded to both a bolt and spout. Where should the error
 -;; occur? -DCJ
+ (defn component-type
+   "Returns the component type (either :bolt or :spout) for a given
+   topology and component id. Returns nil if not found."
+   [^StormTopology topology id]
    (let [bolts (.get_bolts topology)
          spouts (.get_spouts topology)]
      (cond

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/862f6735/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
index 148bcfa,ea3f249..3a91a58
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
@@@ -49,4 -32,4 +49,4 @@@ public class NettyRenameThreadFactory  
              t.setPriority(Thread.NORM_PRIORITY);
          return t;
      }
--}
++}