You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2015/10/09 16:53:29 UTC

[1/9] storm git commit: [STORM-893] - Resource Aware Scheduler implementation. [STORM-894] - Basic Resource Aware Scheduling implementation.

Repository: storm
Updated Branches:
  refs/heads/master 3f79c2f40 -> 86f2d03c2


http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
new file mode 100644
index 0000000..7f9b247
--- /dev/null
+++ b/storm-core/test/clj/backtype/storm/scheduler/resource_aware_scheduler_test.clj
@@ -0,0 +1,669 @@
+;; Licensed to the Apache Software Foundation (ASF) under one
+;; or more contributor license agreements.  See the NOTICE file
+;; distributed with this work for additional information
+;; regarding copyright ownership.  The ASF licenses this file
+;; to you under the Apache License, Version 2.0 (the
+;; "License"); you may not use this file except in compliance
+;; with the License.  You may obtain a copy of the License at
+;;
+;; http://www.apache.org/licenses/LICENSE-2.0
+;;
+;; Unless required by applicable law or agreed to in writing, software
+;; distributed under the License is distributed on an "AS IS" BASIS,
+;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+;; See the License for the specific language governing permissions and
+;; limitations under the License.
+(ns backtype.storm.scheduler.resource-aware-scheduler-test
+  (:use [clojure test])
+  (:use [backtype.storm config testing thrift])
+  (:require [backtype.storm.util :refer [map-val reverse-map sum]])
+  (:require [backtype.storm.daemon [nimbus :as nimbus]])
+  (:import [backtype.storm.generated StormTopology]
+           [backtype.storm Config]
+           [backtype.storm.testing TestWordSpout TestWordCounter]
+           [backtype.storm.topology TopologyBuilder])
+  (:import [backtype.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails
+            SchedulerAssignmentImpl Topologies TopologyDetails])
+  (:import [backtype.storm.scheduler.resource RAS_Node ResourceAwareScheduler])
+  (:import [backtype.storm Config StormSubmitter])
+  (:import [backtype.storm LocalDRPC LocalCluster])
+  (:import [java.util HashMap]))
+
+(defn gen-supervisors [count ports]
+  (into {} (for [id (range count)
+                :let [supervisor (SupervisorDetails. (str "id" id)
+                                       (str "host" id)
+                                       (list ) (map int (range ports))
+                                   {Config/SUPERVISOR_MEMORY_CAPACITY_MB 2000.0
+                                    Config/SUPERVISOR_CPU_CAPACITY 400.0})]]
+            {(.getId supervisor) supervisor})))
+
+(defn to-top-map [topologies]
+  (into {} (for [top topologies] {(.getId top) top})))
+
+(defn ed [id] (ExecutorDetails. (int id) (int id)))
+
+(defn mk-ed-map [arg]
+  (into {}
+    (for [[name start end] arg]
+      (into {}
+        (for [at (range start end)]
+          {(ed at) name})))))
+
+;; get the super->mem HashMap by counting the eds' mem usage of all topos on each super
+(defn get-super->mem-usage [^Cluster cluster ^Topologies topologies]
+  (let [assignments (.values (.getAssignments cluster))
+        supers (.values (.getSupervisors cluster))
+        super->mem-usage (HashMap.)
+        _ (doseq [super supers] 
+             (.put super->mem-usage super 0))]  ;; initialize the mem-usage as 0 for all supers
+    (doseq [assignment assignments]
+      (let [ed->super (into {}
+                            (for [[ed slot] (.getExecutorToSlot assignment)]
+                              {ed (.getSupervisorById cluster (.getNodeId slot))}))
+            super->eds (reverse-map ed->super)
+            topology (.getById topologies (.getTopologyId assignment))
+            super->mem-pertopo (map-val (fn [eds] 
+                                          (reduce + (map #(.getTotalMemReqTask topology %) eds))) 
+                                        super->eds)]  ;; sum up the one topo's eds' mem usage on a super 
+            (doseq [[super mem] super->mem-pertopo]
+              (.put super->mem-usage 
+                    super (+ mem (.get super->mem-usage super)))))) ;; add all topo's mem usage for each super
+    super->mem-usage))
+
+;; get the super->cpu HashMap by counting the eds' cpu usage of all topos on each super
+(defn get-super->cpu-usage [^Cluster cluster ^Topologies topologies]
+  (let [assignments (.values (.getAssignments cluster))
+        supers (.values (.getSupervisors cluster))
+        super->cpu-usage (HashMap.)
+        _ (doseq [super supers] 
+             (.put super->cpu-usage super 0))] ;; initialize the cpu-usage as 0 for all supers
+    (doseq [assignment assignments]
+      (let [ed->super (into {}
+                            (for [[ed slot] (.getExecutorToSlot assignment)]
+                              {ed (.getSupervisorById cluster (.getNodeId slot))}))
+            super->eds (reverse-map ed->super)
+            topology (.getById topologies (.getTopologyId assignment))
+            super->cpu-pertopo (map-val (fn [eds] 
+                                          (reduce + (map #(.getTotalCpuReqTask topology %) eds))) 
+                                        super->eds)] ;; sum up the one topo's eds' cpu usage on a super 
+            (doseq [[super cpu] super->cpu-pertopo]
+              (.put super->cpu-usage 
+                    super (+ cpu (.get super->cpu-usage super))))))  ;; add all topo's cpu usage for each super
+    super->cpu-usage))
+
+;; testing resource/Node class
+(deftest test-node
+  (let [supers (gen-supervisors 5 4)
+        cluster (Cluster. (nimbus/standalone-nimbus) supers {} {})
+        topologies (Topologies. (to-top-map []))
+        node-map (RAS_Node/getAllNodesFrom cluster topologies)
+        topology1 (TopologyDetails. "topology1" {} nil 0)
+        topology2 (TopologyDetails. "topology2" {} nil 0)]
+    (is (= 5 (.size node-map)))
+    (let [node (.get node-map "id0")]
+      (is (= "id0" (.getId node)))
+      (is (= true (.isAlive node)))
+      (is (= 0 (.size (.getRunningTopologies node))))
+      (is (= true (.isTotallyFree node)))
+      (is (= 4 (.totalSlotsFree node)))
+      (is (= 0 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node topology1 (list (ExecutorDetails. 1 1)) cluster)
+      (is (= 1 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 3 (.totalSlotsFree node)))
+      (is (= 1 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node topology1 (list (ExecutorDetails. 2 2)) cluster)
+      (is (= 1 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 2 (.totalSlotsFree node)))
+      (is (= 2 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node topology2 (list (ExecutorDetails. 1 1)) cluster)
+      (is (= 2 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 1 (.totalSlotsFree node)))
+      (is (= 3 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.assign node topology2 (list (ExecutorDetails. 2 2)) cluster)
+      (is (= 2 (.size (.getRunningTopologies node))))
+      (is (= false (.isTotallyFree node)))
+      (is (= 0 (.totalSlotsFree node)))
+      (is (= 4 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+      (.freeAllSlots node cluster)
+      (is (= 0 (.size (.getRunningTopologies node))))
+      (is (= true (.isTotallyFree node)))
+      (is (= 4 (.totalSlotsFree node)))
+      (is (= 0 (.totalSlotsUsed node)))
+      (is (= 4 (.totalSlots node)))
+    )))
+
+(deftest test-sanity-resource-aware-scheduler
+  (let [builder (TopologyBuilder.)
+        _ (.setSpout builder "wordSpout" (TestWordSpout.) 1)
+        _ (.shuffleGrouping (.setBolt builder "wordCountBolt" (TestWordCounter.) 1) "wordSpout")
+        supers (gen-supervisors 1 2)
+        storm-topology (.createTopology builder)
+        topology1 (TopologyDetails. "topology1"
+                    {TOPOLOGY-NAME "topology-name-1"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology
+                    1
+                    (mk-ed-map [["wordSpout" 0 1]
+                                ["wordCountBolt" 1 2]]))
+        cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                  {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                   "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+        topologies (Topologies. (to-top-map [topology1]))
+        node-map (RAS_Node/getAllNodesFrom cluster topologies)
+        scheduler (ResourceAwareScheduler.)]
+    (.schedule scheduler topologies cluster)
+    (let [assignment (.getAssignmentById cluster "topology1")
+          assigned-slots (.getSlots assignment)
+          executors (.getExecutors assignment)]
+      (is (= 1 (.size assigned-slots)))
+      (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+      (is (= 2 (.size executors))))
+    (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
+
+(deftest test-topology-with-multiple-spouts
+  (let [builder1 (TopologyBuilder.)  ;; a topology with multiple spouts
+        _ (.setSpout builder1 "wordSpout1" (TestWordSpout.) 1)
+        _ (.setSpout builder1 "wordSpout2" (TestWordSpout.) 1)
+        _ (doto
+            (.setBolt builder1 "wordCountBolt1" (TestWordCounter.) 1)
+            (.shuffleGrouping "wordSpout1")
+            (.shuffleGrouping "wordSpout2"))
+        _ (.shuffleGrouping (.setBolt builder1 "wordCountBolt2" (TestWordCounter.) 1) "wordCountBolt1")
+        _ (.shuffleGrouping (.setBolt builder1 "wordCountBolt3" (TestWordCounter.) 1) "wordCountBolt1")
+        _ (.shuffleGrouping (.setBolt builder1 "wordCountBolt4" (TestWordCounter.) 1) "wordCountBolt2")
+        _ (.shuffleGrouping (.setBolt builder1 "wordCountBolt5" (TestWordCounter.) 1) "wordSpout2")
+        storm-topology1 (.createTopology builder1)
+        topology1 (TopologyDetails. "topology1"
+                    {TOPOLOGY-NAME "topology-name-1"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology1
+                    1
+                    (mk-ed-map [["wordSpout1" 0 1]
+                                ["wordSpout2" 1 2]
+                                ["wordCountBolt1" 2 3]
+                                ["wordCountBolt2" 3 4]
+                                ["wordCountBolt3" 4 5]
+                                ["wordCountBolt4" 5 6]
+                                ["wordCountBolt5" 6 7]]))
+        builder2 (TopologyBuilder.)  ;; a topology with two unconnected partitions
+        _ (.setSpout builder2 "wordSpoutX" (TestWordSpout.) 1)
+        _ (.setSpout builder2 "wordSpoutY" (TestWordSpout.) 1)
+        storm-topology2 (.createTopology builder1)
+        topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology2
+                    1
+                    (mk-ed-map [["wordSpoutX" 0 1]
+                                ["wordSpoutY" 1 2]]))
+        supers (gen-supervisors 2 4)
+        cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                  {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                   "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+        topologies (Topologies. (to-top-map [topology1 topology2]))
+        scheduler (ResourceAwareScheduler.)]
+    (.schedule scheduler topologies cluster)
+    (let [assignment (.getAssignmentById cluster "topology1")
+          assigned-slots (.getSlots assignment)
+          executors (.getExecutors assignment)]
+      (is (= 1 (.size assigned-slots)))
+      (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+      (is (= 7 (.size executors))))
+    (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+    (let [assignment (.getAssignmentById cluster "topology2")
+          assigned-slots (.getSlots assignment)
+          executors (.getExecutors assignment)]
+      (is (= 1 (.size assigned-slots)))
+      (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+      (is (= 2 (.size executors))))
+    (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+
+(deftest test-topology-set-memory-and-cpu-load
+  (let [builder (TopologyBuilder.)
+        _ (.setSpout builder "wordSpout" (TestWordSpout.) 1)
+        _ (doto
+            (.setBolt builder "wordCountBolt" (TestWordCounter.) 1)
+            (.setMemoryLoad 110.0)
+            (.setCPULoad 20.0)
+            (.shuffleGrouping "wordSpout"))
+        supers (gen-supervisors 2 2)  ;; to test whether two tasks will be assigned to one or two nodes
+        storm-topology (.createTopology builder)
+        topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology
+                    2
+                    (mk-ed-map [["wordSpout" 0 1]
+                                ["wordCountBolt" 1 2]]))
+        cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                  {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                   "backtype.storm.testing.AlternateRackDNSToSwitchMapping"})
+        topologies (Topologies. (to-top-map [topology2]))
+        scheduler (ResourceAwareScheduler.)]
+    (.schedule scheduler topologies cluster)
+    (let [assignment (.getAssignmentById cluster "topology2")
+          assigned-slots (.getSlots assignment)
+          executors (.getExecutors assignment)]
+      ;; 4 slots on 1 machine, all executors assigned
+      (is (= 1 (.size assigned-slots)))
+      (is (= 1 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+      (is (= 2 (.size executors))))
+    (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+
+(deftest test-resource-limitation
+  (let [builder (TopologyBuilder.)
+        _ (doto (.setSpout builder "wordSpout" (TestWordSpout.) 2)
+            (.setMemoryLoad 1000.0 200.0)
+            (.setCPULoad 250.0))
+        _ (doto (.setBolt builder "wordCountBolt" (TestWordCounter.) 1)
+            (.shuffleGrouping  "wordSpout")
+            (.setMemoryLoad 500.0 100.0)
+            (.setCPULoad 100.0))
+        supers (gen-supervisors 2 2)  ;; need at least two nodes to hold these executors
+        storm-topology (.createTopology builder)
+        topology1 (TopologyDetails. "topology1"
+                    {TOPOLOGY-NAME "topology-name-1"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology
+                    2 ;; need two workers, each on one node
+                    (mk-ed-map [["wordSpout" 0 2]
+                                ["wordCountBolt" 2 3]]))
+        cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                  {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                   "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+        topologies (Topologies. (to-top-map [topology1]))
+        scheduler (ResourceAwareScheduler.)]
+    (.schedule scheduler topologies cluster)
+    (let [assignment (.getAssignmentById cluster "topology1")
+          assigned-slots (.getSlots assignment)
+          node-ids (map #(.getNodeId %) assigned-slots)
+          executors (.getExecutors assignment)
+          epsilon 0.000001
+          assigned-ed-mem (sort (map #(.getTotalMemReqTask topology1 %) executors))
+          assigned-ed-cpu (sort (map #(.getTotalCpuReqTask topology1 %) executors))
+          ed->super (into {}
+                            (for [[ed slot] (.getExecutorToSlot assignment)]
+                              {ed (.getSupervisorById cluster (.getNodeId slot))}))
+          super->eds (reverse-map ed->super)
+          mem-avail->used (into []
+                                 (for [[super eds] super->eds]
+                                   [(.getTotalMemory super) (sum (map #(.getTotalMemReqTask topology1 %) eds))]))
+          cpu-avail->used (into []
+                                 (for [[super eds] super->eds]
+                                   [(.getTotalCPU super) (sum (map #(.getTotalCpuReqTask topology1 %) eds))]))]
+    ;; 4 slots on 1 machine, all executors assigned
+    (is (= 2 (.size assigned-slots)))  ;; executor0 resides one one worker (on one), executor1 and executor2 on another worker (on the other node)
+    (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot))))))
+    (is (= 3 (.size executors)))
+    ;; make sure resource (mem/cpu) assigned equals to resource specified
+    (is (< (Math/abs (- 600.0 (first assigned-ed-mem))) epsilon))
+    (is (< (Math/abs (- 1200.0 (second assigned-ed-mem))) epsilon))
+    (is (< (Math/abs (- 1200.0 (last assigned-ed-mem))) epsilon))
+    (is (< (Math/abs (- 100.0 (first assigned-ed-cpu))) epsilon))
+    (is (< (Math/abs (- 250.0 (second assigned-ed-cpu))) epsilon))
+    (is (< (Math/abs (- 250.0 (last assigned-ed-cpu))) epsilon))
+    (doseq [[avail used] mem-avail->used] ;; for each node, assigned mem smaller than total 
+      (is (>= avail used)))
+    (doseq [[avail used] cpu-avail->used] ;; for each node, assigned cpu smaller than total
+      (is (>= avail used))))
+  (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))))
+
+(deftest test-scheduling-resilience
+  (let [supers (gen-supervisors 2 2)
+         builder1 (TopologyBuilder.)
+         _ (.setSpout builder1 "spout1" (TestWordSpout.) 2)
+         storm-topology1 (.createTopology builder1)
+         topology1 (TopologyDetails. "topology1"
+                     {TOPOLOGY-NAME "topology-name-1"
+                      TOPOLOGY-SUBMITTER-USER "userC"
+                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     storm-topology1
+                     3 ;; three workers to hold three executors
+                     (mk-ed-map [["spout1" 0 3]]))
+         builder2 (TopologyBuilder.)
+         _ (.setSpout builder2 "spout2" (TestWordSpout.) 2)
+         storm-topology2 (.createTopology builder2)
+         topology2 (TopologyDetails. "topology2"
+                     {TOPOLOGY-NAME "topology-name-2"
+                      TOPOLOGY-SUBMITTER-USER "userC"
+                      TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 1280.0 ;; large enough thus two eds can not be fully assigned to one node
+                      TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                      TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                      TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                     storm-topology2
+                     2  ;; two workers, each holds one executor and resides on one node
+                     (mk-ed-map [["spout2" 0 2]]))
+        scheduler (ResourceAwareScheduler.)]
+
+    (testing "When a worker fails, RAS does not alter existing assignments on healthy workers"
+      (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            topologies (Topologies. (to-top-map [topology2]))
+            _ (.schedule scheduler topologies cluster)
+            assignment (.getAssignmentById cluster "topology2")
+            failed-worker (first (vec (.getSlots assignment)))  ;; choose a worker to mock as failed
+            ed->slot (.getExecutorToSlot assignment)
+            failed-eds (.get (reverse-map ed->slot) failed-worker)
+            _ (doseq [ed failed-eds] (.remove ed->slot ed))  ;; remove executor details assigned to the worker
+            copy-old-mapping (HashMap. ed->slot)
+            healthy-eds (.keySet copy-old-mapping)
+            _ (.schedule scheduler topologies cluster)
+            new-assignment (.getAssignmentById cluster "topology2")
+            new-ed->slot (.getExecutorToSlot new-assignment)]
+        ;; for each executor that was scheduled on healthy workers, their slots should remain unchanged after a new scheduling
+        (doseq [ed healthy-eds]
+          (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
+        (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))
+    
+    (testing "When a supervisor fails, RAS does not alter existing assignments"
+      (let [existing-assignments {"topology1" (SchedulerAssignmentImpl. "topology1"
+                                                                        {(ExecutorDetails. 0 0) (WorkerSlot. "id0" 0)    ;; worker 0 on the failed super
+                                                                         (ExecutorDetails. 1 1) (WorkerSlot. "id0" 1)    ;; worker 1 on the failed super
+                                                                         (ExecutorDetails. 2 2) (WorkerSlot. "id1" 1)})} ;; worker 2 on the health super
+            cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            topologies (Topologies. (to-top-map [topology1]))
+            assignment (.getAssignmentById cluster "topology1")
+            ed->slot (.getExecutorToSlot assignment)
+            copy-old-mapping (HashMap. ed->slot)
+            existing-eds (.keySet copy-old-mapping)  ;; all the three eds on three workers
+            new-cluster (Cluster. (nimbus/standalone-nimbus) 
+                                  (dissoc supers "id0")        ;; mock the super0 as a failed supervisor
+                                  (.getAssignments cluster)
+                                  {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                                   "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            _ (.schedule scheduler topologies new-cluster) ;; the actual schedule for this topo will not run since it is fully assigned
+            new-assignment (.getAssignmentById new-cluster "topology1")
+            new-ed->slot (.getExecutorToSlot new-assignment)]
+        (doseq [ed existing-eds]
+          (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
+        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap new-cluster) "topology1")))))
+
+    (testing "When a supervisor and a worker on it fails, RAS does not alter existing assignments"
+      (let [existing-assignments {"topology1" (SchedulerAssignmentImpl. "topology1"
+                                                                        {(ExecutorDetails. 0 0) (WorkerSlot. "id0" 1)    ;; the worker to orphan
+                                                                         (ExecutorDetails. 1 1) (WorkerSlot. "id0" 2)    ;; the worker to kill
+                                                                         (ExecutorDetails. 2 2) (WorkerSlot. "id1" 1)})} ;; the healthy worker
+            cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            topologies (Topologies. (to-top-map [topology1]))
+            assignment (.getAssignmentById cluster "topology1")
+            ed->slot (.getExecutorToSlot assignment)
+            _ (.remove ed->slot (ExecutorDetails. 1 1))  ;; delete one worker of super0 (failed) from topo1 assignment to enable actual schedule for testing
+            copy-old-mapping (HashMap. ed->slot)
+            existing-eds (.keySet copy-old-mapping)  ;; namely the two eds on the orphaned worker and the healthy worker
+            new-cluster (Cluster. (nimbus/standalone-nimbus) 
+                                  (dissoc supers "id0")        ;; mock the super0 as a failed supervisor
+                                  (.getAssignments cluster)
+                                  {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                                   "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            _ (.schedule scheduler topologies new-cluster)
+            new-assignment (.getAssignmentById new-cluster "topology1")
+            new-ed->slot (.getExecutorToSlot new-assignment)]
+        (doseq [ed existing-eds]
+          (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))
+        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap new-cluster) "topology1")))))
+
+    (testing "Scheduling a new topology does not disturb other assignments unnecessarily"
+      (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            topologies (Topologies. (to-top-map [topology1]))
+            _ (.schedule scheduler topologies cluster)
+            assignment (.getAssignmentById cluster "topology1")
+            ed->slot (.getExecutorToSlot assignment)
+            copy-old-mapping (HashMap. ed->slot)
+            new-topologies (Topologies. (to-top-map [topology1 topology2]))  ;; a second topology joins
+            _ (.schedule scheduler new-topologies cluster)
+            new-assignment (.getAssignmentById cluster "topology1")
+            new-ed->slot (.getExecutorToSlot new-assignment)]
+        (doseq [ed (.keySet copy-old-mapping)]
+          (is (.equals (.get copy-old-mapping ed) (.get new-ed->slot ed))))  ;; the assignment for topo1 should not change
+        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))))))
+
+;; Automated tests for heterogeneous cluster
+(deftest test-heterogeneous-cluster
+  (let [supers (into {} (for [super [(SupervisorDetails. (str "id" 0) (str "host" 0) (list ) 
+                                                         (map int (list 1 2 3 4))
+                                                         {Config/SUPERVISOR_MEMORY_CAPACITY_MB 4096.0
+                                                          Config/SUPERVISOR_CPU_CAPACITY 800.0})
+                                     (SupervisorDetails. (str "id" 1) (str "host" 1) (list ) 
+                                                         (map int (list 1 2 3 4))
+                                                         {Config/SUPERVISOR_MEMORY_CAPACITY_MB 1024.0
+                                                          Config/SUPERVISOR_CPU_CAPACITY 200.0})]]
+                          {(.getId super) super}))
+        builder1 (TopologyBuilder.)  ;; topo1 has one single huge task that can not be handled by the small-super
+        _ (doto (.setSpout builder1 "spout1" (TestWordSpout.) 1) 
+            (.setMemoryLoad 2000.0 48.0)
+            (.setCPULoad 300.0))
+        storm-topology1 (.createTopology builder1)
+        topology1 (TopologyDetails. "topology1"
+                    {TOPOLOGY-NAME "topology-name-1"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology1
+                    1
+                    (mk-ed-map [["spout1" 0 1]]))
+        builder2 (TopologyBuilder.)  ;; topo2 has 4 large tasks
+        _ (doto (.setSpout builder2 "spout2" (TestWordSpout.) 4)
+            (.setMemoryLoad 500.0 12.0)
+            (.setCPULoad 100.0))
+        storm-topology2 (.createTopology builder2)
+        topology2 (TopologyDetails. "topology2"
+                    {TOPOLOGY-NAME "topology-name-2"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology2
+                    2 
+                    (mk-ed-map [["spout2" 0 4]]))
+        builder3 (TopologyBuilder.) ;; topo3 has 4 medium tasks, launching topo 1-3 together requires the same mem as the cluster's mem capacity (5G)
+        _ (doto (.setSpout builder3 "spout3" (TestWordSpout.) 4)
+            (.setMemoryLoad 200.0 56.0)
+            (.setCPULoad 20.0))
+        storm-topology3 (.createTopology builder3)
+        topology3 (TopologyDetails. "topology3"
+                    {TOPOLOGY-NAME "topology-name-3"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology3
+                    2 
+                    (mk-ed-map [["spout3" 0 4]]))
+        builder4 (TopologyBuilder.) ;; topo4 has 12 small tasks, each's mem req does not exactly divide a node's mem capacity
+        _ (doto (.setSpout builder4 "spout4" (TestWordSpout.) 2)
+            (.setMemoryLoad 100.0 0.0)
+            (.setCPULoad 30.0))
+        storm-topology4 (.createTopology builder4)
+        topology4 (TopologyDetails. "topology4"
+                    {TOPOLOGY-NAME "topology-name-4"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology4
+                    2 
+                    (mk-ed-map [["spout4" 0 12]]))
+        builder5 (TopologyBuilder.) ;; topo5 has 40 small tasks, it should be able to exactly use up both the cpu and mem in teh cluster
+        _ (doto (.setSpout builder5 "spout5" (TestWordSpout.) 40)
+            (.setMemoryLoad 100.0 28.0)
+            (.setCPULoad 25.0))
+        storm-topology5 (.createTopology builder5)
+        topology5 (TopologyDetails. "topology5"
+                    {TOPOLOGY-NAME "topology-name-5"
+                     TOPOLOGY-SUBMITTER-USER "userC"
+                     TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                     TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                     TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                     TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 8192.0}
+                    storm-topology5
+                    2 
+                    (mk-ed-map [["spout5" 0 40]]))
+        epsilon 0.000001
+        topologies (Topologies. (to-top-map [topology1 topology2]))
+        scheduler (ResourceAwareScheduler.)]
+
+    (testing "Launch topo 1-3 together, it should be able to use up either mem or cpu resource due to exact division"
+      (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
+            _ (.schedule scheduler topologies cluster)
+            super->mem-usage (get-super->mem-usage cluster topologies)
+            super->cpu-usage (get-super->cpu-usage cluster topologies)]
+        (is (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))
+        (is (= "topology3 Fully Scheduled" (.get (.getStatusMap cluster) "topology3")))
+        (doseq [super (.values supers)] 
+          (let [mem-avail (.getTotalMemory super)
+                mem-used (.get super->mem-usage super)
+                cpu-avail (.getTotalCPU super)
+                cpu-used (.get super->cpu-usage super)]
+            (is (or (<= (Math/abs (- mem-avail mem-used)) epsilon)
+                    (<= (Math/abs (- cpu-avail cpu-used)) epsilon)))))))
+
+    (testing "Launch topo 1, 2 and 4, they together request a little more mem than available, so one of the 3 topos will not be scheduled"
+      (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
+            _ (.schedule scheduler topologies cluster)
+                scheduled-topos (if (= "topology1 Fully Scheduled" (.get (.getStatusMap cluster) "topology1")) 1 0)
+                scheduled-topos (+ scheduled-topos (if (= "topology2 Fully Scheduled" (.get (.getStatusMap cluster) "topology2")) 1 0))
+                scheduled-topos (+ scheduled-topos (if (= "topology4 Fully Scheduled" (.get (.getStatusMap cluster) "topology4")) 1 0))]
+            (is (= scheduled-topos 2)))) ;; only 2 topos will get (fully) scheduled
+
+    (testing "Launch topo5 only, both mem and cpu should be exactly used up"
+      (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+            topologies (Topologies. (to-top-map [topology5]))
+            _ (.schedule scheduler topologies cluster)
+            super->mem-usage (get-super->mem-usage cluster topologies)
+            super->cpu-usage (get-super->cpu-usage cluster topologies)]
+        (is (= "topology5 Fully Scheduled" (.get (.getStatusMap cluster) "topology5")))
+        (doseq [super (.values supers)] 
+          (let [mem-avail (.getTotalMemory super)
+                mem-used (.get super->mem-usage super)
+                cpu-avail (.getTotalCPU ^SupervisorDetails super)
+                cpu-used (.get super->cpu-usage super)]
+            (is (and (<= (Math/abs (- mem-avail mem-used)) epsilon)
+                    (<= (Math/abs (- cpu-avail cpu-used)) epsilon)))))))))
+
+(deftest test-topology-worker-max-heap-size
+  (let [supers (gen-supervisors 2 2)]
+    (testing "test if RAS will spread executors across mulitple workers based on the set limit for a worker used by the topology")
+    (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+          scheduler (ResourceAwareScheduler.)
+          builder1 (TopologyBuilder.)
+          _ (.setSpout builder1 "spout1" (TestWordSpout.) 2)
+          storm-topology1 (.createTopology builder1)
+          topology1 (TopologyDetails. "topology1"
+                      {TOPOLOGY-NAME "topology-name-1"
+                       TOPOLOGY-SUBMITTER-USER "userC"
+                       TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0}
+                      storm-topology1
+                      1
+                      (mk-ed-map [["spout1" 0 4]]))
+          topologies (Topologies. (to-top-map [topology1]))]
+      (.schedule scheduler topologies cluster)
+      (is (= (.get (.getStatusMap cluster) "topology1") "topology1 Fully Scheduled"))
+      (is (= (.getAssignedNumWorkers cluster topology1) 4)))
+    (testing "test when no more workers are available due to topology worker max heap size limit but there is memory is still available")
+    (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                              {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                               "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+          scheduler (ResourceAwareScheduler.)
+          builder1 (TopologyBuilder.)
+          _ (.setSpout builder1 "spout1" (TestWordSpout.) 2)
+          storm-topology1 (.createTopology builder1)
+          topology1 (TopologyDetails. "topology1"
+                      {TOPOLOGY-NAME "topology-name-1"
+                       TOPOLOGY-SUBMITTER-USER "userC"
+                       TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 128.0
+                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0}
+                      storm-topology1
+                      1
+                      (mk-ed-map [["spout1" 0 5]]))
+          topologies (Topologies. (to-top-map [topology1]))]
+      (.schedule scheduler topologies cluster)
+      ;;spout1 is going to contain 5 executors that needs scheduling. Each of those executors has a memory requirement of 128.0 MB
+      ;;The cluster contains 4 free WorkerSlots. For this topolology each worker is limited to a max heap size of 128.0
+      ;;Thus, one executor not going to be able to get scheduled thus failing the scheduling of this topology and no executors of this topology will be scheduleded
+      (is (= (.size (.getUnassignedExecutors cluster topology1)) 5))
+      (is (= (.get (.getStatusMap cluster) "topology1")  "Unsuccessfull in scheduling topology1")))
+    
+    (let [cluster (Cluster. (nimbus/standalone-nimbus) supers {}
+                             {STORM-NETWORK-TOPOGRAPHY-PLUGIN
+                              "backtype.storm.networktopography.DefaultRackDNSToSwitchMapping"})
+          cluster (LocalCluster.)
+          builder1 (TopologyBuilder.)
+          _ (.setSpout builder1 "spout1" (TestWordSpout.) 2)
+          storm-topology1 (.createTopology builder1)
+          conf  {TOPOLOGY-NAME "topology-name-1"
+                       TOPOLOGY-SUBMITTER-USER "userC"
+                       TOPOLOGY-COMPONENT-RESOURCES-ONHEAP-MEMORY-MB 129.0
+                       TOPOLOGY-COMPONENT-RESOURCES-OFFHEAP-MEMORY-MB 0.0
+                       TOPOLOGY-COMPONENT-CPU-PCORE-PERCENT 10.0
+                       TOPOLOGY-WORKER-MAX-HEAP-SIZE-MB 128.0}
+          topology1 (TopologyDetails. "topology1"
+                      conf
+                      storm-topology1
+                      1
+                      (mk-ed-map [["spout1" 0 5]]))
+          topologies (Topologies. (to-top-map [topology1]))]
+      (is (thrown? IllegalArgumentException
+            (StormSubmitter/submitTopologyWithProgressBar "test" conf storm-topology1)))
+       
+  )))

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/test/clj/backtype/storm/scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler_test.clj
index 45acf75..4360e9d 100644
--- a/storm-core/test/clj/backtype/storm/scheduler_test.clj
+++ b/storm-core/test/clj/backtype/storm/scheduler_test.clj
@@ -129,7 +129,8 @@
         assignment3 (SchedulerAssignmentImpl. "topology3" executor->slot3)
         cluster (Cluster. (nimbus/standalone-nimbus)
                           {"supervisor1" supervisor1 "supervisor2" supervisor2}
-                          {"topology1" assignment1 "topology2" assignment2 "topology3" assignment3})]
+                          {"topology1" assignment1 "topology2" assignment2 "topology3" assignment3}
+                  nil)]
     ;; test Cluster constructor
     (is (= #{"supervisor1" "supervisor2"}
            (->> cluster


[4/9] storm git commit: [STORM-893] - Resource Aware Scheduler implementation. [STORM-894] - Basic Resource Aware Scheduling implementation.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index 6c2cc69..f023f63 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
 public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
 
@@ -1194,14 +1194,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           case 4: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list298 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorSummary>(_list298.size);
-                ExecutorSummary _elem299;
-                for (int _i300 = 0; _i300 < _list298.size; ++_i300)
+                org.apache.thrift.protocol.TList _list308 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorSummary>(_list308.size);
+                ExecutorSummary _elem309;
+                for (int _i310 = 0; _i310 < _list308.size; ++_i310)
                 {
-                  _elem299 = new ExecutorSummary();
-                  _elem299.read(iprot);
-                  struct.executors.add(_elem299);
+                  _elem309 = new ExecutorSummary();
+                  _elem309.read(iprot);
+                  struct.executors.add(_elem309);
                 }
                 iprot.readListEnd();
               }
@@ -1221,26 +1221,26 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           case 6: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map301 = iprot.readMapBegin();
-                struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map301.size);
-                String _key302;
-                List<ErrorInfo> _val303;
-                for (int _i304 = 0; _i304 < _map301.size; ++_i304)
+                org.apache.thrift.protocol.TMap _map311 = iprot.readMapBegin();
+                struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map311.size);
+                String _key312;
+                List<ErrorInfo> _val313;
+                for (int _i314 = 0; _i314 < _map311.size; ++_i314)
                 {
-                  _key302 = iprot.readString();
+                  _key312 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TList _list305 = iprot.readListBegin();
-                    _val303 = new ArrayList<ErrorInfo>(_list305.size);
-                    ErrorInfo _elem306;
-                    for (int _i307 = 0; _i307 < _list305.size; ++_i307)
+                    org.apache.thrift.protocol.TList _list315 = iprot.readListBegin();
+                    _val313 = new ArrayList<ErrorInfo>(_list315.size);
+                    ErrorInfo _elem316;
+                    for (int _i317 = 0; _i317 < _list315.size; ++_i317)
                     {
-                      _elem306 = new ErrorInfo();
-                      _elem306.read(iprot);
-                      _val303.add(_elem306);
+                      _elem316 = new ErrorInfo();
+                      _elem316.read(iprot);
+                      _val313.add(_elem316);
                     }
                     iprot.readListEnd();
                   }
-                  struct.errors.put(_key302, _val303);
+                  struct.errors.put(_key312, _val313);
                 }
                 iprot.readMapEnd();
               }
@@ -1252,16 +1252,16 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           case 7: // COMPONENT_DEBUG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map308 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,DebugOptions>(2*_map308.size);
-                String _key309;
-                DebugOptions _val310;
-                for (int _i311 = 0; _i311 < _map308.size; ++_i311)
+                org.apache.thrift.protocol.TMap _map318 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map318.size);
+                String _key319;
+                DebugOptions _val320;
+                for (int _i321 = 0; _i321 < _map318.size; ++_i321)
                 {
-                  _key309 = iprot.readString();
-                  _val310 = new DebugOptions();
-                  _val310.read(iprot);
-                  struct.component_debug.put(_key309, _val310);
+                  _key319 = iprot.readString();
+                  _val320 = new DebugOptions();
+                  _val320.read(iprot);
+                  struct.component_debug.put(_key319, _val320);
                 }
                 iprot.readMapEnd();
               }
@@ -1324,9 +1324,9 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorSummary _iter312 : struct.executors)
+          for (ExecutorSummary _iter322 : struct.executors)
           {
-            _iter312.write(oprot);
+            _iter322.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1341,14 +1341,14 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
         oprot.writeFieldBegin(ERRORS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.errors.size()));
-          for (Map.Entry<String, List<ErrorInfo>> _iter313 : struct.errors.entrySet())
+          for (Map.Entry<String, List<ErrorInfo>> _iter323 : struct.errors.entrySet())
           {
-            oprot.writeString(_iter313.getKey());
+            oprot.writeString(_iter323.getKey());
             {
-              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter313.getValue().size()));
-              for (ErrorInfo _iter314 : _iter313.getValue())
+              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter323.getValue().size()));
+              for (ErrorInfo _iter324 : _iter323.getValue())
               {
-                _iter314.write(oprot);
+                _iter324.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -1362,10 +1362,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
           oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.component_debug.size()));
-            for (Map.Entry<String, DebugOptions> _iter315 : struct.component_debug.entrySet())
+            for (Map.Entry<String, DebugOptions> _iter325 : struct.component_debug.entrySet())
             {
-              oprot.writeString(_iter315.getKey());
-              _iter315.getValue().write(oprot);
+              oprot.writeString(_iter325.getKey());
+              _iter325.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1413,22 +1413,22 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       oprot.writeI32(struct.uptime_secs);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorSummary _iter316 : struct.executors)
+        for (ExecutorSummary _iter326 : struct.executors)
         {
-          _iter316.write(oprot);
+          _iter326.write(oprot);
         }
       }
       oprot.writeString(struct.status);
       {
         oprot.writeI32(struct.errors.size());
-        for (Map.Entry<String, List<ErrorInfo>> _iter317 : struct.errors.entrySet())
+        for (Map.Entry<String, List<ErrorInfo>> _iter327 : struct.errors.entrySet())
         {
-          oprot.writeString(_iter317.getKey());
+          oprot.writeString(_iter327.getKey());
           {
-            oprot.writeI32(_iter317.getValue().size());
-            for (ErrorInfo _iter318 : _iter317.getValue())
+            oprot.writeI32(_iter327.getValue().size());
+            for (ErrorInfo _iter328 : _iter327.getValue())
             {
-              _iter318.write(oprot);
+              _iter328.write(oprot);
             }
           }
         }
@@ -1450,10 +1450,10 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, DebugOptions> _iter319 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter329 : struct.component_debug.entrySet())
           {
-            oprot.writeString(_iter319.getKey());
-            _iter319.getValue().write(oprot);
+            oprot.writeString(_iter329.getKey());
+            _iter329.getValue().write(oprot);
           }
         }
       }
@@ -1478,55 +1478,55 @@ public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, Topol
       struct.uptime_secs = iprot.readI32();
       struct.set_uptime_secs_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list320 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorSummary>(_list320.size);
-        ExecutorSummary _elem321;
-        for (int _i322 = 0; _i322 < _list320.size; ++_i322)
+        org.apache.thrift.protocol.TList _list330 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorSummary>(_list330.size);
+        ExecutorSummary _elem331;
+        for (int _i332 = 0; _i332 < _list330.size; ++_i332)
         {
-          _elem321 = new ExecutorSummary();
-          _elem321.read(iprot);
-          struct.executors.add(_elem321);
+          _elem331 = new ExecutorSummary();
+          _elem331.read(iprot);
+          struct.executors.add(_elem331);
         }
       }
       struct.set_executors_isSet(true);
       struct.status = iprot.readString();
       struct.set_status_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map323 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
-        struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map323.size);
-        String _key324;
-        List<ErrorInfo> _val325;
-        for (int _i326 = 0; _i326 < _map323.size; ++_i326)
+        org.apache.thrift.protocol.TMap _map333 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, iprot.readI32());
+        struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map333.size);
+        String _key334;
+        List<ErrorInfo> _val335;
+        for (int _i336 = 0; _i336 < _map333.size; ++_i336)
         {
-          _key324 = iprot.readString();
+          _key334 = iprot.readString();
           {
-            org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            _val325 = new ArrayList<ErrorInfo>(_list327.size);
-            ErrorInfo _elem328;
-            for (int _i329 = 0; _i329 < _list327.size; ++_i329)
+            org.apache.thrift.protocol.TList _list337 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val335 = new ArrayList<ErrorInfo>(_list337.size);
+            ErrorInfo _elem338;
+            for (int _i339 = 0; _i339 < _list337.size; ++_i339)
             {
-              _elem328 = new ErrorInfo();
-              _elem328.read(iprot);
-              _val325.add(_elem328);
+              _elem338 = new ErrorInfo();
+              _elem338.read(iprot);
+              _val335.add(_elem338);
             }
           }
-          struct.errors.put(_key324, _val325);
+          struct.errors.put(_key334, _val335);
         }
       }
       struct.set_errors_isSet(true);
       BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map330 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.component_debug = new HashMap<String,DebugOptions>(2*_map330.size);
-          String _key331;
-          DebugOptions _val332;
-          for (int _i333 = 0; _i333 < _map330.size; ++_i333)
+          org.apache.thrift.protocol.TMap _map340 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.component_debug = new HashMap<String,DebugOptions>(2*_map340.size);
+          String _key341;
+          DebugOptions _val342;
+          for (int _i343 = 0; _i343 < _map340.size; ++_i343)
           {
-            _key331 = iprot.readString();
-            _val332 = new DebugOptions();
-            _val332.read(iprot);
-            struct.component_debug.put(_key331, _val332);
+            _key341 = iprot.readString();
+            _val342 = new DebugOptions();
+            _val342.read(iprot);
+            struct.component_debug.put(_key341, _val342);
           }
         }
         struct.set_component_debug_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
index c31e23e..3528ca2 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInfo, TopologyPageInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyPageInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyPageInfo");
 
@@ -1585,16 +1585,16 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
           case 9: // ID_TO_SPOUT_AGG_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map384 = iprot.readMapBegin();
-                struct.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map384.size);
-                String _key385;
-                ComponentAggregateStats _val386;
-                for (int _i387 = 0; _i387 < _map384.size; ++_i387)
+                org.apache.thrift.protocol.TMap _map394 = iprot.readMapBegin();
+                struct.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map394.size);
+                String _key395;
+                ComponentAggregateStats _val396;
+                for (int _i397 = 0; _i397 < _map394.size; ++_i397)
                 {
-                  _key385 = iprot.readString();
-                  _val386 = new ComponentAggregateStats();
-                  _val386.read(iprot);
-                  struct.id_to_spout_agg_stats.put(_key385, _val386);
+                  _key395 = iprot.readString();
+                  _val396 = new ComponentAggregateStats();
+                  _val396.read(iprot);
+                  struct.id_to_spout_agg_stats.put(_key395, _val396);
                 }
                 iprot.readMapEnd();
               }
@@ -1606,16 +1606,16 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
           case 10: // ID_TO_BOLT_AGG_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map388 = iprot.readMapBegin();
-                struct.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map388.size);
-                String _key389;
-                ComponentAggregateStats _val390;
-                for (int _i391 = 0; _i391 < _map388.size; ++_i391)
+                org.apache.thrift.protocol.TMap _map398 = iprot.readMapBegin();
+                struct.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map398.size);
+                String _key399;
+                ComponentAggregateStats _val400;
+                for (int _i401 = 0; _i401 < _map398.size; ++_i401)
                 {
-                  _key389 = iprot.readString();
-                  _val390 = new ComponentAggregateStats();
-                  _val390.read(iprot);
-                  struct.id_to_bolt_agg_stats.put(_key389, _val390);
+                  _key399 = iprot.readString();
+                  _val400 = new ComponentAggregateStats();
+                  _val400.read(iprot);
+                  struct.id_to_bolt_agg_stats.put(_key399, _val400);
                 }
                 iprot.readMapEnd();
               }
@@ -1730,10 +1730,10 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
           oprot.writeFieldBegin(ID_TO_SPOUT_AGG_STATS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.id_to_spout_agg_stats.size()));
-            for (Map.Entry<String, ComponentAggregateStats> _iter392 : struct.id_to_spout_agg_stats.entrySet())
+            for (Map.Entry<String, ComponentAggregateStats> _iter402 : struct.id_to_spout_agg_stats.entrySet())
             {
-              oprot.writeString(_iter392.getKey());
-              _iter392.getValue().write(oprot);
+              oprot.writeString(_iter402.getKey());
+              _iter402.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1745,10 +1745,10 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
           oprot.writeFieldBegin(ID_TO_BOLT_AGG_STATS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.id_to_bolt_agg_stats.size()));
-            for (Map.Entry<String, ComponentAggregateStats> _iter393 : struct.id_to_bolt_agg_stats.entrySet())
+            for (Map.Entry<String, ComponentAggregateStats> _iter403 : struct.id_to_bolt_agg_stats.entrySet())
             {
-              oprot.writeString(_iter393.getKey());
-              _iter393.getValue().write(oprot);
+              oprot.writeString(_iter403.getKey());
+              _iter403.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1874,20 +1874,20 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       if (struct.is_set_id_to_spout_agg_stats()) {
         {
           oprot.writeI32(struct.id_to_spout_agg_stats.size());
-          for (Map.Entry<String, ComponentAggregateStats> _iter394 : struct.id_to_spout_agg_stats.entrySet())
+          for (Map.Entry<String, ComponentAggregateStats> _iter404 : struct.id_to_spout_agg_stats.entrySet())
           {
-            oprot.writeString(_iter394.getKey());
-            _iter394.getValue().write(oprot);
+            oprot.writeString(_iter404.getKey());
+            _iter404.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_id_to_bolt_agg_stats()) {
         {
           oprot.writeI32(struct.id_to_bolt_agg_stats.size());
-          for (Map.Entry<String, ComponentAggregateStats> _iter395 : struct.id_to_bolt_agg_stats.entrySet())
+          for (Map.Entry<String, ComponentAggregateStats> _iter405 : struct.id_to_bolt_agg_stats.entrySet())
           {
-            oprot.writeString(_iter395.getKey());
-            _iter395.getValue().write(oprot);
+            oprot.writeString(_iter405.getKey());
+            _iter405.getValue().write(oprot);
           }
         }
       }
@@ -1944,32 +1944,32 @@ public class TopologyPageInfo implements org.apache.thrift.TBase<TopologyPageInf
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TMap _map396 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map396.size);
-          String _key397;
-          ComponentAggregateStats _val398;
-          for (int _i399 = 0; _i399 < _map396.size; ++_i399)
+          org.apache.thrift.protocol.TMap _map406 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map406.size);
+          String _key407;
+          ComponentAggregateStats _val408;
+          for (int _i409 = 0; _i409 < _map406.size; ++_i409)
           {
-            _key397 = iprot.readString();
-            _val398 = new ComponentAggregateStats();
-            _val398.read(iprot);
-            struct.id_to_spout_agg_stats.put(_key397, _val398);
+            _key407 = iprot.readString();
+            _val408 = new ComponentAggregateStats();
+            _val408.read(iprot);
+            struct.id_to_spout_agg_stats.put(_key407, _val408);
           }
         }
         struct.set_id_to_spout_agg_stats_isSet(true);
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TMap _map400 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map400.size);
-          String _key401;
-          ComponentAggregateStats _val402;
-          for (int _i403 = 0; _i403 < _map400.size; ++_i403)
+          org.apache.thrift.protocol.TMap _map410 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map410.size);
+          String _key411;
+          ComponentAggregateStats _val412;
+          for (int _i413 = 0; _i413 < _map410.size; ++_i413)
           {
-            _key401 = iprot.readString();
-            _val402 = new ComponentAggregateStats();
-            _val402.read(iprot);
-            struct.id_to_bolt_agg_stats.put(_key401, _val402);
+            _key411 = iprot.readString();
+            _val412 = new ComponentAggregateStats();
+            _val412.read(iprot);
+            struct.id_to_bolt_agg_stats.put(_key411, _val412);
           }
         }
         struct.set_id_to_bolt_agg_stats_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
index 453ead2..b2f5b23 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, TopologyStats._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyStats");
 
@@ -737,15 +737,15 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           case 1: // WINDOW_TO_EMITTED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map334 = iprot.readMapBegin();
-                struct.window_to_emitted = new HashMap<String,Long>(2*_map334.size);
-                String _key335;
-                long _val336;
-                for (int _i337 = 0; _i337 < _map334.size; ++_i337)
+                org.apache.thrift.protocol.TMap _map344 = iprot.readMapBegin();
+                struct.window_to_emitted = new HashMap<String,Long>(2*_map344.size);
+                String _key345;
+                long _val346;
+                for (int _i347 = 0; _i347 < _map344.size; ++_i347)
                 {
-                  _key335 = iprot.readString();
-                  _val336 = iprot.readI64();
-                  struct.window_to_emitted.put(_key335, _val336);
+                  _key345 = iprot.readString();
+                  _val346 = iprot.readI64();
+                  struct.window_to_emitted.put(_key345, _val346);
                 }
                 iprot.readMapEnd();
               }
@@ -757,15 +757,15 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           case 2: // WINDOW_TO_TRANSFERRED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map338 = iprot.readMapBegin();
-                struct.window_to_transferred = new HashMap<String,Long>(2*_map338.size);
-                String _key339;
-                long _val340;
-                for (int _i341 = 0; _i341 < _map338.size; ++_i341)
+                org.apache.thrift.protocol.TMap _map348 = iprot.readMapBegin();
+                struct.window_to_transferred = new HashMap<String,Long>(2*_map348.size);
+                String _key349;
+                long _val350;
+                for (int _i351 = 0; _i351 < _map348.size; ++_i351)
                 {
-                  _key339 = iprot.readString();
-                  _val340 = iprot.readI64();
-                  struct.window_to_transferred.put(_key339, _val340);
+                  _key349 = iprot.readString();
+                  _val350 = iprot.readI64();
+                  struct.window_to_transferred.put(_key349, _val350);
                 }
                 iprot.readMapEnd();
               }
@@ -777,15 +777,15 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           case 3: // WINDOW_TO_COMPLETE_LATENCIES_MS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map342 = iprot.readMapBegin();
-                struct.window_to_complete_latencies_ms = new HashMap<String,Double>(2*_map342.size);
-                String _key343;
-                double _val344;
-                for (int _i345 = 0; _i345 < _map342.size; ++_i345)
+                org.apache.thrift.protocol.TMap _map352 = iprot.readMapBegin();
+                struct.window_to_complete_latencies_ms = new HashMap<String,Double>(2*_map352.size);
+                String _key353;
+                double _val354;
+                for (int _i355 = 0; _i355 < _map352.size; ++_i355)
                 {
-                  _key343 = iprot.readString();
-                  _val344 = iprot.readDouble();
-                  struct.window_to_complete_latencies_ms.put(_key343, _val344);
+                  _key353 = iprot.readString();
+                  _val354 = iprot.readDouble();
+                  struct.window_to_complete_latencies_ms.put(_key353, _val354);
                 }
                 iprot.readMapEnd();
               }
@@ -797,15 +797,15 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           case 4: // WINDOW_TO_ACKED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map346 = iprot.readMapBegin();
-                struct.window_to_acked = new HashMap<String,Long>(2*_map346.size);
-                String _key347;
-                long _val348;
-                for (int _i349 = 0; _i349 < _map346.size; ++_i349)
+                org.apache.thrift.protocol.TMap _map356 = iprot.readMapBegin();
+                struct.window_to_acked = new HashMap<String,Long>(2*_map356.size);
+                String _key357;
+                long _val358;
+                for (int _i359 = 0; _i359 < _map356.size; ++_i359)
                 {
-                  _key347 = iprot.readString();
-                  _val348 = iprot.readI64();
-                  struct.window_to_acked.put(_key347, _val348);
+                  _key357 = iprot.readString();
+                  _val358 = iprot.readI64();
+                  struct.window_to_acked.put(_key357, _val358);
                 }
                 iprot.readMapEnd();
               }
@@ -817,15 +817,15 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           case 5: // WINDOW_TO_FAILED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map350 = iprot.readMapBegin();
-                struct.window_to_failed = new HashMap<String,Long>(2*_map350.size);
-                String _key351;
-                long _val352;
-                for (int _i353 = 0; _i353 < _map350.size; ++_i353)
+                org.apache.thrift.protocol.TMap _map360 = iprot.readMapBegin();
+                struct.window_to_failed = new HashMap<String,Long>(2*_map360.size);
+                String _key361;
+                long _val362;
+                for (int _i363 = 0; _i363 < _map360.size; ++_i363)
                 {
-                  _key351 = iprot.readString();
-                  _val352 = iprot.readI64();
-                  struct.window_to_failed.put(_key351, _val352);
+                  _key361 = iprot.readString();
+                  _val362 = iprot.readI64();
+                  struct.window_to_failed.put(_key361, _val362);
                 }
                 iprot.readMapEnd();
               }
@@ -852,10 +852,10 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           oprot.writeFieldBegin(WINDOW_TO_EMITTED_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_emitted.size()));
-            for (Map.Entry<String, Long> _iter354 : struct.window_to_emitted.entrySet())
+            for (Map.Entry<String, Long> _iter364 : struct.window_to_emitted.entrySet())
             {
-              oprot.writeString(_iter354.getKey());
-              oprot.writeI64(_iter354.getValue());
+              oprot.writeString(_iter364.getKey());
+              oprot.writeI64(_iter364.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -867,10 +867,10 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           oprot.writeFieldBegin(WINDOW_TO_TRANSFERRED_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_transferred.size()));
-            for (Map.Entry<String, Long> _iter355 : struct.window_to_transferred.entrySet())
+            for (Map.Entry<String, Long> _iter365 : struct.window_to_transferred.entrySet())
             {
-              oprot.writeString(_iter355.getKey());
-              oprot.writeI64(_iter355.getValue());
+              oprot.writeString(_iter365.getKey());
+              oprot.writeI64(_iter365.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -882,10 +882,10 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           oprot.writeFieldBegin(WINDOW_TO_COMPLETE_LATENCIES_MS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.window_to_complete_latencies_ms.size()));
-            for (Map.Entry<String, Double> _iter356 : struct.window_to_complete_latencies_ms.entrySet())
+            for (Map.Entry<String, Double> _iter366 : struct.window_to_complete_latencies_ms.entrySet())
             {
-              oprot.writeString(_iter356.getKey());
-              oprot.writeDouble(_iter356.getValue());
+              oprot.writeString(_iter366.getKey());
+              oprot.writeDouble(_iter366.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -897,10 +897,10 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           oprot.writeFieldBegin(WINDOW_TO_ACKED_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_acked.size()));
-            for (Map.Entry<String, Long> _iter357 : struct.window_to_acked.entrySet())
+            for (Map.Entry<String, Long> _iter367 : struct.window_to_acked.entrySet())
             {
-              oprot.writeString(_iter357.getKey());
-              oprot.writeI64(_iter357.getValue());
+              oprot.writeString(_iter367.getKey());
+              oprot.writeI64(_iter367.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -912,10 +912,10 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
           oprot.writeFieldBegin(WINDOW_TO_FAILED_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_failed.size()));
-            for (Map.Entry<String, Long> _iter358 : struct.window_to_failed.entrySet())
+            for (Map.Entry<String, Long> _iter368 : struct.window_to_failed.entrySet())
             {
-              oprot.writeString(_iter358.getKey());
-              oprot.writeI64(_iter358.getValue());
+              oprot.writeString(_iter368.getKey());
+              oprot.writeI64(_iter368.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -959,50 +959,50 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
       if (struct.is_set_window_to_emitted()) {
         {
           oprot.writeI32(struct.window_to_emitted.size());
-          for (Map.Entry<String, Long> _iter359 : struct.window_to_emitted.entrySet())
+          for (Map.Entry<String, Long> _iter369 : struct.window_to_emitted.entrySet())
           {
-            oprot.writeString(_iter359.getKey());
-            oprot.writeI64(_iter359.getValue());
+            oprot.writeString(_iter369.getKey());
+            oprot.writeI64(_iter369.getValue());
           }
         }
       }
       if (struct.is_set_window_to_transferred()) {
         {
           oprot.writeI32(struct.window_to_transferred.size());
-          for (Map.Entry<String, Long> _iter360 : struct.window_to_transferred.entrySet())
+          for (Map.Entry<String, Long> _iter370 : struct.window_to_transferred.entrySet())
           {
-            oprot.writeString(_iter360.getKey());
-            oprot.writeI64(_iter360.getValue());
+            oprot.writeString(_iter370.getKey());
+            oprot.writeI64(_iter370.getValue());
           }
         }
       }
       if (struct.is_set_window_to_complete_latencies_ms()) {
         {
           oprot.writeI32(struct.window_to_complete_latencies_ms.size());
-          for (Map.Entry<String, Double> _iter361 : struct.window_to_complete_latencies_ms.entrySet())
+          for (Map.Entry<String, Double> _iter371 : struct.window_to_complete_latencies_ms.entrySet())
           {
-            oprot.writeString(_iter361.getKey());
-            oprot.writeDouble(_iter361.getValue());
+            oprot.writeString(_iter371.getKey());
+            oprot.writeDouble(_iter371.getValue());
           }
         }
       }
       if (struct.is_set_window_to_acked()) {
         {
           oprot.writeI32(struct.window_to_acked.size());
-          for (Map.Entry<String, Long> _iter362 : struct.window_to_acked.entrySet())
+          for (Map.Entry<String, Long> _iter372 : struct.window_to_acked.entrySet())
           {
-            oprot.writeString(_iter362.getKey());
-            oprot.writeI64(_iter362.getValue());
+            oprot.writeString(_iter372.getKey());
+            oprot.writeI64(_iter372.getValue());
           }
         }
       }
       if (struct.is_set_window_to_failed()) {
         {
           oprot.writeI32(struct.window_to_failed.size());
-          for (Map.Entry<String, Long> _iter363 : struct.window_to_failed.entrySet())
+          for (Map.Entry<String, Long> _iter373 : struct.window_to_failed.entrySet())
           {
-            oprot.writeString(_iter363.getKey());
-            oprot.writeI64(_iter363.getValue());
+            oprot.writeString(_iter373.getKey());
+            oprot.writeI64(_iter373.getValue());
           }
         }
       }
@@ -1014,75 +1014,75 @@ public class TopologyStats implements org.apache.thrift.TBase<TopologyStats, Top
       BitSet incoming = iprot.readBitSet(5);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map364 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.window_to_emitted = new HashMap<String,Long>(2*_map364.size);
-          String _key365;
-          long _val366;
-          for (int _i367 = 0; _i367 < _map364.size; ++_i367)
+          org.apache.thrift.protocol.TMap _map374 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_emitted = new HashMap<String,Long>(2*_map374.size);
+          String _key375;
+          long _val376;
+          for (int _i377 = 0; _i377 < _map374.size; ++_i377)
           {
-            _key365 = iprot.readString();
-            _val366 = iprot.readI64();
-            struct.window_to_emitted.put(_key365, _val366);
+            _key375 = iprot.readString();
+            _val376 = iprot.readI64();
+            struct.window_to_emitted.put(_key375, _val376);
           }
         }
         struct.set_window_to_emitted_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map368 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.window_to_transferred = new HashMap<String,Long>(2*_map368.size);
-          String _key369;
-          long _val370;
-          for (int _i371 = 0; _i371 < _map368.size; ++_i371)
+          org.apache.thrift.protocol.TMap _map378 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_transferred = new HashMap<String,Long>(2*_map378.size);
+          String _key379;
+          long _val380;
+          for (int _i381 = 0; _i381 < _map378.size; ++_i381)
           {
-            _key369 = iprot.readString();
-            _val370 = iprot.readI64();
-            struct.window_to_transferred.put(_key369, _val370);
+            _key379 = iprot.readString();
+            _val380 = iprot.readI64();
+            struct.window_to_transferred.put(_key379, _val380);
           }
         }
         struct.set_window_to_transferred_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map372 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-          struct.window_to_complete_latencies_ms = new HashMap<String,Double>(2*_map372.size);
-          String _key373;
-          double _val374;
-          for (int _i375 = 0; _i375 < _map372.size; ++_i375)
+          org.apache.thrift.protocol.TMap _map382 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.window_to_complete_latencies_ms = new HashMap<String,Double>(2*_map382.size);
+          String _key383;
+          double _val384;
+          for (int _i385 = 0; _i385 < _map382.size; ++_i385)
           {
-            _key373 = iprot.readString();
-            _val374 = iprot.readDouble();
-            struct.window_to_complete_latencies_ms.put(_key373, _val374);
+            _key383 = iprot.readString();
+            _val384 = iprot.readDouble();
+            struct.window_to_complete_latencies_ms.put(_key383, _val384);
           }
         }
         struct.set_window_to_complete_latencies_ms_isSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map376 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.window_to_acked = new HashMap<String,Long>(2*_map376.size);
-          String _key377;
-          long _val378;
-          for (int _i379 = 0; _i379 < _map376.size; ++_i379)
+          org.apache.thrift.protocol.TMap _map386 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_acked = new HashMap<String,Long>(2*_map386.size);
+          String _key387;
+          long _val388;
+          for (int _i389 = 0; _i389 < _map386.size; ++_i389)
           {
-            _key377 = iprot.readString();
-            _val378 = iprot.readI64();
-            struct.window_to_acked.put(_key377, _val378);
+            _key387 = iprot.readString();
+            _val388 = iprot.readI64();
+            struct.window_to_acked.put(_key387, _val388);
           }
         }
         struct.set_window_to_acked_isSet(true);
       }
       if (incoming.get(4)) {
         {
-          org.apache.thrift.protocol.TMap _map380 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.window_to_failed = new HashMap<String,Long>(2*_map380.size);
-          String _key381;
-          long _val382;
-          for (int _i383 = 0; _i383 < _map380.size; ++_i383)
+          org.apache.thrift.protocol.TMap _map390 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.window_to_failed = new HashMap<String,Long>(2*_map390.size);
+          String _key391;
+          long _val392;
+          for (int _i393 = 0; _i393 < _map390.size; ++_i393)
           {
-            _key381 = iprot.readString();
-            _val382 = iprot.readI64();
-            struct.window_to_failed.put(_key381, _val382);
+            _key391 = iprot.readString();
+            _val392 = iprot.readI64();
+            struct.window_to_failed.put(_key391, _val392);
           }
         }
         struct.set_window_to_failed_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/networktopography/AbstractDNSToSwitchMapping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/networktopography/AbstractDNSToSwitchMapping.java b/storm-core/src/jvm/backtype/storm/networktopography/AbstractDNSToSwitchMapping.java
new file mode 100644
index 0000000..e91e54a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/networktopography/AbstractDNSToSwitchMapping.java
@@ -0,0 +1,95 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.networktopography;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This is a base class for DNS to Switch mappings. <p/> It is not mandatory to
+ * derive {@link DNSToSwitchMapping} implementations from it, but it is strongly
+ * recommended, as it makes it easy for the developers to add new methods
+ * to this base class that are automatically picked up by all implementations.
+ * <p/>
+ *
+ */
+public abstract class AbstractDNSToSwitchMapping
+        implements DNSToSwitchMapping {
+
+    /**
+     * Create an unconfigured instance
+     */
+    protected AbstractDNSToSwitchMapping() {
+    }
+
+    /**
+     * Predicate that indicates that the switch mapping is known to be
+     * single-switch. The base class returns false: it assumes all mappings are
+     * multi-rack. Subclasses may override this with methods that are more aware
+     * of their topologies.
+     *
+     * <p/>
+     *
+     *
+     * @return true if the mapping thinks that it is on a single switch
+     */
+    public boolean isSingleSwitch() {
+        return false;
+    }
+
+    /**
+     * Get a copy of the map (for diagnostics)
+     * @return a clone of the map or null for none known
+     */
+    public Map<String, String> getSwitchMap() {
+        return null;
+    }
+
+    /**
+     * Generate a string listing the switch mapping implementation,
+     * the mapping for every known node and the number of nodes and
+     * unique switches known about -each entry to a separate line.
+     * @return a string that can be presented to the ops team or used in
+     * debug messages.
+     */
+    public String dumpTopology() {
+        Map<String, String> rack = getSwitchMap();
+        StringBuilder builder = new StringBuilder();
+        builder.append("Mapping: ").append(toString()).append("\n");
+        if (rack != null) {
+            builder.append("Map:\n");
+            Set<String> switches = new HashSet<String>();
+            for (Map.Entry<String, String> entry : rack.entrySet()) {
+                builder.append("  ")
+                        .append(entry.getKey())
+                        .append(" -> ")
+                        .append(entry.getValue())
+                        .append("\n");
+                switches.add(entry.getValue());
+            }
+            builder.append("Nodes: ").append(rack.size()).append("\n");
+            builder.append("Switches: ").append(switches.size()).append("\n");
+        } else {
+            builder.append("No topology information");
+        }
+        return builder.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/networktopography/DNSToSwitchMapping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/networktopography/DNSToSwitchMapping.java b/storm-core/src/jvm/backtype/storm/networktopography/DNSToSwitchMapping.java
new file mode 100644
index 0000000..fbfbb19
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/networktopography/DNSToSwitchMapping.java
@@ -0,0 +1,50 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.networktopography;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An interface that must be implemented to allow pluggable
+ * DNS-name/IP-address to RackID resolvers.
+ *
+ */
+public interface DNSToSwitchMapping {
+    public final static String DEFAULT_RACK = "/default-rack";
+
+    /**
+     * Resolves a list of DNS-names/IP-address and returns back a map of DNS-name->switch information ( network paths).
+     * Consider an element in the argument list - x.y.com. The switch information
+     * that is returned must be a network path of the form /foo/rack,
+     * where / is the root, and 'foo' is the switch where 'rack' is connected.
+     * Note the hostname/ip-address is not part of the returned path.
+     * The network topology of the cluster would determine the number of
+     * components in the network path.
+     * <p/>
+     *
+     * If a name cannot be resolved to a rack, the implementation
+     * should return {DEFAULT_RACK}. This
+     * is what the bundled implementations do, though it is not a formal requirement
+     *
+     * @param names the list of hosts to resolve (can be empty)
+     * @return Map of hosts to resolved network paths.
+     * If <i>names</i> is empty, then return empty Map
+     */
+    public Map<String, String> resolve(List<String> names);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/networktopography/DefaultRackDNSToSwitchMapping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/networktopography/DefaultRackDNSToSwitchMapping.java b/storm-core/src/jvm/backtype/storm/networktopography/DefaultRackDNSToSwitchMapping.java
new file mode 100644
index 0000000..f3f6152
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/networktopography/DefaultRackDNSToSwitchMapping.java
@@ -0,0 +1,35 @@
+package backtype.storm.networktopography;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class implements the {@link DNSToSwitchMapping} interface
+ *    It returns the DEFAULT_RACK for every host.
+ */
+public final class DefaultRackDNSToSwitchMapping extends AbstractDNSToSwitchMapping {
+
+    private Map<String, String> mappingCache = new ConcurrentHashMap<String, String>();
+
+    @Override
+    public Map<String,String> resolve(List<String> names) {
+
+        Map<String, String> m = new HashMap<String, String>();
+        if (names.isEmpty()) {
+            //name list is empty, return an empty map
+            return m;
+        }
+        for (String name : names) {
+            m.put(name, DEFAULT_RACK);
+            mappingCache.put(name, DEFAULT_RACK);
+        }
+        return m;
+    }
+
+    @Override
+    public String toString() {
+        return "DefaultRackDNSToSwitchMapping (" + mappingCache.size() + " mappings cached)" + dumpTopology();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index e0c7cc7..dd15999 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -25,12 +25,22 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import backtype.storm.Config;
+import backtype.storm.networktopography.DNSToSwitchMapping;
+import backtype.storm.utils.Utils;
+
 public class Cluster {
 
     /**
      * key: supervisor id, value: supervisor details
      */
     private Map<String, SupervisorDetails>   supervisors;
+
+    /**
+     * key: rack, value: nodes in that rack
+     */
+    private Map<String, List<String>> networkTopography;
+
     /**
      * key: topologyId, value: topology's current assignments.
      */
@@ -44,11 +54,13 @@ public class Cluster {
      * a map from hostname to supervisor id.
      */
     private Map<String, List<String>>        hostToId;
-    
+
+    private Map conf = null;
+
     private Set<String> blackListedHosts = new HashSet<String>();
     private INimbus inimbus;
 
-    public Cluster(INimbus nimbus, Map<String, SupervisorDetails> supervisors, Map<String, SchedulerAssignmentImpl> assignments){
+    public Cluster(INimbus nimbus, Map<String, SupervisorDetails> supervisors, Map<String, SchedulerAssignmentImpl> assignments, Map storm_conf){
         this.inimbus = nimbus;
         this.supervisors = new HashMap<String, SupervisorDetails>(supervisors.size());
         this.supervisors.putAll(supervisors);
@@ -64,6 +76,7 @@ public class Cluster {
             }
             this.hostToId.get(host).add(nodeId);
         }
+        this.conf = storm_conf;
     }
     
     public void setBlacklistedHosts(Set<String> hosts) {
@@ -438,6 +451,35 @@ public class Cluster {
         return this.supervisors;
     }
 
+    /*
+    * Note: Make sure the proper conf was passed into the Cluster constructor before calling this function
+    * It tries to load the proper network topography detection plugin specified in the config.
+    */
+    public Map<String, List<String>> getNetworkTopography() {
+        if (networkTopography == null) {
+            networkTopography = new HashMap<String, List<String>>();
+            ArrayList<String> supervisorHostNames = new ArrayList<String>();
+            for (SupervisorDetails s : supervisors.values()) {
+                supervisorHostNames.add(s.getHost());
+            }
+
+            String clazz = (String) conf.get(Config.STORM_NETWORK_TOPOGRAPHY_PLUGIN);
+            DNSToSwitchMapping topographyMapper = (DNSToSwitchMapping) Utils.newInstance(clazz);
+
+            Map<String, String> resolvedSuperVisors = topographyMapper.resolve(supervisorHostNames);
+            for (String hostName : resolvedSuperVisors.keySet()) {
+                String rack = resolvedSuperVisors.get(hostName);
+
+                if (!networkTopography.containsKey(rack)) {
+                    networkTopography.put(rack, new ArrayList<String>());
+                }
+                List<String> nodesForRack = networkTopography.get(rack);
+                nodesForRack.add(hostName);
+            }
+        }
+        return networkTopography;
+    }
+
     public void setStatus(String topologyId, String status) {
         this.status.put(topologyId, status);
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
index 7497f26..65e6e9b 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
@@ -20,8 +20,14 @@ package backtype.storm.scheduler;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.Map;
+
+import backtype.storm.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class SupervisorDetails {
+    private static final Logger LOG = LoggerFactory.getLogger(SupervisorDetails.class);
 
     String id;
     /**
@@ -37,25 +43,46 @@ public class SupervisorDetails {
      * all the ports of the supervisor
      */
     Set<Integer> allPorts;
+    /**
+     * Map containing a manifest of resources for the node the superivsor resides
+     */
+    private Map<String, Double> _total_resources;
+
+    public SupervisorDetails(String id, String host, Object meta, Object schedulerMeta,
+                             Collection<Number> allPorts, Map<String, Double> total_resources){
 
-    public SupervisorDetails(String id, Object meta){
         this.id = id;
+        this.host = host;
         this.meta = meta;
-        allPorts = new HashSet();
+        this.schedulerMeta = schedulerMeta;
+        if(allPorts!=null) {
+            setAllPorts(allPorts);
+        } else {
+            this.allPorts = new HashSet();
+        }
+        this._total_resources = total_resources;
+        LOG.debug("Creating a new supervisor ({}-{}) with resources: {}", this.host, this.id, total_resources);
     }
-    
+
+    public SupervisorDetails(String id, Object meta){
+        this(id, null, meta, null, null, null);
+    }
+
+    public SupervisorDetails(String id, Object meta, Map<String, Double> total_resources) {
+        this(id, null, meta, null, null, total_resources);
+    }
+
     public SupervisorDetails(String id, Object meta, Collection<Number> allPorts){
-        this.id = id;
-        this.meta = meta;
-        setAllPorts(allPorts);
+        this(id, null, meta, null, allPorts, null);
     }
 
-    public SupervisorDetails(String id, String host, Object schedulerMeta, Collection<Number> allPorts){
-        this.id = id;
-        this.host = host;
-        this.schedulerMeta = schedulerMeta;
+    public SupervisorDetails(String id, String host, Object schedulerMeta, Collection<Number> allPorts) {
+        this(id, host, null, schedulerMeta, allPorts, null);
+    }
 
-        setAllPorts(allPorts);
+    public SupervisorDetails(String id, String host, Object schedulerMeta,
+                             Collection<Number> allPorts, Map<String, Double> total_resources) {
+        this(id, host, null, schedulerMeta, allPorts, total_resources);
     }
 
     private void setAllPorts(Collection<Number> allPorts) {
@@ -86,4 +113,16 @@ public class SupervisorDetails {
     public Object getSchedulerMeta() {
         return this.schedulerMeta;
     }
+
+    private Double getTotalResource(String type) {
+        return this._total_resources.get(type);
+    }
+
+    public Double getTotalMemory() {
+        return getTotalResource(Config.SUPERVISOR_MEMORY_CAPACITY_MB);
+    }
+
+    public Double getTotalCPU() {
+        return getTotalResource(Config.SUPERVISOR_CPU_CAPACITY);
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index 70af1b4..409ecb3 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -21,10 +21,13 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 
+import backtype.storm.scheduler.resource.Component;
+
 public class Topologies {
     Map<String, TopologyDetails> topologies;
     Map<String, String> nameToId;
-    
+    Map<String, Map<String, Component>> _allComponents;
+
     public Topologies(Map<String, TopologyDetails> topologies) {
         if(topologies==null) topologies = new HashMap();
         this.topologies = new HashMap<String, TopologyDetails>(topologies.size());
@@ -54,4 +57,14 @@ public class Topologies {
     public Collection<TopologyDetails> getTopologies() {
         return this.topologies.values();
     }
+
+    public Map<String, Map<String, Component>> getAllComponents() {
+        if (_allComponents == null) {
+            _allComponents = new HashMap<>();
+            for (Map.Entry<String, TopologyDetails> entry : this.topologies.entrySet()) {
+                _allComponents.put(entry.getKey(), entry.getValue().getComponents());
+            }
+        }
+        return _allComponents;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 6daf4ed..872a834 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
@@ -17,12 +17,24 @@
  */
 package backtype.storm.scheduler;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import backtype.storm.Config;
+import backtype.storm.generated.Bolt;
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.generated.Grouping;
+import backtype.storm.generated.SpoutSpec;
 import backtype.storm.generated.StormTopology;
+import backtype.storm.scheduler.resource.Component;
+import backtype.storm.scheduler.resource.ResourceUtils;
+import backtype.storm.utils.Utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 public class TopologyDetails {
@@ -31,41 +43,45 @@ public class TopologyDetails {
     StormTopology topology;
     Map<ExecutorDetails, String> executorToComponent;
     int numWorkers;
- 
+    //<ExecutorDetails - Task, Map<String - Type of resource, Map<String - type of that resource, Double - amount>>>
+    private Map<ExecutorDetails, Map<String, Double>> _resourceList;
+    //Max heap size for a worker used by topology
+    private Double topologyWorkerMaxHeapSize;
+
+    private static final Logger LOG = LoggerFactory.getLogger(TopologyDetails.class);
+
     public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers) {
         this.topologyId = topologyId;
         this.topologyConf = topologyConf;
         this.topology = topology;
         this.numWorkers = numWorkers;
     }
-    
+
     public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers, Map<ExecutorDetails, String> executorToComponents) {
         this(topologyId, topologyConf, topology, numWorkers);
         this.executorToComponent = new HashMap<ExecutorDetails, String>(0);
         if (executorToComponents != null) {
             this.executorToComponent.putAll(executorToComponents);
         }
+        this.initResourceList();
+        this.initConfigs();
     }
-    
+
     public String getId() {
         return topologyId;
     }
-    
+
     public String getName() {
-        return (String)this.topologyConf.get(Config.TOPOLOGY_NAME);
+        return (String) this.topologyConf.get(Config.TOPOLOGY_NAME);
     }
-    
+
     public Map getConf() {
         return topologyConf;
     }
-    
+
     public int getNumWorkers() {
         return numWorkers;
     }
-    
-    public StormTopology getTopology() {
-        return topology;
-    }
 
     public Map<ExecutorDetails, String> getExecutorToComponent() {
         return this.executorToComponent;
@@ -79,11 +95,299 @@ public class TopologyDetails {
                 ret.put(executor, compId);
             }
         }
-        
+
         return ret;
     }
-    
+
     public Collection<ExecutorDetails> getExecutors() {
         return this.executorToComponent.keySet();
     }
+
+    private void initResourceList() {
+        _resourceList = new HashMap<ExecutorDetails, Map<String, Double>>();
+        // Extract bolt memory info
+        if (this.topology.get_bolts() != null) {
+            for (Map.Entry<String, Bolt> bolt : this.topology.get_bolts().entrySet()) {
+                //the json_conf is populated by TopologyBuilder (e.g. boltDeclarer.setMemoryLoad)
+                Map<String, Double> topology_resources = ResourceUtils.parseResources(bolt
+                        .getValue().get_common().get_json_conf());
+                ResourceUtils.checkIntialization(topology_resources, bolt.getValue().toString(), this.topologyConf);
+                for (Map.Entry<ExecutorDetails, String> anExecutorToComponent : executorToComponent.entrySet()) {
+                    if (bolt.getKey().equals(anExecutorToComponent.getValue())) {
+                        _resourceList.put(anExecutorToComponent.getKey(), topology_resources);
+                    }
+                }
+            }
+        }
+        // Extract spout memory info
+        if (this.topology.get_spouts() != null) {
+            for (Map.Entry<String, SpoutSpec> spout : this.topology.get_spouts().entrySet()) {
+                Map<String, Double> topology_resources = ResourceUtils.parseResources(spout
+                        .getValue().get_common().get_json_conf());
+                ResourceUtils.checkIntialization(topology_resources, spout.getValue().toString(), this.topologyConf);
+                for (Map.Entry<ExecutorDetails, String> anExecutorToComponent : executorToComponent.entrySet()) {
+                    if (spout.getKey().equals(anExecutorToComponent.getValue())) {
+                        _resourceList.put(anExecutorToComponent.getKey(), topology_resources);
+                    }
+                }
+            }
+        } else {
+            LOG.warn("Topology " + topologyId + " does not seem to have any spouts!");
+        }
+        //schedule tasks that are not part of components returned from topology.get_spout or topology.getbolt (AKA sys tasks most specifically __acker tasks)
+        for(ExecutorDetails exec : this.getExecutors()) {
+            if (_resourceList.containsKey(exec) == false) {
+                LOG.debug(
+                        "Scheduling {} {} with memory requirement as 'on heap' - {} and 'off heap' - {} and CPU requirement as {}",
+                        this.getExecutorToComponent().get(exec),
+                        exec,
+                        this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB),
+                        this.topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB),
+                        this.topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT));
+                this.addDefaultResforExec(exec);
+            } 
+        }
+    }
+
+    private List<ExecutorDetails> componentToExecs(String comp) {
+        List<ExecutorDetails> execs = new ArrayList<>();
+        for (Map.Entry<ExecutorDetails, String> entry : executorToComponent.entrySet()) {
+            if (entry.getValue().equals(comp)) {
+                execs.add(entry.getKey());
+            }
+        }
+        return execs;
+    }
+
+    /**
+     * Returns a representation of the non-system components of the topology graph
+     * Each Component object in the returning map is populated with the list of its
+     * parents, children and execs assigned to that component.
+     * @return a map of components
+     */
+    public Map<String, Component> getComponents() {
+        Map<String, Component> all_comp = new HashMap<String, Component>();
+
+        StormTopology storm_topo = this.topology;
+        // spouts
+        if (storm_topo.get_spouts() != null) {
+            for (Map.Entry<String, SpoutSpec> spoutEntry : storm_topo
+                    .get_spouts().entrySet()) {
+                if (!Utils.isSystemId(spoutEntry.getKey())) {
+                    Component newComp = null;
+                    if (all_comp.containsKey(spoutEntry.getKey())) {
+                        newComp = all_comp.get(spoutEntry.getKey());
+                        newComp.execs = componentToExecs(newComp.id);
+                    } else {
+                        newComp = new Component(spoutEntry.getKey());
+                        newComp.execs = componentToExecs(newComp.id);
+                        all_comp.put(spoutEntry.getKey(), newComp);
+                    }
+                    newComp.type = Component.ComponentType.SPOUT;
+
+                    for (Map.Entry<GlobalStreamId, Grouping> spoutInput : spoutEntry
+                            .getValue().get_common().get_inputs()
+                            .entrySet()) {
+                        newComp.parents.add(spoutInput.getKey()
+                                .get_componentId());
+                        if (!all_comp.containsKey(spoutInput
+                                .getKey().get_componentId())) {
+                            all_comp.put(spoutInput.getKey()
+                                            .get_componentId(),
+                                    new Component(spoutInput.getKey()
+                                            .get_componentId()));
+                        }
+                        all_comp.get(spoutInput.getKey()
+                                .get_componentId()).children.add(spoutEntry
+                                .getKey());
+                    }
+                }
+            }
+        }
+        // bolts
+        if (storm_topo.get_bolts() != null) {
+            for (Map.Entry<String, Bolt> boltEntry : storm_topo.get_bolts()
+                    .entrySet()) {
+                if (!Utils.isSystemId(boltEntry.getKey())) {
+                    Component newComp = null;
+                    if (all_comp.containsKey(boltEntry.getKey())) {
+                        newComp = all_comp.get(boltEntry.getKey());
+                        newComp.execs = componentToExecs(newComp.id);
+                    } else {
+                        newComp = new Component(boltEntry.getKey());
+                        newComp.execs = componentToExecs(newComp.id);
+                        all_comp.put(boltEntry.getKey(), newComp);
+                    }
+                    newComp.type = Component.ComponentType.BOLT;
+
+                    for (Map.Entry<GlobalStreamId, Grouping> boltInput : boltEntry
+                            .getValue().get_common().get_inputs()
+                            .entrySet()) {
+                        newComp.parents.add(boltInput.getKey()
+                                .get_componentId());
+                        if (!all_comp.containsKey(boltInput
+                                .getKey().get_componentId())) {
+                            all_comp.put(boltInput.getKey()
+                                            .get_componentId(),
+                                    new Component(boltInput.getKey()
+                                            .get_componentId()));
+                        }
+                        all_comp.get(boltInput.getKey()
+                                .get_componentId()).children.add(boltEntry
+                                .getKey());
+                    }
+                }
+            }
+        }
+        return all_comp;
+    }
+
+    /**
+     * Gets the on heap memory requirement for a
+     * certain task within a topology
+     * @param exec the executor the inquiry is concerning.
+     * @return Double the amount of on heap memory
+     * requirement for this exec in topology topoId.
+     */
+    public Double getOnHeapMemoryRequirement(ExecutorDetails exec) {
+        Double ret = null;
+        if (hasExecInTopo(exec)) {
+            ret = _resourceList
+                    .get(exec)
+                    .get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
+        }
+        return ret;
+    }
+
+    /**
+     * Gets the off heap memory requirement for a
+     * certain task within a topology
+     * @param exec the executor the inquiry is concerning.
+     * @return Double the amount of off heap memory
+     * requirement for this exec in topology topoId.
+     */
+    public Double getOffHeapMemoryRequirement(ExecutorDetails exec) {
+        Double ret = null;
+        if (hasExecInTopo(exec)) {
+            ret = _resourceList
+                    .get(exec)
+                    .get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB);
+        }
+        return ret;
+    }
+
+    /**
+     * Gets the total memory requirement for a task
+     * @param exec the executor the inquiry is concerning.
+     * @return Double the total memory requirement
+     *  for this exec in topology topoId.
+     */
+    public Double getTotalMemReqTask(ExecutorDetails exec) {
+        if (hasExecInTopo(exec)) {
+            return getOffHeapMemoryRequirement(exec)
+                    + getOnHeapMemoryRequirement(exec);
+        }
+        return null;
+    }
+
+    /**
+     * Gets the total memory resource list for a
+     * set of tasks that is part of a topology.
+     * @return Map<ExecutorDetails, Double> a map of the total memory requirement
+     *  for all tasks in topology topoId.
+     */
+    public Map<ExecutorDetails, Double> getTotalMemoryResourceList() {
+        Map<ExecutorDetails, Double> ret = new HashMap<ExecutorDetails, Double>();
+        for (ExecutorDetails exec : _resourceList.keySet()) {
+            ret.put(exec, getTotalMemReqTask(exec));
+        }
+        return ret;
+    }
+
+    /**
+     * Get the total CPU requirement for executor
+     * @param exec
+     * @return Double the total about of cpu requirement for executor
+     */
+    public Double getTotalCpuReqTask(ExecutorDetails exec) {
+        if (hasExecInTopo(exec)) {
+            return _resourceList
+                    .get(exec)
+                    .get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);
+        }
+        return null;
+    }
+
+    /**
+     * get the resources requirements for a executor
+     * @param exec
+     * @return a map containing the resource requirements for this exec
+     */
+    public Map<String, Double> getTaskResourceReqList(ExecutorDetails exec) {
+        if (hasExecInTopo(exec)) {
+            return _resourceList.get(exec);
+        }
+        return null;
+    }
+
+    /**
+     * Checks if a executor is part of this topology
+     * @return Boolean whether or not a certain ExecutorDetail is included in the _resourceList.
+     */
+    public boolean hasExecInTopo(ExecutorDetails exec) {
+        if (_resourceList != null) { // null is possible if the first constructor of TopologyDetails is used
+            return _resourceList.containsKey(exec);
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * add resource requirements for a executor
+     * @param exec
+     * @param resourceList
+     */
+    public void addResourcesForExec(ExecutorDetails exec, Map<String, Double> resourceList) {
+        if (hasExecInTopo(exec)) {
+            LOG.warn("Executor {} already exists...ResourceList: {}", exec, getTaskResourceReqList(exec));
+            return;
+        }
+        _resourceList.put(exec, resourceList);
+    }
+
+    /**
+     * Add default resource requirements for a executor
+     * @param exec
+     */
+    public void addDefaultResforExec(ExecutorDetails exec) {
+        Map<String, Double> defaultResourceList = new HashMap<String, Double>();
+        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT,
+                        Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null));
+        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB,
+                        Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null));
+        defaultResourceList.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
+                        Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null));
+        LOG.debug("Scheduling Executor: {} with memory requirement as onHeap: {} - offHeap: {} " +
+                        "and CPU requirement: {}",
+                exec, topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB),
+                topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB),
+                topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT));
+        addResourcesForExec(exec, defaultResourceList);
+    }
+
+    /**
+     * initalizes the scheduler member variable by extracting what scheduler
+     * this topology is going to use from topologyConf
+     */
+    private void initConfigs() {
+        this.topologyWorkerMaxHeapSize = Utils.getDouble(this.topologyConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB), null);
+    }
+
+    /**
+     * Get the max heap size for a worker used by this topology
+     * @return the worker max heap size
+     */
+    public Double getTopologyWorkerMaxHeapSize() {
+        return this.topologyWorkerMaxHeapSize;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/resource/Component.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/Component.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/Component.java
new file mode 100644
index 0000000..869b5ce
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/Component.java
@@ -0,0 +1,54 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.resource;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import backtype.storm.scheduler.ExecutorDetails;
+
+public class Component {
+    public enum ComponentType {
+        SPOUT(1), BOLT(2);
+        private int value;
+
+        private ComponentType(int value) {
+            this.value = value;
+        }
+    }
+
+    public String id;
+    public List<String> parents = null;
+    public List<String> children = null;
+    public List<ExecutorDetails> execs = null;
+    public ComponentType type = null;
+
+    public Component(String id) {
+        this.parents = new ArrayList<String>();
+        this.children = new ArrayList<String>();
+        this.execs = new ArrayList<ExecutorDetails>();
+        this.id = id;
+    }
+
+    @Override
+    public String toString() {
+        String retVal = "{id: " + this.id + " Parents: " + this.parents.toString() + " Children: " + this.children.toString() + " Execs: " + this.execs + "}";
+        return retVal;
+    }
+}


[2/9] storm git commit: [STORM-893] - Resource Aware Scheduler implementation. [STORM-894] - Basic Resource Aware Scheduling implementation.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/py/storm/ttypes.py
----------------------------------------------------------------------
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index d365566..4df4521 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -2400,6 +2400,7 @@ class SupervisorSummary:
    - num_used_workers
    - supervisor_id
    - version
+   - total_resources
   """
 
   thrift_spec = (
@@ -2410,15 +2411,17 @@ class SupervisorSummary:
     (4, TType.I32, 'num_used_workers', None, None, ), # 4
     (5, TType.STRING, 'supervisor_id', None, None, ), # 5
     (6, TType.STRING, 'version', None, "VERSION_NOT_PROVIDED", ), # 6
+    (7, TType.MAP, 'total_resources', (TType.STRING,None,TType.DOUBLE,None), None, ), # 7
   )
 
-  def __init__(self, host=None, uptime_secs=None, num_workers=None, num_used_workers=None, supervisor_id=None, version=thrift_spec[6][4],):
+  def __init__(self, host=None, uptime_secs=None, num_workers=None, num_used_workers=None, supervisor_id=None, version=thrift_spec[6][4], total_resources=None,):
     self.host = host
     self.uptime_secs = uptime_secs
     self.num_workers = num_workers
     self.num_used_workers = num_used_workers
     self.supervisor_id = supervisor_id
     self.version = version
+    self.total_resources = total_resources
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2459,6 +2462,17 @@ class SupervisorSummary:
           self.version = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.MAP:
+          self.total_resources = {}
+          (_ktype67, _vtype68, _size66 ) = iprot.readMapBegin()
+          for _i70 in xrange(_size66):
+            _key71 = iprot.readString().decode('utf-8')
+            _val72 = iprot.readDouble();
+            self.total_resources[_key71] = _val72
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2493,6 +2507,14 @@ class SupervisorSummary:
       oprot.writeFieldBegin('version', TType.STRING, 6)
       oprot.writeString(self.version.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.total_resources is not None:
+      oprot.writeFieldBegin('total_resources', TType.MAP, 7)
+      oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.total_resources))
+      for kiter73,viter74 in self.total_resources.items():
+        oprot.writeString(kiter73.encode('utf-8'))
+        oprot.writeDouble(viter74)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2518,6 +2540,7 @@ class SupervisorSummary:
     value = (value * 31) ^ hash(self.num_used_workers)
     value = (value * 31) ^ hash(self.supervisor_id)
     value = (value * 31) ^ hash(self.version)
+    value = (value * 31) ^ hash(self.total_resources)
     return value
 
   def __repr__(self):
@@ -2691,33 +2714,33 @@ class ClusterSummary:
       if fid == 1:
         if ftype == TType.LIST:
           self.supervisors = []
-          (_etype69, _size66) = iprot.readListBegin()
-          for _i70 in xrange(_size66):
-            _elem71 = SupervisorSummary()
-            _elem71.read(iprot)
-            self.supervisors.append(_elem71)
+          (_etype78, _size75) = iprot.readListBegin()
+          for _i79 in xrange(_size75):
+            _elem80 = SupervisorSummary()
+            _elem80.read(iprot)
+            self.supervisors.append(_elem80)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.topologies = []
-          (_etype75, _size72) = iprot.readListBegin()
-          for _i76 in xrange(_size72):
-            _elem77 = TopologySummary()
-            _elem77.read(iprot)
-            self.topologies.append(_elem77)
+          (_etype84, _size81) = iprot.readListBegin()
+          for _i85 in xrange(_size81):
+            _elem86 = TopologySummary()
+            _elem86.read(iprot)
+            self.topologies.append(_elem86)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.nimbuses = []
-          (_etype81, _size78) = iprot.readListBegin()
-          for _i82 in xrange(_size78):
-            _elem83 = NimbusSummary()
-            _elem83.read(iprot)
-            self.nimbuses.append(_elem83)
+          (_etype90, _size87) = iprot.readListBegin()
+          for _i91 in xrange(_size87):
+            _elem92 = NimbusSummary()
+            _elem92.read(iprot)
+            self.nimbuses.append(_elem92)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -2734,22 +2757,22 @@ class ClusterSummary:
     if self.supervisors is not None:
       oprot.writeFieldBegin('supervisors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.supervisors))
-      for iter84 in self.supervisors:
-        iter84.write(oprot)
+      for iter93 in self.supervisors:
+        iter93.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.topologies is not None:
       oprot.writeFieldBegin('topologies', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.topologies))
-      for iter85 in self.topologies:
-        iter85.write(oprot)
+      for iter94 in self.topologies:
+        iter94.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.nimbuses is not None:
       oprot.writeFieldBegin('nimbuses', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.nimbuses))
-      for iter86 in self.nimbuses:
-        iter86.write(oprot)
+      for iter95 in self.nimbuses:
+        iter95.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -2929,90 +2952,90 @@ class BoltStats:
       if fid == 1:
         if ftype == TType.MAP:
           self.acked = {}
-          (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin()
-          for _i91 in xrange(_size87):
-            _key92 = iprot.readString().decode('utf-8')
-            _val93 = {}
-            (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin()
-            for _i98 in xrange(_size94):
-              _key99 = GlobalStreamId()
-              _key99.read(iprot)
-              _val100 = iprot.readI64();
-              _val93[_key99] = _val100
+          (_ktype97, _vtype98, _size96 ) = iprot.readMapBegin()
+          for _i100 in xrange(_size96):
+            _key101 = iprot.readString().decode('utf-8')
+            _val102 = {}
+            (_ktype104, _vtype105, _size103 ) = iprot.readMapBegin()
+            for _i107 in xrange(_size103):
+              _key108 = GlobalStreamId()
+              _key108.read(iprot)
+              _val109 = iprot.readI64();
+              _val102[_key108] = _val109
             iprot.readMapEnd()
-            self.acked[_key92] = _val93
+            self.acked[_key101] = _val102
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.failed = {}
-          (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin()
-          for _i105 in xrange(_size101):
-            _key106 = iprot.readString().decode('utf-8')
-            _val107 = {}
-            (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin()
-            for _i112 in xrange(_size108):
-              _key113 = GlobalStreamId()
-              _key113.read(iprot)
-              _val114 = iprot.readI64();
-              _val107[_key113] = _val114
+          (_ktype111, _vtype112, _size110 ) = iprot.readMapBegin()
+          for _i114 in xrange(_size110):
+            _key115 = iprot.readString().decode('utf-8')
+            _val116 = {}
+            (_ktype118, _vtype119, _size117 ) = iprot.readMapBegin()
+            for _i121 in xrange(_size117):
+              _key122 = GlobalStreamId()
+              _key122.read(iprot)
+              _val123 = iprot.readI64();
+              _val116[_key122] = _val123
             iprot.readMapEnd()
-            self.failed[_key106] = _val107
+            self.failed[_key115] = _val116
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.process_ms_avg = {}
-          (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin()
-          for _i119 in xrange(_size115):
-            _key120 = iprot.readString().decode('utf-8')
-            _val121 = {}
-            (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin()
-            for _i126 in xrange(_size122):
-              _key127 = GlobalStreamId()
-              _key127.read(iprot)
-              _val128 = iprot.readDouble();
-              _val121[_key127] = _val128
+          (_ktype125, _vtype126, _size124 ) = iprot.readMapBegin()
+          for _i128 in xrange(_size124):
+            _key129 = iprot.readString().decode('utf-8')
+            _val130 = {}
+            (_ktype132, _vtype133, _size131 ) = iprot.readMapBegin()
+            for _i135 in xrange(_size131):
+              _key136 = GlobalStreamId()
+              _key136.read(iprot)
+              _val137 = iprot.readDouble();
+              _val130[_key136] = _val137
             iprot.readMapEnd()
-            self.process_ms_avg[_key120] = _val121
+            self.process_ms_avg[_key129] = _val130
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.executed = {}
-          (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin()
-          for _i133 in xrange(_size129):
-            _key134 = iprot.readString().decode('utf-8')
-            _val135 = {}
-            (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin()
-            for _i140 in xrange(_size136):
-              _key141 = GlobalStreamId()
-              _key141.read(iprot)
-              _val142 = iprot.readI64();
-              _val135[_key141] = _val142
+          (_ktype139, _vtype140, _size138 ) = iprot.readMapBegin()
+          for _i142 in xrange(_size138):
+            _key143 = iprot.readString().decode('utf-8')
+            _val144 = {}
+            (_ktype146, _vtype147, _size145 ) = iprot.readMapBegin()
+            for _i149 in xrange(_size145):
+              _key150 = GlobalStreamId()
+              _key150.read(iprot)
+              _val151 = iprot.readI64();
+              _val144[_key150] = _val151
             iprot.readMapEnd()
-            self.executed[_key134] = _val135
+            self.executed[_key143] = _val144
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.MAP:
           self.execute_ms_avg = {}
-          (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin()
-          for _i147 in xrange(_size143):
-            _key148 = iprot.readString().decode('utf-8')
-            _val149 = {}
-            (_ktype151, _vtype152, _size150 ) = iprot.readMapBegin()
-            for _i154 in xrange(_size150):
-              _key155 = GlobalStreamId()
-              _key155.read(iprot)
-              _val156 = iprot.readDouble();
-              _val149[_key155] = _val156
+          (_ktype153, _vtype154, _size152 ) = iprot.readMapBegin()
+          for _i156 in xrange(_size152):
+            _key157 = iprot.readString().decode('utf-8')
+            _val158 = {}
+            (_ktype160, _vtype161, _size159 ) = iprot.readMapBegin()
+            for _i163 in xrange(_size159):
+              _key164 = GlobalStreamId()
+              _key164.read(iprot)
+              _val165 = iprot.readDouble();
+              _val158[_key164] = _val165
             iprot.readMapEnd()
-            self.execute_ms_avg[_key148] = _val149
+            self.execute_ms_avg[_key157] = _val158
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3029,60 +3052,60 @@ class BoltStats:
     if self.acked is not None:
       oprot.writeFieldBegin('acked', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked))
-      for kiter157,viter158 in self.acked.items():
-        oprot.writeString(kiter157.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter158))
-        for kiter159,viter160 in viter158.items():
-          kiter159.write(oprot)
-          oprot.writeI64(viter160)
+      for kiter166,viter167 in self.acked.items():
+        oprot.writeString(kiter166.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter167))
+        for kiter168,viter169 in viter167.items():
+          kiter168.write(oprot)
+          oprot.writeI64(viter169)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.failed is not None:
       oprot.writeFieldBegin('failed', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed))
-      for kiter161,viter162 in self.failed.items():
-        oprot.writeString(kiter161.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter162))
-        for kiter163,viter164 in viter162.items():
-          kiter163.write(oprot)
-          oprot.writeI64(viter164)
+      for kiter170,viter171 in self.failed.items():
+        oprot.writeString(kiter170.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter171))
+        for kiter172,viter173 in viter171.items():
+          kiter172.write(oprot)
+          oprot.writeI64(viter173)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.process_ms_avg is not None:
       oprot.writeFieldBegin('process_ms_avg', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.process_ms_avg))
-      for kiter165,viter166 in self.process_ms_avg.items():
-        oprot.writeString(kiter165.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter166))
-        for kiter167,viter168 in viter166.items():
-          kiter167.write(oprot)
-          oprot.writeDouble(viter168)
+      for kiter174,viter175 in self.process_ms_avg.items():
+        oprot.writeString(kiter174.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter175))
+        for kiter176,viter177 in viter175.items():
+          kiter176.write(oprot)
+          oprot.writeDouble(viter177)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executed is not None:
       oprot.writeFieldBegin('executed', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.executed))
-      for kiter169,viter170 in self.executed.items():
-        oprot.writeString(kiter169.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter170))
-        for kiter171,viter172 in viter170.items():
-          kiter171.write(oprot)
-          oprot.writeI64(viter172)
+      for kiter178,viter179 in self.executed.items():
+        oprot.writeString(kiter178.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter179))
+        for kiter180,viter181 in viter179.items():
+          kiter180.write(oprot)
+          oprot.writeI64(viter181)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.execute_ms_avg is not None:
       oprot.writeFieldBegin('execute_ms_avg', TType.MAP, 5)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.execute_ms_avg))
-      for kiter173,viter174 in self.execute_ms_avg.items():
-        oprot.writeString(kiter173.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter174))
-        for kiter175,viter176 in viter174.items():
-          kiter175.write(oprot)
-          oprot.writeDouble(viter176)
+      for kiter182,viter183 in self.execute_ms_avg.items():
+        oprot.writeString(kiter182.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter183))
+        for kiter184,viter185 in viter183.items():
+          kiter184.write(oprot)
+          oprot.writeDouble(viter185)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -3155,51 +3178,51 @@ class SpoutStats:
       if fid == 1:
         if ftype == TType.MAP:
           self.acked = {}
-          (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin()
-          for _i181 in xrange(_size177):
-            _key182 = iprot.readString().decode('utf-8')
-            _val183 = {}
-            (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin()
-            for _i188 in xrange(_size184):
-              _key189 = iprot.readString().decode('utf-8')
-              _val190 = iprot.readI64();
-              _val183[_key189] = _val190
+          (_ktype187, _vtype188, _size186 ) = iprot.readMapBegin()
+          for _i190 in xrange(_size186):
+            _key191 = iprot.readString().decode('utf-8')
+            _val192 = {}
+            (_ktype194, _vtype195, _size193 ) = iprot.readMapBegin()
+            for _i197 in xrange(_size193):
+              _key198 = iprot.readString().decode('utf-8')
+              _val199 = iprot.readI64();
+              _val192[_key198] = _val199
             iprot.readMapEnd()
-            self.acked[_key182] = _val183
+            self.acked[_key191] = _val192
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.failed = {}
-          (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin()
-          for _i195 in xrange(_size191):
-            _key196 = iprot.readString().decode('utf-8')
-            _val197 = {}
-            (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin()
-            for _i202 in xrange(_size198):
-              _key203 = iprot.readString().decode('utf-8')
-              _val204 = iprot.readI64();
-              _val197[_key203] = _val204
+          (_ktype201, _vtype202, _size200 ) = iprot.readMapBegin()
+          for _i204 in xrange(_size200):
+            _key205 = iprot.readString().decode('utf-8')
+            _val206 = {}
+            (_ktype208, _vtype209, _size207 ) = iprot.readMapBegin()
+            for _i211 in xrange(_size207):
+              _key212 = iprot.readString().decode('utf-8')
+              _val213 = iprot.readI64();
+              _val206[_key212] = _val213
             iprot.readMapEnd()
-            self.failed[_key196] = _val197
+            self.failed[_key205] = _val206
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.complete_ms_avg = {}
-          (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin()
-          for _i209 in xrange(_size205):
-            _key210 = iprot.readString().decode('utf-8')
-            _val211 = {}
-            (_ktype213, _vtype214, _size212 ) = iprot.readMapBegin()
-            for _i216 in xrange(_size212):
-              _key217 = iprot.readString().decode('utf-8')
-              _val218 = iprot.readDouble();
-              _val211[_key217] = _val218
+          (_ktype215, _vtype216, _size214 ) = iprot.readMapBegin()
+          for _i218 in xrange(_size214):
+            _key219 = iprot.readString().decode('utf-8')
+            _val220 = {}
+            (_ktype222, _vtype223, _size221 ) = iprot.readMapBegin()
+            for _i225 in xrange(_size221):
+              _key226 = iprot.readString().decode('utf-8')
+              _val227 = iprot.readDouble();
+              _val220[_key226] = _val227
             iprot.readMapEnd()
-            self.complete_ms_avg[_key210] = _val211
+            self.complete_ms_avg[_key219] = _val220
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3216,36 +3239,36 @@ class SpoutStats:
     if self.acked is not None:
       oprot.writeFieldBegin('acked', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked))
-      for kiter219,viter220 in self.acked.items():
-        oprot.writeString(kiter219.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter220))
-        for kiter221,viter222 in viter220.items():
-          oprot.writeString(kiter221.encode('utf-8'))
-          oprot.writeI64(viter222)
+      for kiter228,viter229 in self.acked.items():
+        oprot.writeString(kiter228.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter229))
+        for kiter230,viter231 in viter229.items():
+          oprot.writeString(kiter230.encode('utf-8'))
+          oprot.writeI64(viter231)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.failed is not None:
       oprot.writeFieldBegin('failed', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed))
-      for kiter223,viter224 in self.failed.items():
-        oprot.writeString(kiter223.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter224))
-        for kiter225,viter226 in viter224.items():
-          oprot.writeString(kiter225.encode('utf-8'))
-          oprot.writeI64(viter226)
+      for kiter232,viter233 in self.failed.items():
+        oprot.writeString(kiter232.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter233))
+        for kiter234,viter235 in viter233.items():
+          oprot.writeString(kiter234.encode('utf-8'))
+          oprot.writeI64(viter235)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.complete_ms_avg is not None:
       oprot.writeFieldBegin('complete_ms_avg', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.complete_ms_avg))
-      for kiter227,viter228 in self.complete_ms_avg.items():
-        oprot.writeString(kiter227.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter228))
-        for kiter229,viter230 in viter228.items():
-          oprot.writeString(kiter229.encode('utf-8'))
-          oprot.writeDouble(viter230)
+      for kiter236,viter237 in self.complete_ms_avg.items():
+        oprot.writeString(kiter236.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter237))
+        for kiter238,viter239 in viter237.items():
+          oprot.writeString(kiter238.encode('utf-8'))
+          oprot.writeDouble(viter239)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -3395,34 +3418,34 @@ class ExecutorStats:
       if fid == 1:
         if ftype == TType.MAP:
           self.emitted = {}
-          (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin()
-          for _i235 in xrange(_size231):
-            _key236 = iprot.readString().decode('utf-8')
-            _val237 = {}
-            (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin()
-            for _i242 in xrange(_size238):
-              _key243 = iprot.readString().decode('utf-8')
-              _val244 = iprot.readI64();
-              _val237[_key243] = _val244
+          (_ktype241, _vtype242, _size240 ) = iprot.readMapBegin()
+          for _i244 in xrange(_size240):
+            _key245 = iprot.readString().decode('utf-8')
+            _val246 = {}
+            (_ktype248, _vtype249, _size247 ) = iprot.readMapBegin()
+            for _i251 in xrange(_size247):
+              _key252 = iprot.readString().decode('utf-8')
+              _val253 = iprot.readI64();
+              _val246[_key252] = _val253
             iprot.readMapEnd()
-            self.emitted[_key236] = _val237
+            self.emitted[_key245] = _val246
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.transferred = {}
-          (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin()
-          for _i249 in xrange(_size245):
-            _key250 = iprot.readString().decode('utf-8')
-            _val251 = {}
-            (_ktype253, _vtype254, _size252 ) = iprot.readMapBegin()
-            for _i256 in xrange(_size252):
-              _key257 = iprot.readString().decode('utf-8')
-              _val258 = iprot.readI64();
-              _val251[_key257] = _val258
+          (_ktype255, _vtype256, _size254 ) = iprot.readMapBegin()
+          for _i258 in xrange(_size254):
+            _key259 = iprot.readString().decode('utf-8')
+            _val260 = {}
+            (_ktype262, _vtype263, _size261 ) = iprot.readMapBegin()
+            for _i265 in xrange(_size261):
+              _key266 = iprot.readString().decode('utf-8')
+              _val267 = iprot.readI64();
+              _val260[_key266] = _val267
             iprot.readMapEnd()
-            self.transferred[_key250] = _val251
+            self.transferred[_key259] = _val260
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3450,24 +3473,24 @@ class ExecutorStats:
     if self.emitted is not None:
       oprot.writeFieldBegin('emitted', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.emitted))
-      for kiter259,viter260 in self.emitted.items():
-        oprot.writeString(kiter259.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter260))
-        for kiter261,viter262 in viter260.items():
-          oprot.writeString(kiter261.encode('utf-8'))
-          oprot.writeI64(viter262)
+      for kiter268,viter269 in self.emitted.items():
+        oprot.writeString(kiter268.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter269))
+        for kiter270,viter271 in viter269.items():
+          oprot.writeString(kiter270.encode('utf-8'))
+          oprot.writeI64(viter271)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.transferred is not None:
       oprot.writeFieldBegin('transferred', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.transferred))
-      for kiter263,viter264 in self.transferred.items():
-        oprot.writeString(kiter263.encode('utf-8'))
-        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter264))
-        for kiter265,viter266 in viter264.items():
-          oprot.writeString(kiter265.encode('utf-8'))
-          oprot.writeI64(viter266)
+      for kiter272,viter273 in self.transferred.items():
+        oprot.writeString(kiter272.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter273))
+        for kiter274,viter275 in viter273.items():
+          oprot.writeString(kiter274.encode('utf-8'))
+          oprot.writeI64(viter275)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -4311,11 +4334,11 @@ class TopologyInfo:
       elif fid == 4:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype270, _size267) = iprot.readListBegin()
-          for _i271 in xrange(_size267):
-            _elem272 = ExecutorSummary()
-            _elem272.read(iprot)
-            self.executors.append(_elem272)
+          (_etype279, _size276) = iprot.readListBegin()
+          for _i280 in xrange(_size276):
+            _elem281 = ExecutorSummary()
+            _elem281.read(iprot)
+            self.executors.append(_elem281)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4327,29 +4350,29 @@ class TopologyInfo:
       elif fid == 6:
         if ftype == TType.MAP:
           self.errors = {}
-          (_ktype274, _vtype275, _size273 ) = iprot.readMapBegin()
-          for _i277 in xrange(_size273):
-            _key278 = iprot.readString().decode('utf-8')
-            _val279 = []
-            (_etype283, _size280) = iprot.readListBegin()
-            for _i284 in xrange(_size280):
-              _elem285 = ErrorInfo()
-              _elem285.read(iprot)
-              _val279.append(_elem285)
+          (_ktype283, _vtype284, _size282 ) = iprot.readMapBegin()
+          for _i286 in xrange(_size282):
+            _key287 = iprot.readString().decode('utf-8')
+            _val288 = []
+            (_etype292, _size289) = iprot.readListBegin()
+            for _i293 in xrange(_size289):
+              _elem294 = ErrorInfo()
+              _elem294.read(iprot)
+              _val288.append(_elem294)
             iprot.readListEnd()
-            self.errors[_key278] = _val279
+            self.errors[_key287] = _val288
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.MAP:
           self.component_debug = {}
-          (_ktype287, _vtype288, _size286 ) = iprot.readMapBegin()
-          for _i290 in xrange(_size286):
-            _key291 = iprot.readString().decode('utf-8')
-            _val292 = DebugOptions()
-            _val292.read(iprot)
-            self.component_debug[_key291] = _val292
+          (_ktype296, _vtype297, _size295 ) = iprot.readMapBegin()
+          for _i299 in xrange(_size295):
+            _key300 = iprot.readString().decode('utf-8')
+            _val301 = DebugOptions()
+            _val301.read(iprot)
+            self.component_debug[_key300] = _val301
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4393,8 +4416,8 @@ class TopologyInfo:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter293 in self.executors:
-        iter293.write(oprot)
+      for iter302 in self.executors:
+        iter302.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.status is not None:
@@ -4404,20 +4427,20 @@ class TopologyInfo:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.LIST, len(self.errors))
-      for kiter294,viter295 in self.errors.items():
-        oprot.writeString(kiter294.encode('utf-8'))
-        oprot.writeListBegin(TType.STRUCT, len(viter295))
-        for iter296 in viter295:
-          iter296.write(oprot)
+      for kiter303,viter304 in self.errors.items():
+        oprot.writeString(kiter303.encode('utf-8'))
+        oprot.writeListBegin(TType.STRUCT, len(viter304))
+        for iter305 in viter304:
+          iter305.write(oprot)
         oprot.writeListEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 7)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
-      for kiter297,viter298 in self.component_debug.items():
-        oprot.writeString(kiter297.encode('utf-8'))
-        viter298.write(oprot)
+      for kiter306,viter307 in self.component_debug.items():
+        oprot.writeString(kiter306.encode('utf-8'))
+        viter307.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.sched_status is not None:
@@ -5078,55 +5101,55 @@ class TopologyStats:
       if fid == 1:
         if ftype == TType.MAP:
           self.window_to_emitted = {}
-          (_ktype300, _vtype301, _size299 ) = iprot.readMapBegin()
-          for _i303 in xrange(_size299):
-            _key304 = iprot.readString().decode('utf-8')
-            _val305 = iprot.readI64();
-            self.window_to_emitted[_key304] = _val305
+          (_ktype309, _vtype310, _size308 ) = iprot.readMapBegin()
+          for _i312 in xrange(_size308):
+            _key313 = iprot.readString().decode('utf-8')
+            _val314 = iprot.readI64();
+            self.window_to_emitted[_key313] = _val314
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.MAP:
           self.window_to_transferred = {}
-          (_ktype307, _vtype308, _size306 ) = iprot.readMapBegin()
-          for _i310 in xrange(_size306):
-            _key311 = iprot.readString().decode('utf-8')
-            _val312 = iprot.readI64();
-            self.window_to_transferred[_key311] = _val312
+          (_ktype316, _vtype317, _size315 ) = iprot.readMapBegin()
+          for _i319 in xrange(_size315):
+            _key320 = iprot.readString().decode('utf-8')
+            _val321 = iprot.readI64();
+            self.window_to_transferred[_key320] = _val321
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.window_to_complete_latencies_ms = {}
-          (_ktype314, _vtype315, _size313 ) = iprot.readMapBegin()
-          for _i317 in xrange(_size313):
-            _key318 = iprot.readString().decode('utf-8')
-            _val319 = iprot.readDouble();
-            self.window_to_complete_latencies_ms[_key318] = _val319
+          (_ktype323, _vtype324, _size322 ) = iprot.readMapBegin()
+          for _i326 in xrange(_size322):
+            _key327 = iprot.readString().decode('utf-8')
+            _val328 = iprot.readDouble();
+            self.window_to_complete_latencies_ms[_key327] = _val328
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.window_to_acked = {}
-          (_ktype321, _vtype322, _size320 ) = iprot.readMapBegin()
-          for _i324 in xrange(_size320):
-            _key325 = iprot.readString().decode('utf-8')
-            _val326 = iprot.readI64();
-            self.window_to_acked[_key325] = _val326
+          (_ktype330, _vtype331, _size329 ) = iprot.readMapBegin()
+          for _i333 in xrange(_size329):
+            _key334 = iprot.readString().decode('utf-8')
+            _val335 = iprot.readI64();
+            self.window_to_acked[_key334] = _val335
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.MAP:
           self.window_to_failed = {}
-          (_ktype328, _vtype329, _size327 ) = iprot.readMapBegin()
-          for _i331 in xrange(_size327):
-            _key332 = iprot.readString().decode('utf-8')
-            _val333 = iprot.readI64();
-            self.window_to_failed[_key332] = _val333
+          (_ktype337, _vtype338, _size336 ) = iprot.readMapBegin()
+          for _i340 in xrange(_size336):
+            _key341 = iprot.readString().decode('utf-8')
+            _val342 = iprot.readI64();
+            self.window_to_failed[_key341] = _val342
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5143,41 +5166,41 @@ class TopologyStats:
     if self.window_to_emitted is not None:
       oprot.writeFieldBegin('window_to_emitted', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_emitted))
-      for kiter334,viter335 in self.window_to_emitted.items():
-        oprot.writeString(kiter334.encode('utf-8'))
-        oprot.writeI64(viter335)
+      for kiter343,viter344 in self.window_to_emitted.items():
+        oprot.writeString(kiter343.encode('utf-8'))
+        oprot.writeI64(viter344)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.window_to_transferred is not None:
       oprot.writeFieldBegin('window_to_transferred', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_transferred))
-      for kiter336,viter337 in self.window_to_transferred.items():
-        oprot.writeString(kiter336.encode('utf-8'))
-        oprot.writeI64(viter337)
+      for kiter345,viter346 in self.window_to_transferred.items():
+        oprot.writeString(kiter345.encode('utf-8'))
+        oprot.writeI64(viter346)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.window_to_complete_latencies_ms is not None:
       oprot.writeFieldBegin('window_to_complete_latencies_ms', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.window_to_complete_latencies_ms))
-      for kiter338,viter339 in self.window_to_complete_latencies_ms.items():
-        oprot.writeString(kiter338.encode('utf-8'))
-        oprot.writeDouble(viter339)
+      for kiter347,viter348 in self.window_to_complete_latencies_ms.items():
+        oprot.writeString(kiter347.encode('utf-8'))
+        oprot.writeDouble(viter348)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.window_to_acked is not None:
       oprot.writeFieldBegin('window_to_acked', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_acked))
-      for kiter340,viter341 in self.window_to_acked.items():
-        oprot.writeString(kiter340.encode('utf-8'))
-        oprot.writeI64(viter341)
+      for kiter349,viter350 in self.window_to_acked.items():
+        oprot.writeString(kiter349.encode('utf-8'))
+        oprot.writeI64(viter350)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.window_to_failed is not None:
       oprot.writeFieldBegin('window_to_failed', TType.MAP, 5)
       oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_failed))
-      for kiter342,viter343 in self.window_to_failed.items():
-        oprot.writeString(kiter342.encode('utf-8'))
-        oprot.writeI64(viter343)
+      for kiter351,viter352 in self.window_to_failed.items():
+        oprot.writeString(kiter351.encode('utf-8'))
+        oprot.writeI64(viter352)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5315,24 +5338,24 @@ class TopologyPageInfo:
       elif fid == 9:
         if ftype == TType.MAP:
           self.id_to_spout_agg_stats = {}
-          (_ktype345, _vtype346, _size344 ) = iprot.readMapBegin()
-          for _i348 in xrange(_size344):
-            _key349 = iprot.readString().decode('utf-8')
-            _val350 = ComponentAggregateStats()
-            _val350.read(iprot)
-            self.id_to_spout_agg_stats[_key349] = _val350
+          (_ktype354, _vtype355, _size353 ) = iprot.readMapBegin()
+          for _i357 in xrange(_size353):
+            _key358 = iprot.readString().decode('utf-8')
+            _val359 = ComponentAggregateStats()
+            _val359.read(iprot)
+            self.id_to_spout_agg_stats[_key358] = _val359
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 10:
         if ftype == TType.MAP:
           self.id_to_bolt_agg_stats = {}
-          (_ktype352, _vtype353, _size351 ) = iprot.readMapBegin()
-          for _i355 in xrange(_size351):
-            _key356 = iprot.readString().decode('utf-8')
-            _val357 = ComponentAggregateStats()
-            _val357.read(iprot)
-            self.id_to_bolt_agg_stats[_key356] = _val357
+          (_ktype361, _vtype362, _size360 ) = iprot.readMapBegin()
+          for _i364 in xrange(_size360):
+            _key365 = iprot.readString().decode('utf-8')
+            _val366 = ComponentAggregateStats()
+            _val366.read(iprot)
+            self.id_to_bolt_agg_stats[_key365] = _val366
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5408,17 +5431,17 @@ class TopologyPageInfo:
     if self.id_to_spout_agg_stats is not None:
       oprot.writeFieldBegin('id_to_spout_agg_stats', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_spout_agg_stats))
-      for kiter358,viter359 in self.id_to_spout_agg_stats.items():
-        oprot.writeString(kiter358.encode('utf-8'))
-        viter359.write(oprot)
+      for kiter367,viter368 in self.id_to_spout_agg_stats.items():
+        oprot.writeString(kiter367.encode('utf-8'))
+        viter368.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.id_to_bolt_agg_stats is not None:
       oprot.writeFieldBegin('id_to_bolt_agg_stats', TType.MAP, 10)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_bolt_agg_stats))
-      for kiter360,viter361 in self.id_to_bolt_agg_stats.items():
-        oprot.writeString(kiter360.encode('utf-8'))
-        viter361.write(oprot)
+      for kiter369,viter370 in self.id_to_bolt_agg_stats.items():
+        oprot.writeString(kiter369.encode('utf-8'))
+        viter370.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.sched_status is not None:
@@ -5658,59 +5681,59 @@ class ComponentPageInfo:
       elif fid == 7:
         if ftype == TType.MAP:
           self.window_to_stats = {}
-          (_ktype363, _vtype364, _size362 ) = iprot.readMapBegin()
-          for _i366 in xrange(_size362):
-            _key367 = iprot.readString().decode('utf-8')
-            _val368 = ComponentAggregateStats()
-            _val368.read(iprot)
-            self.window_to_stats[_key367] = _val368
+          (_ktype372, _vtype373, _size371 ) = iprot.readMapBegin()
+          for _i375 in xrange(_size371):
+            _key376 = iprot.readString().decode('utf-8')
+            _val377 = ComponentAggregateStats()
+            _val377.read(iprot)
+            self.window_to_stats[_key376] = _val377
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 8:
         if ftype == TType.MAP:
           self.gsid_to_input_stats = {}
-          (_ktype370, _vtype371, _size369 ) = iprot.readMapBegin()
-          for _i373 in xrange(_size369):
-            _key374 = GlobalStreamId()
-            _key374.read(iprot)
-            _val375 = ComponentAggregateStats()
-            _val375.read(iprot)
-            self.gsid_to_input_stats[_key374] = _val375
+          (_ktype379, _vtype380, _size378 ) = iprot.readMapBegin()
+          for _i382 in xrange(_size378):
+            _key383 = GlobalStreamId()
+            _key383.read(iprot)
+            _val384 = ComponentAggregateStats()
+            _val384.read(iprot)
+            self.gsid_to_input_stats[_key383] = _val384
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 9:
         if ftype == TType.MAP:
           self.sid_to_output_stats = {}
-          (_ktype377, _vtype378, _size376 ) = iprot.readMapBegin()
-          for _i380 in xrange(_size376):
-            _key381 = iprot.readString().decode('utf-8')
-            _val382 = ComponentAggregateStats()
-            _val382.read(iprot)
-            self.sid_to_output_stats[_key381] = _val382
+          (_ktype386, _vtype387, _size385 ) = iprot.readMapBegin()
+          for _i389 in xrange(_size385):
+            _key390 = iprot.readString().decode('utf-8')
+            _val391 = ComponentAggregateStats()
+            _val391.read(iprot)
+            self.sid_to_output_stats[_key390] = _val391
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 10:
         if ftype == TType.LIST:
           self.exec_stats = []
-          (_etype386, _size383) = iprot.readListBegin()
-          for _i387 in xrange(_size383):
-            _elem388 = ExecutorAggregateStats()
-            _elem388.read(iprot)
-            self.exec_stats.append(_elem388)
+          (_etype395, _size392) = iprot.readListBegin()
+          for _i396 in xrange(_size392):
+            _elem397 = ExecutorAggregateStats()
+            _elem397.read(iprot)
+            self.exec_stats.append(_elem397)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 11:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype392, _size389) = iprot.readListBegin()
-          for _i393 in xrange(_size389):
-            _elem394 = ErrorInfo()
-            _elem394.read(iprot)
-            self.errors.append(_elem394)
+          (_etype401, _size398) = iprot.readListBegin()
+          for _i402 in xrange(_size398):
+            _elem403 = ErrorInfo()
+            _elem403.read(iprot)
+            self.errors.append(_elem403)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -5772,39 +5795,39 @@ class ComponentPageInfo:
     if self.window_to_stats is not None:
       oprot.writeFieldBegin('window_to_stats', TType.MAP, 7)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.window_to_stats))
-      for kiter395,viter396 in self.window_to_stats.items():
-        oprot.writeString(kiter395.encode('utf-8'))
-        viter396.write(oprot)
+      for kiter404,viter405 in self.window_to_stats.items():
+        oprot.writeString(kiter404.encode('utf-8'))
+        viter405.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.gsid_to_input_stats is not None:
       oprot.writeFieldBegin('gsid_to_input_stats', TType.MAP, 8)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.gsid_to_input_stats))
-      for kiter397,viter398 in self.gsid_to_input_stats.items():
-        kiter397.write(oprot)
-        viter398.write(oprot)
+      for kiter406,viter407 in self.gsid_to_input_stats.items():
+        kiter406.write(oprot)
+        viter407.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.sid_to_output_stats is not None:
       oprot.writeFieldBegin('sid_to_output_stats', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.sid_to_output_stats))
-      for kiter399,viter400 in self.sid_to_output_stats.items():
-        oprot.writeString(kiter399.encode('utf-8'))
-        viter400.write(oprot)
+      for kiter408,viter409 in self.sid_to_output_stats.items():
+        oprot.writeString(kiter408.encode('utf-8'))
+        viter409.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.exec_stats is not None:
       oprot.writeFieldBegin('exec_stats', TType.LIST, 10)
       oprot.writeListBegin(TType.STRUCT, len(self.exec_stats))
-      for iter401 in self.exec_stats:
-        iter401.write(oprot)
+      for iter410 in self.exec_stats:
+        iter410.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 11)
       oprot.writeListBegin(TType.STRUCT, len(self.errors))
-      for iter402 in self.errors:
-        iter402.write(oprot)
+      for iter411 in self.errors:
+        iter411.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.eventlog_host is not None:
@@ -5971,11 +5994,11 @@ class RebalanceOptions:
       elif fid == 3:
         if ftype == TType.MAP:
           self.num_executors = {}
-          (_ktype404, _vtype405, _size403 ) = iprot.readMapBegin()
-          for _i407 in xrange(_size403):
-            _key408 = iprot.readString().decode('utf-8')
-            _val409 = iprot.readI32();
-            self.num_executors[_key408] = _val409
+          (_ktype413, _vtype414, _size412 ) = iprot.readMapBegin()
+          for _i416 in xrange(_size412):
+            _key417 = iprot.readString().decode('utf-8')
+            _val418 = iprot.readI32();
+            self.num_executors[_key417] = _val418
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6000,9 +6023,9 @@ class RebalanceOptions:
     if self.num_executors is not None:
       oprot.writeFieldBegin('num_executors', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.num_executors))
-      for kiter410,viter411 in self.num_executors.items():
-        oprot.writeString(kiter410.encode('utf-8'))
-        oprot.writeI32(viter411)
+      for kiter419,viter420 in self.num_executors.items():
+        oprot.writeString(kiter419.encode('utf-8'))
+        oprot.writeI32(viter420)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6056,11 +6079,11 @@ class Credentials:
       if fid == 1:
         if ftype == TType.MAP:
           self.creds = {}
-          (_ktype413, _vtype414, _size412 ) = iprot.readMapBegin()
-          for _i416 in xrange(_size412):
-            _key417 = iprot.readString().decode('utf-8')
-            _val418 = iprot.readString().decode('utf-8')
-            self.creds[_key417] = _val418
+          (_ktype422, _vtype423, _size421 ) = iprot.readMapBegin()
+          for _i425 in xrange(_size421):
+            _key426 = iprot.readString().decode('utf-8')
+            _val427 = iprot.readString().decode('utf-8')
+            self.creds[_key426] = _val427
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6077,9 +6100,9 @@ class Credentials:
     if self.creds is not None:
       oprot.writeFieldBegin('creds', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds))
-      for kiter419,viter420 in self.creds.items():
-        oprot.writeString(kiter419.encode('utf-8'))
-        oprot.writeString(viter420.encode('utf-8'))
+      for kiter428,viter429 in self.creds.items():
+        oprot.writeString(kiter428.encode('utf-8'))
+        oprot.writeString(viter429.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6199,6 +6222,7 @@ class SupervisorInfo:
    - scheduler_meta
    - uptime_secs
    - version
+   - resources_map
   """
 
   thrift_spec = (
@@ -6211,9 +6235,10 @@ class SupervisorInfo:
     (6, TType.MAP, 'scheduler_meta', (TType.STRING,None,TType.STRING,None), None, ), # 6
     (7, TType.I64, 'uptime_secs', None, None, ), # 7
     (8, TType.STRING, 'version', None, None, ), # 8
+    (9, TType.MAP, 'resources_map', (TType.STRING,None,TType.DOUBLE,None), None, ), # 9
   )
 
-  def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None, version=None,):
+  def __init__(self, time_secs=None, hostname=None, assignment_id=None, used_ports=None, meta=None, scheduler_meta=None, uptime_secs=None, version=None, resources_map=None,):
     self.time_secs = time_secs
     self.hostname = hostname
     self.assignment_id = assignment_id
@@ -6222,6 +6247,7 @@ class SupervisorInfo:
     self.scheduler_meta = scheduler_meta
     self.uptime_secs = uptime_secs
     self.version = version
+    self.resources_map = resources_map
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -6250,31 +6276,31 @@ class SupervisorInfo:
       elif fid == 4:
         if ftype == TType.LIST:
           self.used_ports = []
-          (_etype424, _size421) = iprot.readListBegin()
-          for _i425 in xrange(_size421):
-            _elem426 = iprot.readI64();
-            self.used_ports.append(_elem426)
+          (_etype433, _size430) = iprot.readListBegin()
+          for _i434 in xrange(_size430):
+            _elem435 = iprot.readI64();
+            self.used_ports.append(_elem435)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.meta = []
-          (_etype430, _size427) = iprot.readListBegin()
-          for _i431 in xrange(_size427):
-            _elem432 = iprot.readI64();
-            self.meta.append(_elem432)
+          (_etype439, _size436) = iprot.readListBegin()
+          for _i440 in xrange(_size436):
+            _elem441 = iprot.readI64();
+            self.meta.append(_elem441)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.MAP:
           self.scheduler_meta = {}
-          (_ktype434, _vtype435, _size433 ) = iprot.readMapBegin()
-          for _i437 in xrange(_size433):
-            _key438 = iprot.readString().decode('utf-8')
-            _val439 = iprot.readString().decode('utf-8')
-            self.scheduler_meta[_key438] = _val439
+          (_ktype443, _vtype444, _size442 ) = iprot.readMapBegin()
+          for _i446 in xrange(_size442):
+            _key447 = iprot.readString().decode('utf-8')
+            _val448 = iprot.readString().decode('utf-8')
+            self.scheduler_meta[_key447] = _val448
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6288,6 +6314,17 @@ class SupervisorInfo:
           self.version = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.MAP:
+          self.resources_map = {}
+          (_ktype450, _vtype451, _size449 ) = iprot.readMapBegin()
+          for _i453 in xrange(_size449):
+            _key454 = iprot.readString().decode('utf-8')
+            _val455 = iprot.readDouble();
+            self.resources_map[_key454] = _val455
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -6313,23 +6350,23 @@ class SupervisorInfo:
     if self.used_ports is not None:
       oprot.writeFieldBegin('used_ports', TType.LIST, 4)
       oprot.writeListBegin(TType.I64, len(self.used_ports))
-      for iter440 in self.used_ports:
-        oprot.writeI64(iter440)
+      for iter456 in self.used_ports:
+        oprot.writeI64(iter456)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.meta is not None:
       oprot.writeFieldBegin('meta', TType.LIST, 5)
       oprot.writeListBegin(TType.I64, len(self.meta))
-      for iter441 in self.meta:
-        oprot.writeI64(iter441)
+      for iter457 in self.meta:
+        oprot.writeI64(iter457)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.scheduler_meta is not None:
       oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta))
-      for kiter442,viter443 in self.scheduler_meta.items():
-        oprot.writeString(kiter442.encode('utf-8'))
-        oprot.writeString(viter443.encode('utf-8'))
+      for kiter458,viter459 in self.scheduler_meta.items():
+        oprot.writeString(kiter458.encode('utf-8'))
+        oprot.writeString(viter459.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.uptime_secs is not None:
@@ -6340,6 +6377,14 @@ class SupervisorInfo:
       oprot.writeFieldBegin('version', TType.STRING, 8)
       oprot.writeString(self.version.encode('utf-8'))
       oprot.writeFieldEnd()
+    if self.resources_map is not None:
+      oprot.writeFieldBegin('resources_map', TType.MAP, 9)
+      oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.resources_map))
+      for kiter460,viter461 in self.resources_map.items():
+        oprot.writeString(kiter460.encode('utf-8'))
+        oprot.writeDouble(viter461)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -6361,6 +6406,7 @@ class SupervisorInfo:
     value = (value * 31) ^ hash(self.scheduler_meta)
     value = (value * 31) ^ hash(self.uptime_secs)
     value = (value * 31) ^ hash(self.version)
+    value = (value * 31) ^ hash(self.resources_map)
     return value
 
   def __repr__(self):
@@ -6408,10 +6454,10 @@ class NodeInfo:
       elif fid == 2:
         if ftype == TType.SET:
           self.port = set()
-          (_etype447, _size444) = iprot.readSetBegin()
-          for _i448 in xrange(_size444):
-            _elem449 = iprot.readI64();
-            self.port.add(_elem449)
+          (_etype465, _size462) = iprot.readSetBegin()
+          for _i466 in xrange(_size462):
+            _elem467 = iprot.readI64();
+            self.port.add(_elem467)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -6432,8 +6478,8 @@ class NodeInfo:
     if self.port is not None:
       oprot.writeFieldBegin('port', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.port))
-      for iter450 in self.port:
-        oprot.writeI64(iter450)
+      for iter468 in self.port:
+        oprot.writeI64(iter468)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6516,44 +6562,44 @@ class Assignment:
       elif fid == 2:
         if ftype == TType.MAP:
           self.node_host = {}
-          (_ktype452, _vtype453, _size451 ) = iprot.readMapBegin()
-          for _i455 in xrange(_size451):
-            _key456 = iprot.readString().decode('utf-8')
-            _val457 = iprot.readString().decode('utf-8')
-            self.node_host[_key456] = _val457
+          (_ktype470, _vtype471, _size469 ) = iprot.readMapBegin()
+          for _i473 in xrange(_size469):
+            _key474 = iprot.readString().decode('utf-8')
+            _val475 = iprot.readString().decode('utf-8')
+            self.node_host[_key474] = _val475
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.executor_node_port = {}
-          (_ktype459, _vtype460, _size458 ) = iprot.readMapBegin()
-          for _i462 in xrange(_size458):
-            _key463 = []
-            (_etype468, _size465) = iprot.readListBegin()
-            for _i469 in xrange(_size465):
-              _elem470 = iprot.readI64();
-              _key463.append(_elem470)
+          (_ktype477, _vtype478, _size476 ) = iprot.readMapBegin()
+          for _i480 in xrange(_size476):
+            _key481 = []
+            (_etype486, _size483) = iprot.readListBegin()
+            for _i487 in xrange(_size483):
+              _elem488 = iprot.readI64();
+              _key481.append(_elem488)
             iprot.readListEnd()
-            _val464 = NodeInfo()
-            _val464.read(iprot)
-            self.executor_node_port[_key463] = _val464
+            _val482 = NodeInfo()
+            _val482.read(iprot)
+            self.executor_node_port[_key481] = _val482
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.executor_start_time_secs = {}
-          (_ktype472, _vtype473, _size471 ) = iprot.readMapBegin()
-          for _i475 in xrange(_size471):
-            _key476 = []
-            (_etype481, _size478) = iprot.readListBegin()
-            for _i482 in xrange(_size478):
-              _elem483 = iprot.readI64();
-              _key476.append(_elem483)
+          (_ktype490, _vtype491, _size489 ) = iprot.readMapBegin()
+          for _i493 in xrange(_size489):
+            _key494 = []
+            (_etype499, _size496) = iprot.readListBegin()
+            for _i500 in xrange(_size496):
+              _elem501 = iprot.readI64();
+              _key494.append(_elem501)
             iprot.readListEnd()
-            _val477 = iprot.readI64();
-            self.executor_start_time_secs[_key476] = _val477
+            _val495 = iprot.readI64();
+            self.executor_start_time_secs[_key494] = _val495
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6574,31 +6620,31 @@ class Assignment:
     if self.node_host is not None:
       oprot.writeFieldBegin('node_host', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.node_host))
-      for kiter484,viter485 in self.node_host.items():
-        oprot.writeString(kiter484.encode('utf-8'))
-        oprot.writeString(viter485.encode('utf-8'))
+      for kiter502,viter503 in self.node_host.items():
+        oprot.writeString(kiter502.encode('utf-8'))
+        oprot.writeString(viter503.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_node_port is not None:
       oprot.writeFieldBegin('executor_node_port', TType.MAP, 3)
       oprot.writeMapBegin(TType.LIST, TType.STRUCT, len(self.executor_node_port))
-      for kiter486,viter487 in self.executor_node_port.items():
-        oprot.writeListBegin(TType.I64, len(kiter486))
-        for iter488 in kiter486:
-          oprot.writeI64(iter488)
+      for kiter504,viter505 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter504))
+        for iter506 in kiter504:
+          oprot.writeI64(iter506)
         oprot.writeListEnd()
-        viter487.write(oprot)
+        viter505.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_start_time_secs is not None:
       oprot.writeFieldBegin('executor_start_time_secs', TType.MAP, 4)
       oprot.writeMapBegin(TType.LIST, TType.I64, len(self.executor_start_time_secs))
-      for kiter489,viter490 in self.executor_start_time_secs.items():
-        oprot.writeListBegin(TType.I64, len(kiter489))
-        for iter491 in kiter489:
-          oprot.writeI64(iter491)
+      for kiter507,viter508 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter507))
+        for iter509 in kiter507:
+          oprot.writeI64(iter509)
         oprot.writeListEnd()
-        oprot.writeI64(viter490)
+        oprot.writeI64(viter508)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6774,11 +6820,11 @@ class StormBase:
       elif fid == 4:
         if ftype == TType.MAP:
           self.component_executors = {}
-          (_ktype493, _vtype494, _size492 ) = iprot.readMapBegin()
-          for _i496 in xrange(_size492):
-            _key497 = iprot.readString().decode('utf-8')
-            _val498 = iprot.readI32();
-            self.component_executors[_key497] = _val498
+          (_ktype511, _vtype512, _size510 ) = iprot.readMapBegin()
+          for _i514 in xrange(_size510):
+            _key515 = iprot.readString().decode('utf-8')
+            _val516 = iprot.readI32();
+            self.component_executors[_key515] = _val516
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6806,12 +6852,12 @@ class StormBase:
       elif fid == 9:
         if ftype == TType.MAP:
           self.component_debug = {}
-          (_ktype500, _vtype501, _size499 ) = iprot.readMapBegin()
-          for _i503 in xrange(_size499):
-            _key504 = iprot.readString().decode('utf-8')
-            _val505 = DebugOptions()
-            _val505.read(iprot)
-            self.component_debug[_key504] = _val505
+          (_ktype518, _vtype519, _size517 ) = iprot.readMapBegin()
+          for _i521 in xrange(_size517):
+            _key522 = iprot.readString().decode('utf-8')
+            _val523 = DebugOptions()
+            _val523.read(iprot)
+            self.component_debug[_key522] = _val523
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6840,9 +6886,9 @@ class StormBase:
     if self.component_executors is not None:
       oprot.writeFieldBegin('component_executors', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.component_executors))
-      for kiter506,viter507 in self.component_executors.items():
-        oprot.writeString(kiter506.encode('utf-8'))
-        oprot.writeI32(viter507)
+      for kiter524,viter525 in self.component_executors.items():
+        oprot.writeString(kiter524.encode('utf-8'))
+        oprot.writeI32(viter525)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.launch_time_secs is not None:
@@ -6864,9 +6910,9 @@ class StormBase:
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
-      for kiter508,viter509 in self.component_debug.items():
-        oprot.writeString(kiter508.encode('utf-8'))
-        viter509.write(oprot)
+      for kiter526,viter527 in self.component_debug.items():
+        oprot.writeString(kiter526.encode('utf-8'))
+        viter527.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6946,13 +6992,13 @@ class ClusterWorkerHeartbeat:
       elif fid == 2:
         if ftype == TType.MAP:
           self.executor_stats = {}
-          (_ktype511, _vtype512, _size510 ) = iprot.readMapBegin()
-          for _i514 in xrange(_size510):
-            _key515 = ExecutorInfo()
-            _key515.read(iprot)
-            _val516 = ExecutorStats()
-            _val516.read(iprot)
-            self.executor_stats[_key515] = _val516
+          (_ktype529, _vtype530, _size528 ) = iprot.readMapBegin()
+          for _i532 in xrange(_size528):
+            _key533 = ExecutorInfo()
+            _key533.read(iprot)
+            _val534 = ExecutorStats()
+            _val534.read(iprot)
+            self.executor_stats[_key533] = _val534
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6983,9 +7029,9 @@ class ClusterWorkerHeartbeat:
     if self.executor_stats is not None:
       oprot.writeFieldBegin('executor_stats', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.executor_stats))
-      for kiter517,viter518 in self.executor_stats.items():
-        kiter517.write(oprot)
-        viter518.write(oprot)
+      for kiter535,viter536 in self.executor_stats.items():
+        kiter535.write(oprot)
+        viter536.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.time_secs is not None:
@@ -7138,12 +7184,12 @@ class LocalStateData:
       if fid == 1:
         if ftype == TType.MAP:
           self.serialized_parts = {}
-          (_ktype520, _vtype521, _size519 ) = iprot.readMapBegin()
-          for _i523 in xrange(_size519):
-            _key524 = iprot.readString().decode('utf-8')
-            _val525 = ThriftSerializedObject()
-            _val525.read(iprot)
-            self.serialized_parts[_key524] = _val525
+          (_ktype538, _vtype539, _size537 ) = iprot.readMapBegin()
+          for _i541 in xrange(_size537):
+            _key542 = iprot.readString().decode('utf-8')
+            _val543 = ThriftSerializedObject()
+            _val543.read(iprot)
+            self.serialized_parts[_key542] = _val543
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7160,9 +7206,9 @@ class LocalStateData:
     if self.serialized_parts is not None:
       oprot.writeFieldBegin('serialized_parts', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.serialized_parts))
-      for kiter526,viter527 in self.serialized_parts.items():
-        oprot.writeString(kiter526.encode('utf-8'))
-        viter527.write(oprot)
+      for kiter544,viter545 in self.serialized_parts.items():
+        oprot.writeString(kiter544.encode('utf-8'))
+        viter545.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7224,11 +7270,11 @@ class LocalAssignment:
       elif fid == 2:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype531, _size528) = iprot.readListBegin()
-          for _i532 in xrange(_size528):
-            _elem533 = ExecutorInfo()
-            _elem533.read(iprot)
-            self.executors.append(_elem533)
+          (_etype549, _size546) = iprot.readListBegin()
+          for _i550 in xrange(_size546):
+            _elem551 = ExecutorInfo()
+            _elem551.read(iprot)
+            self.executors.append(_elem551)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7249,8 +7295,8 @@ class LocalAssignment:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter534 in self.executors:
-        iter534.write(oprot)
+      for iter552 in self.executors:
+        iter552.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7374,11 +7420,11 @@ class LSApprovedWorkers:
       if fid == 1:
         if ftype == TType.MAP:
           self.approved_workers = {}
-          (_ktype536, _vtype537, _size535 ) = iprot.readMapBegin()
-          for _i539 in xrange(_size535):
-            _key540 = iprot.readString().decode('utf-8')
-            _val541 = iprot.readI32();
-            self.approved_workers[_key540] = _val541
+          (_ktype554, _vtype555, _size553 ) = iprot.readMapBegin()
+          for _i557 in xrange(_size553):
+            _key558 = iprot.readString().decode('utf-8')
+            _val559 = iprot.readI32();
+            self.approved_workers[_key558] = _val559
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7395,9 +7441,9 @@ class LSApprovedWorkers:
     if self.approved_workers is not None:
       oprot.writeFieldBegin('approved_workers', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.approved_workers))
-      for kiter542,viter543 in self.approved_workers.items():
-        oprot.writeString(kiter542.encode('utf-8'))
-        oprot.writeI32(viter543)
+      for kiter560,viter561 in self.approved_workers.items():
+        oprot.writeString(kiter560.encode('utf-8'))
+        oprot.writeI32(viter561)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7451,12 +7497,12 @@ class LSSupervisorAssignments:
       if fid == 1:
         if ftype == TType.MAP:
           self.assignments = {}
-          (_ktype545, _vtype546, _size544 ) = iprot.readMapBegin()
-          for _i548 in xrange(_size544):
-            _key549 = iprot.readI32();
-            _val550 = LocalAssignment()
-            _val550.read(iprot)
-            self.assignments[_key549] = _val550
+          (_ktype563, _vtype564, _size562 ) = iprot.readMapBegin()
+          for _i566 in xrange(_size562):
+            _key567 = iprot.readI32();
+            _val568 = LocalAssignment()
+            _val568.read(iprot)
+            self.assignments[_key567] = _val568
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7473,9 +7519,9 @@ class LSSupervisorAssignments:
     if self.assignments is not None:
       oprot.writeFieldBegin('assignments', TType.MAP, 1)
       oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments))
-      for kiter551,viter552 in self.assignments.items():
-        oprot.writeI32(kiter551)
-        viter552.write(oprot)
+      for kiter569,viter570 in self.assignments.items():
+        oprot.writeI32(kiter569)
+        viter570.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -7548,11 +7594,11 @@ class LSWorkerHeartbeat:
       elif fid == 3:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype556, _size553) = iprot.readListBegin()
-          for _i557 in xrange(_size553):
-            _elem558 = ExecutorInfo()
-            _elem558.read(iprot)
-            self.executors.append(_elem558)
+          (_etype574, _size571) = iprot.readListBegin()
+          for _i575 in xrange(_size571):
+            _elem576 = ExecutorInfo()
+            _elem576.read(iprot)
+            self.executors.append(_elem576)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -7582,8 +7628,8 @@ class LSWorkerHeartbeat:
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter559 in self.executors:
-        iter559.write(oprot)
+      for iter577 in self.executors:
+        iter577.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.port is not None:
@@ -7835,12 +7881,12 @@ class LogConfig:
       if fid == 2:
         if ftype == TType.MAP:
           self.named_logger_level = {}
-          (_ktype561, _vtype562, _size560 ) = iprot.readMapBegin()
-          for _i564 in xrange(_size560):
-            _key565 = iprot.readString().decode('utf-8')
-            _val566 = LogLevel()
-            _val566.read(iprot)
-            self.named_logger_level[_key565] = _val566
+          (_ktype579, _vtype580, _size578 ) = iprot.readMapBegin()
+          for _i582 in xrange(_size578):
+            _key583 = iprot.readString().decode('utf-8')
+            _val584 = LogLevel()
+            _val584.read(iprot)
+            self.named_logger_level[_key583] = _val584
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -7857,9 +7903,9 @@ class LogConfig:
     if self.named_logger_level is not None:
       oprot.writeFieldBegin('named_logger_level', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.named_logger_level))
-      for kiter567,viter568 in self.named_logger_level.items():
-        oprot.writeString(kiter567.encode('utf-8'))
-        viter568.write(oprot)
+      for kiter585,viter586 in self.named_logger_level.items():
+        oprot.writeString(kiter585.encode('utf-8'))
+        viter586.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/storm.thrift
----------------------------------------------------------------------
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index 296c3f0..1b507df 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -155,6 +155,7 @@ struct SupervisorSummary {
   4: required i32 num_used_workers;
   5: required string supervisor_id;
   6: optional string version = "VERSION_NOT_PROVIDED";
+  7: optional map<string, double> total_resources;
 }
 
 struct NimbusSummary {
@@ -356,6 +357,7 @@ struct SupervisorInfo {
     6: optional map<string, string> scheduler_meta;
     7: optional i64 uptime_secs;
     8: optional string version;
+    9: optional map<string, double> resources_map;
 }
 
 struct NodeInfo {

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/test/clj/backtype/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/cluster_test.clj b/storm-core/test/clj/backtype/storm/cluster_test.clj
index b413d43..c99ee49 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -264,8 +264,8 @@
   (with-inprocess-zookeeper zk-port
     (let [state1 (mk-storm-state zk-port)
           state2 (mk-storm-state zk-port)
-          supervisor-info1 (SupervisorInfo. 10 "hostname-1" "id1" [1 2] [] {} 1000 "0.9.2")
-          supervisor-info2 (SupervisorInfo. 10 "hostname-2" "id2" [1 2] [] {} 1000 "0.9.2")
+          supervisor-info1 (SupervisorInfo. 10 "hostname-1" "id1" [1 2] [] {} 1000 "0.9.2" nil)
+          supervisor-info2 (SupervisorInfo. 10 "hostname-2" "id2" [1 2] [] {} 1000 "0.9.2" nil)
           ]
       (is (= [] (.supervisors state1 nil)))
       (.supervisor-heartbeat! state2 "2" supervisor-info2)

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
index d347ed5..4e7e884 100644
--- a/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
+++ b/storm-core/test/clj/backtype/storm/scheduler/multitenant_scheduler_test.clj
@@ -42,7 +42,7 @@
 
 (deftest test-node
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)]
     (is (= 5 (.size node-map)))
     (let [node (.get node-map "super0")]
@@ -87,7 +87,7 @@
 
 (deftest test-free-pool
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )]
     ;; assign one node so it is not in the pool
@@ -125,7 +125,7 @@
 
 (deftest test-default-pool-simple
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        default-pool (DefaultPool. )
@@ -162,7 +162,7 @@
 
 (deftest test-default-pool-big-request
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        default-pool (DefaultPool. )
@@ -199,7 +199,7 @@
 
 (deftest test-default-pool-big-request-2
   (let [supers (gen-supervisors 1)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        default-pool (DefaultPool. )
@@ -241,7 +241,7 @@
 (deftest test-default-pool-full
   (let [supers (gen-supervisors 2) ;;make 2 supervisors but only schedule with one of them
        single-super {(ffirst supers) (second (first supers))}
-       single-cluster (Cluster. (nimbus/standalone-nimbus) single-super {})
+       single-cluster (Cluster. (nimbus/standalone-nimbus) single-super {} nil)
        executor1 (ed 1)
        executor2 (ed 2)
        executor3 (ed 3)
@@ -268,7 +268,7 @@
       (is (= "Running with fewer slots than requested (4/5)" (.get (.getStatusMap single-cluster) "topology1")))
     )
 
-    (let [cluster (Cluster. (nimbus/standalone-nimbus) supers (.getAssignments single-cluster))
+    (let [cluster (Cluster. (nimbus/standalone-nimbus) supers (.getAssignments single-cluster) nil)
          node-map (Node/getAllNodesFrom cluster)
          free-pool (FreePool. )
          default-pool (DefaultPool. )]
@@ -285,7 +285,7 @@
 
 (deftest test-default-pool-complex
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        default-pool (DefaultPool. )
@@ -367,7 +367,7 @@
 
 (deftest test-isolated-pool-simple
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        isolated-pool (IsolatedPool. 5)
@@ -411,7 +411,7 @@
 
 (deftest test-isolated-pool-big-ask
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        isolated-pool (IsolatedPool. 5)
@@ -455,7 +455,7 @@
 
 (deftest test-isolated-pool-complex
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        isolated-pool (IsolatedPool. 5)
@@ -559,7 +559,7 @@
 
 (deftest test-isolated-pool-complex-2
   (let [supers (gen-supervisors 5)
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        free-pool (FreePool. )
        ;;like before but now we can only hold 2 nodes max.  Don't go over
@@ -657,7 +657,7 @@
                                 ["bolt22" 10 20]
                                 ["bolt23" 20 30]
                                 ["bolt24" 30 40]]))
-       cluster (Cluster. (nimbus/standalone-nimbus) supers {})
+       cluster (Cluster. (nimbus/standalone-nimbus) supers {} nil)
        node-map (Node/getAllNodesFrom cluster)
        topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
        conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}}
@@ -696,7 +696,7 @@
                                                                                   (ExecutorDetails. 10 15) (WorkerSlot. "super0" 1)
                                                                                   (ExecutorDetails. 15 20) (WorkerSlot. "super0" 1)})
                                }
-        cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments)
+        cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil)
         node-map (Node/getAllNodesFrom cluster)
         topologies (Topologies. (to-top-map [topology1]))
         conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 5 "userB" 5}}
@@ -741,7 +741,7 @@
           worker-slot-with-multiple-assignments (WorkerSlot. "super1" 1)
           existing-assignments {"topology2" (SchedulerAssignmentImpl. "topology2" {(ExecutorDetails. 1 1) worker-slot-with-multiple-assignments})
                                 "topology3" (SchedulerAssignmentImpl. "topology3" {(ExecutorDetails. 2 2) worker-slot-with-multiple-assignments})}
-          cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments)
+          cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil)
           topologies (Topologies. (to-top-map [topology1 topology2 topology3]))
           conf {MULTITENANT-SCHEDULER-USER-POOLS {"userA" 2 "userB" 1}}
           scheduler (MultitenantScheduler.)]
@@ -769,7 +769,7 @@
           existing-assignments {"topology1"
                                 (SchedulerAssignmentImpl. "topology1"
                                   {(ExecutorDetails. 0 0) (WorkerSlot. "super0" port-not-reported-by-supervisor)})}
-          cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments)
+          cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil)
           topologies (Topologies. (to-top-map [topology1]))
           conf {}
           scheduler (MultitenantScheduler.)]
@@ -809,7 +809,7 @@
                                 (SchedulerAssignmentImpl. "topology2"
                                   {(ExecutorDetails. 4 4) worker-slot-with-multiple-assignments
                                    (ExecutorDetails. 5 5) (WorkerSlot. dead-supervisor port-not-reported-by-supervisor)})}
-          cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments)
+          cluster (Cluster. (nimbus/standalone-nimbus) supers existing-assignments nil)
           topologies (Topologies. (to-top-map [topology1 topology2]))
           conf {}
           scheduler (MultitenantScheduler.)]


[5/9] storm git commit: [STORM-893] - Resource Aware Scheduler implementation. [STORM-894] - Basic Resource Aware Scheduling implementation.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
index 1420db4..b9bfae2 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-5")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._Fields>, java.io.Serializable, Cloneable, Comparable<LogConfig> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LogConfig");
 
@@ -368,16 +368,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           case 2: // NAMED_LOGGER_LEVEL
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map626 = iprot.readMapBegin();
-                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map626.size);
-                String _key627;
-                LogLevel _val628;
-                for (int _i629 = 0; _i629 < _map626.size; ++_i629)
+                org.apache.thrift.protocol.TMap _map646 = iprot.readMapBegin();
+                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map646.size);
+                String _key647;
+                LogLevel _val648;
+                for (int _i649 = 0; _i649 < _map646.size; ++_i649)
                 {
-                  _key627 = iprot.readString();
-                  _val628 = new LogLevel();
-                  _val628.read(iprot);
-                  struct.named_logger_level.put(_key627, _val628);
+                  _key647 = iprot.readString();
+                  _val648 = new LogLevel();
+                  _val648.read(iprot);
+                  struct.named_logger_level.put(_key647, _val648);
                 }
                 iprot.readMapEnd();
               }
@@ -404,10 +404,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
           oprot.writeFieldBegin(NAMED_LOGGER_LEVEL_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.named_logger_level.size()));
-            for (Map.Entry<String, LogLevel> _iter630 : struct.named_logger_level.entrySet())
+            for (Map.Entry<String, LogLevel> _iter650 : struct.named_logger_level.entrySet())
             {
-              oprot.writeString(_iter630.getKey());
-              _iter630.getValue().write(oprot);
+              oprot.writeString(_iter650.getKey());
+              _iter650.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -439,10 +439,10 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       if (struct.is_set_named_logger_level()) {
         {
           oprot.writeI32(struct.named_logger_level.size());
-          for (Map.Entry<String, LogLevel> _iter631 : struct.named_logger_level.entrySet())
+          for (Map.Entry<String, LogLevel> _iter651 : struct.named_logger_level.entrySet())
           {
-            oprot.writeString(_iter631.getKey());
-            _iter631.getValue().write(oprot);
+            oprot.writeString(_iter651.getKey());
+            _iter651.getValue().write(oprot);
           }
         }
       }
@@ -454,16 +454,16 @@ public class LogConfig implements org.apache.thrift.TBase<LogConfig, LogConfig._
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map632 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map632.size);
-          String _key633;
-          LogLevel _val634;
-          for (int _i635 = 0; _i635 < _map632.size; ++_i635)
+          org.apache.thrift.protocol.TMap _map652 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.named_logger_level = new HashMap<String,LogLevel>(2*_map652.size);
+          String _key653;
+          LogLevel _val654;
+          for (int _i655 = 0; _i655 < _map652.size; ++_i655)
           {
-            _key633 = iprot.readString();
-            _val634 = new LogLevel();
-            _val634.read(iprot);
-            struct.named_logger_level.put(_key633, _val634);
+            _key653 = iprot.readString();
+            _val654 = new LogLevel();
+            _val654.read(iprot);
+            struct.named_logger_level.put(_key653, _val654);
           }
         }
         struct.set_named_logger_level_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogLevel.java b/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
index dba55d0..becf92c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-24")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class LogLevel implements org.apache.thrift.TBase<LogLevel, LogLevel._Fields>, java.io.Serializable, Cloneable, Comparable<LogLevel> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LogLevel");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index 4632059..9e28e38 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-5")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class Nimbus {
 
   public interface Iface {

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
index 55ee3b0..a2b5c1e 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<NodeInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo");
 
@@ -461,13 +461,13 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
           case 2: // PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set496 = iprot.readSetBegin();
-                struct.port = new HashSet<Long>(2*_set496.size);
-                long _elem497;
-                for (int _i498 = 0; _i498 < _set496.size; ++_i498)
+                org.apache.thrift.protocol.TSet _set516 = iprot.readSetBegin();
+                struct.port = new HashSet<Long>(2*_set516.size);
+                long _elem517;
+                for (int _i518 = 0; _i518 < _set516.size; ++_i518)
                 {
-                  _elem497 = iprot.readI64();
-                  struct.port.add(_elem497);
+                  _elem517 = iprot.readI64();
+                  struct.port.add(_elem517);
                 }
                 iprot.readSetEnd();
               }
@@ -498,9 +498,9 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
         oprot.writeFieldBegin(PORT_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size()));
-          for (long _iter499 : struct.port)
+          for (long _iter519 : struct.port)
           {
-            oprot.writeI64(_iter499);
+            oprot.writeI64(_iter519);
           }
           oprot.writeSetEnd();
         }
@@ -526,9 +526,9 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       oprot.writeString(struct.node);
       {
         oprot.writeI32(struct.port.size());
-        for (long _iter500 : struct.port)
+        for (long _iter520 : struct.port)
         {
-          oprot.writeI64(_iter500);
+          oprot.writeI64(_iter520);
         }
       }
     }
@@ -539,13 +539,13 @@ public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fie
       struct.node = iprot.readString();
       struct.set_node_isSet(true);
       {
-        org.apache.thrift.protocol.TSet _set501 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.port = new HashSet<Long>(2*_set501.size);
-        long _elem502;
-        for (int _i503 = 0; _i503 < _set501.size; ++_i503)
+        org.apache.thrift.protocol.TSet _set521 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.port = new HashSet<Long>(2*_set521.size);
+        long _elem522;
+        for (int _i523 = 0; _i523 < _set521.size; ++_i523)
         {
-          _elem502 = iprot.readI64();
-          struct.port.add(_elem502);
+          _elem522 = iprot.readI64();
+          struct.port.add(_elem522);
         }
       }
       struct.set_port_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
index 5764c1d..d49d4a9 100644
--- a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOptions, RebalanceOptions._Fields>, java.io.Serializable, Cloneable, Comparable<RebalanceOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RebalanceOptions");
 
@@ -529,15 +529,15 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
           case 3: // NUM_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map450 = iprot.readMapBegin();
-                struct.num_executors = new HashMap<String,Integer>(2*_map450.size);
-                String _key451;
-                int _val452;
-                for (int _i453 = 0; _i453 < _map450.size; ++_i453)
+                org.apache.thrift.protocol.TMap _map460 = iprot.readMapBegin();
+                struct.num_executors = new HashMap<String,Integer>(2*_map460.size);
+                String _key461;
+                int _val462;
+                for (int _i463 = 0; _i463 < _map460.size; ++_i463)
                 {
-                  _key451 = iprot.readString();
-                  _val452 = iprot.readI32();
-                  struct.num_executors.put(_key451, _val452);
+                  _key461 = iprot.readString();
+                  _val462 = iprot.readI32();
+                  struct.num_executors.put(_key461, _val462);
                 }
                 iprot.readMapEnd();
               }
@@ -574,10 +574,10 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
           oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.num_executors.size()));
-            for (Map.Entry<String, Integer> _iter454 : struct.num_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter464 : struct.num_executors.entrySet())
             {
-              oprot.writeString(_iter454.getKey());
-              oprot.writeI32(_iter454.getValue());
+              oprot.writeString(_iter464.getKey());
+              oprot.writeI32(_iter464.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -621,10 +621,10 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
       if (struct.is_set_num_executors()) {
         {
           oprot.writeI32(struct.num_executors.size());
-          for (Map.Entry<String, Integer> _iter455 : struct.num_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter465 : struct.num_executors.entrySet())
           {
-            oprot.writeString(_iter455.getKey());
-            oprot.writeI32(_iter455.getValue());
+            oprot.writeString(_iter465.getKey());
+            oprot.writeI32(_iter465.getValue());
           }
         }
       }
@@ -644,15 +644,15 @@ public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOption
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map456 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-          struct.num_executors = new HashMap<String,Integer>(2*_map456.size);
-          String _key457;
-          int _val458;
-          for (int _i459 = 0; _i459 < _map456.size; ++_i459)
+          org.apache.thrift.protocol.TMap _map466 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+          struct.num_executors = new HashMap<String,Integer>(2*_map466.size);
+          String _key467;
+          int _val468;
+          for (int _i469 = 0; _i469 < _map466.size; ++_i469)
           {
-            _key457 = iprot.readString();
-            _val458 = iprot.readI32();
-            struct.num_executors.put(_key457, _val458);
+            _key467 = iprot.readString();
+            _val468 = iprot.readI32();
+            struct.num_executors.put(_key467, _val468);
           }
         }
         struct.set_num_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
index cf5072c..a42f434 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class SpoutAggregateStats implements org.apache.thrift.TBase<SpoutAggregateStats, SpoutAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutAggregateStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutAggregateStats");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
index c18a7d0..2c39746 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
 public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStats._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutStats");
 
@@ -602,27 +602,27 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
           case 1: // ACKED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map198 = iprot.readMapBegin();
-                struct.acked = new HashMap<String,Map<String,Long>>(2*_map198.size);
-                String _key199;
-                Map<String,Long> _val200;
-                for (int _i201 = 0; _i201 < _map198.size; ++_i201)
+                org.apache.thrift.protocol.TMap _map208 = iprot.readMapBegin();
+                struct.acked = new HashMap<String,Map<String,Long>>(2*_map208.size);
+                String _key209;
+                Map<String,Long> _val210;
+                for (int _i211 = 0; _i211 < _map208.size; ++_i211)
                 {
-                  _key199 = iprot.readString();
+                  _key209 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map202 = iprot.readMapBegin();
-                    _val200 = new HashMap<String,Long>(2*_map202.size);
-                    String _key203;
-                    long _val204;
-                    for (int _i205 = 0; _i205 < _map202.size; ++_i205)
+                    org.apache.thrift.protocol.TMap _map212 = iprot.readMapBegin();
+                    _val210 = new HashMap<String,Long>(2*_map212.size);
+                    String _key213;
+                    long _val214;
+                    for (int _i215 = 0; _i215 < _map212.size; ++_i215)
                     {
-                      _key203 = iprot.readString();
-                      _val204 = iprot.readI64();
-                      _val200.put(_key203, _val204);
+                      _key213 = iprot.readString();
+                      _val214 = iprot.readI64();
+                      _val210.put(_key213, _val214);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.acked.put(_key199, _val200);
+                  struct.acked.put(_key209, _val210);
                 }
                 iprot.readMapEnd();
               }
@@ -634,27 +634,27 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
           case 2: // FAILED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map206 = iprot.readMapBegin();
-                struct.failed = new HashMap<String,Map<String,Long>>(2*_map206.size);
-                String _key207;
-                Map<String,Long> _val208;
-                for (int _i209 = 0; _i209 < _map206.size; ++_i209)
+                org.apache.thrift.protocol.TMap _map216 = iprot.readMapBegin();
+                struct.failed = new HashMap<String,Map<String,Long>>(2*_map216.size);
+                String _key217;
+                Map<String,Long> _val218;
+                for (int _i219 = 0; _i219 < _map216.size; ++_i219)
                 {
-                  _key207 = iprot.readString();
+                  _key217 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map210 = iprot.readMapBegin();
-                    _val208 = new HashMap<String,Long>(2*_map210.size);
-                    String _key211;
-                    long _val212;
-                    for (int _i213 = 0; _i213 < _map210.size; ++_i213)
+                    org.apache.thrift.protocol.TMap _map220 = iprot.readMapBegin();
+                    _val218 = new HashMap<String,Long>(2*_map220.size);
+                    String _key221;
+                    long _val222;
+                    for (int _i223 = 0; _i223 < _map220.size; ++_i223)
                     {
-                      _key211 = iprot.readString();
-                      _val212 = iprot.readI64();
-                      _val208.put(_key211, _val212);
+                      _key221 = iprot.readString();
+                      _val222 = iprot.readI64();
+                      _val218.put(_key221, _val222);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.failed.put(_key207, _val208);
+                  struct.failed.put(_key217, _val218);
                 }
                 iprot.readMapEnd();
               }
@@ -666,27 +666,27 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
           case 3: // COMPLETE_MS_AVG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map214 = iprot.readMapBegin();
-                struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map214.size);
-                String _key215;
-                Map<String,Double> _val216;
-                for (int _i217 = 0; _i217 < _map214.size; ++_i217)
+                org.apache.thrift.protocol.TMap _map224 = iprot.readMapBegin();
+                struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map224.size);
+                String _key225;
+                Map<String,Double> _val226;
+                for (int _i227 = 0; _i227 < _map224.size; ++_i227)
                 {
-                  _key215 = iprot.readString();
+                  _key225 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map218 = iprot.readMapBegin();
-                    _val216 = new HashMap<String,Double>(2*_map218.size);
-                    String _key219;
-                    double _val220;
-                    for (int _i221 = 0; _i221 < _map218.size; ++_i221)
+                    org.apache.thrift.protocol.TMap _map228 = iprot.readMapBegin();
+                    _val226 = new HashMap<String,Double>(2*_map228.size);
+                    String _key229;
+                    double _val230;
+                    for (int _i231 = 0; _i231 < _map228.size; ++_i231)
                     {
-                      _key219 = iprot.readString();
-                      _val220 = iprot.readDouble();
-                      _val216.put(_key219, _val220);
+                      _key229 = iprot.readString();
+                      _val230 = iprot.readDouble();
+                      _val226.put(_key229, _val230);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.complete_ms_avg.put(_key215, _val216);
+                  struct.complete_ms_avg.put(_key225, _val226);
                 }
                 iprot.readMapEnd();
               }
@@ -712,15 +712,15 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
         oprot.writeFieldBegin(ACKED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter222 : struct.acked.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter232 : struct.acked.entrySet())
           {
-            oprot.writeString(_iter222.getKey());
+            oprot.writeString(_iter232.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter222.getValue().size()));
-              for (Map.Entry<String, Long> _iter223 : _iter222.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter232.getValue().size()));
+              for (Map.Entry<String, Long> _iter233 : _iter232.getValue().entrySet())
               {
-                oprot.writeString(_iter223.getKey());
-                oprot.writeI64(_iter223.getValue());
+                oprot.writeString(_iter233.getKey());
+                oprot.writeI64(_iter233.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -733,15 +733,15 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
         oprot.writeFieldBegin(FAILED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter224 : struct.failed.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter234 : struct.failed.entrySet())
           {
-            oprot.writeString(_iter224.getKey());
+            oprot.writeString(_iter234.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter224.getValue().size()));
-              for (Map.Entry<String, Long> _iter225 : _iter224.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter234.getValue().size()));
+              for (Map.Entry<String, Long> _iter235 : _iter234.getValue().entrySet())
               {
-                oprot.writeString(_iter225.getKey());
-                oprot.writeI64(_iter225.getValue());
+                oprot.writeString(_iter235.getKey());
+                oprot.writeI64(_iter235.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -754,15 +754,15 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
         oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.complete_ms_avg.size()));
-          for (Map.Entry<String, Map<String,Double>> _iter226 : struct.complete_ms_avg.entrySet())
+          for (Map.Entry<String, Map<String,Double>> _iter236 : struct.complete_ms_avg.entrySet())
           {
-            oprot.writeString(_iter226.getKey());
+            oprot.writeString(_iter236.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, _iter226.getValue().size()));
-              for (Map.Entry<String, Double> _iter227 : _iter226.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, _iter236.getValue().size()));
+              for (Map.Entry<String, Double> _iter237 : _iter236.getValue().entrySet())
               {
-                oprot.writeString(_iter227.getKey());
-                oprot.writeDouble(_iter227.getValue());
+                oprot.writeString(_iter237.getKey());
+                oprot.writeDouble(_iter237.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -790,45 +790,45 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.acked.size());
-        for (Map.Entry<String, Map<String,Long>> _iter228 : struct.acked.entrySet())
+        for (Map.Entry<String, Map<String,Long>> _iter238 : struct.acked.entrySet())
         {
-          oprot.writeString(_iter228.getKey());
+          oprot.writeString(_iter238.getKey());
           {
-            oprot.writeI32(_iter228.getValue().size());
-            for (Map.Entry<String, Long> _iter229 : _iter228.getValue().entrySet())
+            oprot.writeI32(_iter238.getValue().size());
+            for (Map.Entry<String, Long> _iter239 : _iter238.getValue().entrySet())
             {
-              oprot.writeString(_iter229.getKey());
-              oprot.writeI64(_iter229.getValue());
+              oprot.writeString(_iter239.getKey());
+              oprot.writeI64(_iter239.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.failed.size());
-        for (Map.Entry<String, Map<String,Long>> _iter230 : struct.failed.entrySet())
+        for (Map.Entry<String, Map<String,Long>> _iter240 : struct.failed.entrySet())
         {
-          oprot.writeString(_iter230.getKey());
+          oprot.writeString(_iter240.getKey());
           {
-            oprot.writeI32(_iter230.getValue().size());
-            for (Map.Entry<String, Long> _iter231 : _iter230.getValue().entrySet())
+            oprot.writeI32(_iter240.getValue().size());
+            for (Map.Entry<String, Long> _iter241 : _iter240.getValue().entrySet())
             {
-              oprot.writeString(_iter231.getKey());
-              oprot.writeI64(_iter231.getValue());
+              oprot.writeString(_iter241.getKey());
+              oprot.writeI64(_iter241.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.complete_ms_avg.size());
-        for (Map.Entry<String, Map<String,Double>> _iter232 : struct.complete_ms_avg.entrySet())
+        for (Map.Entry<String, Map<String,Double>> _iter242 : struct.complete_ms_avg.entrySet())
         {
-          oprot.writeString(_iter232.getKey());
+          oprot.writeString(_iter242.getKey());
           {
-            oprot.writeI32(_iter232.getValue().size());
-            for (Map.Entry<String, Double> _iter233 : _iter232.getValue().entrySet())
+            oprot.writeI32(_iter242.getValue().size());
+            for (Map.Entry<String, Double> _iter243 : _iter242.getValue().entrySet())
             {
-              oprot.writeString(_iter233.getKey());
-              oprot.writeDouble(_iter233.getValue());
+              oprot.writeString(_iter243.getKey());
+              oprot.writeDouble(_iter243.getValue());
             }
           }
         }
@@ -839,74 +839,74 @@ public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStat
     public void read(org.apache.thrift.protocol.TProtocol prot, SpoutStats struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map234 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.acked = new HashMap<String,Map<String,Long>>(2*_map234.size);
-        String _key235;
-        Map<String,Long> _val236;
-        for (int _i237 = 0; _i237 < _map234.size; ++_i237)
+        org.apache.thrift.protocol.TMap _map244 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.acked = new HashMap<String,Map<String,Long>>(2*_map244.size);
+        String _key245;
+        Map<String,Long> _val246;
+        for (int _i247 = 0; _i247 < _map244.size; ++_i247)
         {
-          _key235 = iprot.readString();
+          _key245 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map238 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-            _val236 = new HashMap<String,Long>(2*_map238.size);
-            String _key239;
-            long _val240;
-            for (int _i241 = 0; _i241 < _map238.size; ++_i241)
+            org.apache.thrift.protocol.TMap _map248 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val246 = new HashMap<String,Long>(2*_map248.size);
+            String _key249;
+            long _val250;
+            for (int _i251 = 0; _i251 < _map248.size; ++_i251)
             {
-              _key239 = iprot.readString();
-              _val240 = iprot.readI64();
-              _val236.put(_key239, _val240);
+              _key249 = iprot.readString();
+              _val250 = iprot.readI64();
+              _val246.put(_key249, _val250);
             }
           }
-          struct.acked.put(_key235, _val236);
+          struct.acked.put(_key245, _val246);
         }
       }
       struct.set_acked_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map242 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.failed = new HashMap<String,Map<String,Long>>(2*_map242.size);
-        String _key243;
-        Map<String,Long> _val244;
-        for (int _i245 = 0; _i245 < _map242.size; ++_i245)
+        org.apache.thrift.protocol.TMap _map252 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.failed = new HashMap<String,Map<String,Long>>(2*_map252.size);
+        String _key253;
+        Map<String,Long> _val254;
+        for (int _i255 = 0; _i255 < _map252.size; ++_i255)
         {
-          _key243 = iprot.readString();
+          _key253 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map246 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-            _val244 = new HashMap<String,Long>(2*_map246.size);
-            String _key247;
-            long _val248;
-            for (int _i249 = 0; _i249 < _map246.size; ++_i249)
+            org.apache.thrift.protocol.TMap _map256 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val254 = new HashMap<String,Long>(2*_map256.size);
+            String _key257;
+            long _val258;
+            for (int _i259 = 0; _i259 < _map256.size; ++_i259)
             {
-              _key247 = iprot.readString();
-              _val248 = iprot.readI64();
-              _val244.put(_key247, _val248);
+              _key257 = iprot.readString();
+              _val258 = iprot.readI64();
+              _val254.put(_key257, _val258);
             }
           }
-          struct.failed.put(_key243, _val244);
+          struct.failed.put(_key253, _val254);
         }
       }
       struct.set_failed_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map250 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map250.size);
-        String _key251;
-        Map<String,Double> _val252;
-        for (int _i253 = 0; _i253 < _map250.size; ++_i253)
+        org.apache.thrift.protocol.TMap _map260 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map260.size);
+        String _key261;
+        Map<String,Double> _val262;
+        for (int _i263 = 0; _i263 < _map260.size; ++_i263)
         {
-          _key251 = iprot.readString();
+          _key261 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map254 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-            _val252 = new HashMap<String,Double>(2*_map254.size);
-            String _key255;
-            double _val256;
-            for (int _i257 = 0; _i257 < _map254.size; ++_i257)
+            org.apache.thrift.protocol.TMap _map264 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val262 = new HashMap<String,Double>(2*_map264.size);
+            String _key265;
+            double _val266;
+            for (int _i267 = 0; _i267 < _map264.size; ++_i267)
             {
-              _key255 = iprot.readString();
-              _val256 = iprot.readDouble();
-              _val252.put(_key255, _val256);
+              _key265 = iprot.readString();
+              _val266 = iprot.readDouble();
+              _val262.put(_key265, _val266);
             }
           }
-          struct.complete_ms_avg.put(_key251, _val252);
+          struct.complete_ms_avg.put(_key261, _val262);
         }
       }
       struct.set_complete_ms_avg_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
index b880bc3..f7ad22c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StormBase.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
 
@@ -1090,15 +1090,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 4: // COMPONENT_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map550 = iprot.readMapBegin();
-                struct.component_executors = new HashMap<String,Integer>(2*_map550.size);
-                String _key551;
-                int _val552;
-                for (int _i553 = 0; _i553 < _map550.size; ++_i553)
+                org.apache.thrift.protocol.TMap _map570 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map570.size);
+                String _key571;
+                int _val572;
+                for (int _i573 = 0; _i573 < _map570.size; ++_i573)
                 {
-                  _key551 = iprot.readString();
-                  _val552 = iprot.readI32();
-                  struct.component_executors.put(_key551, _val552);
+                  _key571 = iprot.readString();
+                  _val572 = iprot.readI32();
+                  struct.component_executors.put(_key571, _val572);
                 }
                 iprot.readMapEnd();
               }
@@ -1143,16 +1143,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           case 9: // COMPONENT_DEBUG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map554 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,DebugOptions>(2*_map554.size);
-                String _key555;
-                DebugOptions _val556;
-                for (int _i557 = 0; _i557 < _map554.size; ++_i557)
+                org.apache.thrift.protocol.TMap _map574 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map574.size);
+                String _key575;
+                DebugOptions _val576;
+                for (int _i577 = 0; _i577 < _map574.size; ++_i577)
                 {
-                  _key555 = iprot.readString();
-                  _val556 = new DebugOptions();
-                  _val556.read(iprot);
-                  struct.component_debug.put(_key555, _val556);
+                  _key575 = iprot.readString();
+                  _val576 = new DebugOptions();
+                  _val576.read(iprot);
+                  struct.component_debug.put(_key575, _val576);
                 }
                 iprot.readMapEnd();
               }
@@ -1192,10 +1192,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.component_executors.size()));
-            for (Map.Entry<String, Integer> _iter558 : struct.component_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter578 : struct.component_executors.entrySet())
             {
-              oprot.writeString(_iter558.getKey());
-              oprot.writeI32(_iter558.getValue());
+              oprot.writeString(_iter578.getKey());
+              oprot.writeI32(_iter578.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1233,10 +1233,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
           oprot.writeFieldBegin(COMPONENT_DEBUG_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.component_debug.size()));
-            for (Map.Entry<String, DebugOptions> _iter559 : struct.component_debug.entrySet())
+            for (Map.Entry<String, DebugOptions> _iter579 : struct.component_debug.entrySet())
             {
-              oprot.writeString(_iter559.getKey());
-              _iter559.getValue().write(oprot);
+              oprot.writeString(_iter579.getKey());
+              _iter579.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1286,10 +1286,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_executors()) {
         {
           oprot.writeI32(struct.component_executors.size());
-          for (Map.Entry<String, Integer> _iter560 : struct.component_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter580 : struct.component_executors.entrySet())
           {
-            oprot.writeString(_iter560.getKey());
-            oprot.writeI32(_iter560.getValue());
+            oprot.writeString(_iter580.getKey());
+            oprot.writeI32(_iter580.getValue());
           }
         }
       }
@@ -1308,10 +1308,10 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, DebugOptions> _iter561 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter581 : struct.component_debug.entrySet())
           {
-            oprot.writeString(_iter561.getKey());
-            _iter561.getValue().write(oprot);
+            oprot.writeString(_iter581.getKey());
+            _iter581.getValue().write(oprot);
           }
         }
       }
@@ -1329,15 +1329,15 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map562 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-          struct.component_executors = new HashMap<String,Integer>(2*_map562.size);
-          String _key563;
-          int _val564;
-          for (int _i565 = 0; _i565 < _map562.size; ++_i565)
+          org.apache.thrift.protocol.TMap _map582 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+          struct.component_executors = new HashMap<String,Integer>(2*_map582.size);
+          String _key583;
+          int _val584;
+          for (int _i585 = 0; _i585 < _map582.size; ++_i585)
           {
-            _key563 = iprot.readString();
-            _val564 = iprot.readI32();
-            struct.component_executors.put(_key563, _val564);
+            _key583 = iprot.readString();
+            _val584 = iprot.readI32();
+            struct.component_executors.put(_key583, _val584);
           }
         }
         struct.set_component_executors_isSet(true);
@@ -1361,16 +1361,16 @@ public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map566 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.component_debug = new HashMap<String,DebugOptions>(2*_map566.size);
-          String _key567;
-          DebugOptions _val568;
-          for (int _i569 = 0; _i569 < _map566.size; ++_i569)
+          org.apache.thrift.protocol.TMap _map586 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.component_debug = new HashMap<String,DebugOptions>(2*_map586.size);
+          String _key587;
+          DebugOptions _val588;
+          for (int _i589 = 0; _i589 < _map586.size; ++_i589)
           {
-            _key567 = iprot.readString();
-            _val568 = new DebugOptions();
-            _val568.read(iprot);
-            struct.component_debug.put(_key567, _val568);
+            _key587 = iprot.readString();
+            _val588 = new DebugOptions();
+            _val588.read(iprot);
+            struct.component_debug.put(_key587, _val588);
           }
         }
         struct.set_component_debug_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
index de48f99..6dc35be 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo");
 
@@ -63,6 +63,7 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   private static final org.apache.thrift.protocol.TField SCHEDULER_META_FIELD_DESC = new org.apache.thrift.protocol.TField("scheduler_meta", org.apache.thrift.protocol.TType.MAP, (short)6);
   private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I64, (short)7);
   private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField RESOURCES_MAP_FIELD_DESC = new org.apache.thrift.protocol.TField("resources_map", org.apache.thrift.protocol.TType.MAP, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -78,6 +79,7 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   private Map<String,String> scheduler_meta; // optional
   private long uptime_secs; // optional
   private String version; // optional
+  private Map<String,Double> resources_map; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -88,7 +90,8 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
     META((short)5, "meta"),
     SCHEDULER_META((short)6, "scheduler_meta"),
     UPTIME_SECS((short)7, "uptime_secs"),
-    VERSION((short)8, "version");
+    VERSION((short)8, "version"),
+    RESOURCES_MAP((short)9, "resources_map");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -119,6 +122,8 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           return UPTIME_SECS;
         case 8: // VERSION
           return VERSION;
+        case 9: // RESOURCES_MAP
+          return RESOURCES_MAP;
         default:
           return null;
       }
@@ -162,7 +167,7 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
   private static final int __TIME_SECS_ISSET_ID = 0;
   private static final int __UPTIME_SECS_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.ASSIGNMENT_ID,_Fields.USED_PORTS,_Fields.META,_Fields.SCHEDULER_META,_Fields.UPTIME_SECS,_Fields.VERSION};
+  private static final _Fields optionals[] = {_Fields.ASSIGNMENT_ID,_Fields.USED_PORTS,_Fields.META,_Fields.SCHEDULER_META,_Fields.UPTIME_SECS,_Fields.VERSION,_Fields.RESOURCES_MAP};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -186,6 +191,10 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.RESOURCES_MAP, new org.apache.thrift.meta_data.FieldMetaData("resources_map", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorInfo.class, metaDataMap);
   }
@@ -231,6 +240,10 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
     if (other.is_set_version()) {
       this.version = other.version;
     }
+    if (other.is_set_resources_map()) {
+      Map<String,Double> __this__resources_map = new HashMap<String,Double>(other.resources_map);
+      this.resources_map = __this__resources_map;
+    }
   }
 
   public SupervisorInfo deepCopy() {
@@ -249,6 +262,7 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
     set_uptime_secs_isSet(false);
     this.uptime_secs = 0;
     this.version = null;
+    this.resources_map = null;
   }
 
   public long get_time_secs() {
@@ -474,6 +488,40 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
     }
   }
 
+  public int get_resources_map_size() {
+    return (this.resources_map == null) ? 0 : this.resources_map.size();
+  }
+
+  public void put_to_resources_map(String key, double val) {
+    if (this.resources_map == null) {
+      this.resources_map = new HashMap<String,Double>();
+    }
+    this.resources_map.put(key, val);
+  }
+
+  public Map<String,Double> get_resources_map() {
+    return this.resources_map;
+  }
+
+  public void set_resources_map(Map<String,Double> resources_map) {
+    this.resources_map = resources_map;
+  }
+
+  public void unset_resources_map() {
+    this.resources_map = null;
+  }
+
+  /** Returns true if field resources_map is set (has been assigned a value) and false otherwise */
+  public boolean is_set_resources_map() {
+    return this.resources_map != null;
+  }
+
+  public void set_resources_map_isSet(boolean value) {
+    if (!value) {
+      this.resources_map = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TIME_SECS:
@@ -540,6 +588,14 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       }
       break;
 
+    case RESOURCES_MAP:
+      if (value == null) {
+        unset_resources_map();
+      } else {
+        set_resources_map((Map<String,Double>)value);
+      }
+      break;
+
     }
   }
 
@@ -569,6 +625,9 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
     case VERSION:
       return get_version();
 
+    case RESOURCES_MAP:
+      return get_resources_map();
+
     }
     throw new IllegalStateException();
   }
@@ -596,6 +655,8 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       return is_set_uptime_secs();
     case VERSION:
       return is_set_version();
+    case RESOURCES_MAP:
+      return is_set_resources_map();
     }
     throw new IllegalStateException();
   }
@@ -685,6 +746,15 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
         return false;
     }
 
+    boolean this_present_resources_map = true && this.is_set_resources_map();
+    boolean that_present_resources_map = true && that.is_set_resources_map();
+    if (this_present_resources_map || that_present_resources_map) {
+      if (!(this_present_resources_map && that_present_resources_map))
+        return false;
+      if (!this.resources_map.equals(that.resources_map))
+        return false;
+    }
+
     return true;
   }
 
@@ -732,6 +802,11 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
     if (present_version)
       list.add(version);
 
+    boolean present_resources_map = true && (is_set_resources_map());
+    list.add(present_resources_map);
+    if (present_resources_map)
+      list.add(resources_map);
+
     return list.hashCode();
   }
 
@@ -823,6 +898,16 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_resources_map()).compareTo(other.is_set_resources_map());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_resources_map()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resources_map, other.resources_map);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -910,6 +995,16 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       }
       first = false;
     }
+    if (is_set_resources_map()) {
+      if (!first) sb.append(", ");
+      sb.append("resources_map:");
+      if (this.resources_map == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.resources_map);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -990,13 +1085,13 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 4: // USED_PORTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list470 = iprot.readListBegin();
-                struct.used_ports = new ArrayList<Long>(_list470.size);
-                long _elem471;
-                for (int _i472 = 0; _i472 < _list470.size; ++_i472)
+                org.apache.thrift.protocol.TList _list480 = iprot.readListBegin();
+                struct.used_ports = new ArrayList<Long>(_list480.size);
+                long _elem481;
+                for (int _i482 = 0; _i482 < _list480.size; ++_i482)
                 {
-                  _elem471 = iprot.readI64();
-                  struct.used_ports.add(_elem471);
+                  _elem481 = iprot.readI64();
+                  struct.used_ports.add(_elem481);
                 }
                 iprot.readListEnd();
               }
@@ -1008,13 +1103,13 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 5: // META
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list473 = iprot.readListBegin();
-                struct.meta = new ArrayList<Long>(_list473.size);
-                long _elem474;
-                for (int _i475 = 0; _i475 < _list473.size; ++_i475)
+                org.apache.thrift.protocol.TList _list483 = iprot.readListBegin();
+                struct.meta = new ArrayList<Long>(_list483.size);
+                long _elem484;
+                for (int _i485 = 0; _i485 < _list483.size; ++_i485)
                 {
-                  _elem474 = iprot.readI64();
-                  struct.meta.add(_elem474);
+                  _elem484 = iprot.readI64();
+                  struct.meta.add(_elem484);
                 }
                 iprot.readListEnd();
               }
@@ -1026,15 +1121,15 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           case 6: // SCHEDULER_META
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map476 = iprot.readMapBegin();
-                struct.scheduler_meta = new HashMap<String,String>(2*_map476.size);
-                String _key477;
-                String _val478;
-                for (int _i479 = 0; _i479 < _map476.size; ++_i479)
+                org.apache.thrift.protocol.TMap _map486 = iprot.readMapBegin();
+                struct.scheduler_meta = new HashMap<String,String>(2*_map486.size);
+                String _key487;
+                String _val488;
+                for (int _i489 = 0; _i489 < _map486.size; ++_i489)
                 {
-                  _key477 = iprot.readString();
-                  _val478 = iprot.readString();
-                  struct.scheduler_meta.put(_key477, _val478);
+                  _key487 = iprot.readString();
+                  _val488 = iprot.readString();
+                  struct.scheduler_meta.put(_key487, _val488);
                 }
                 iprot.readMapEnd();
               }
@@ -1059,6 +1154,26 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 9: // RESOURCES_MAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map490 = iprot.readMapBegin();
+                struct.resources_map = new HashMap<String,Double>(2*_map490.size);
+                String _key491;
+                double _val492;
+                for (int _i493 = 0; _i493 < _map490.size; ++_i493)
+                {
+                  _key491 = iprot.readString();
+                  _val492 = iprot.readDouble();
+                  struct.resources_map.put(_key491, _val492);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_resources_map_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1092,9 +1207,9 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(USED_PORTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.used_ports.size()));
-            for (long _iter480 : struct.used_ports)
+            for (long _iter494 : struct.used_ports)
             {
-              oprot.writeI64(_iter480);
+              oprot.writeI64(_iter494);
             }
             oprot.writeListEnd();
           }
@@ -1106,9 +1221,9 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(META_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size()));
-            for (long _iter481 : struct.meta)
+            for (long _iter495 : struct.meta)
             {
-              oprot.writeI64(_iter481);
+              oprot.writeI64(_iter495);
             }
             oprot.writeListEnd();
           }
@@ -1120,10 +1235,10 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.scheduler_meta.size()));
-            for (Map.Entry<String, String> _iter482 : struct.scheduler_meta.entrySet())
+            for (Map.Entry<String, String> _iter496 : struct.scheduler_meta.entrySet())
             {
-              oprot.writeString(_iter482.getKey());
-              oprot.writeString(_iter482.getValue());
+              oprot.writeString(_iter496.getKey());
+              oprot.writeString(_iter496.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1142,6 +1257,21 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
           oprot.writeFieldEnd();
         }
       }
+      if (struct.resources_map != null) {
+        if (struct.is_set_resources_map()) {
+          oprot.writeFieldBegin(RESOURCES_MAP_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.resources_map.size()));
+            for (Map.Entry<String, Double> _iter497 : struct.resources_map.entrySet())
+            {
+              oprot.writeString(_iter497.getKey());
+              oprot.writeDouble(_iter497.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1180,35 +1310,38 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       if (struct.is_set_version()) {
         optionals.set(5);
       }
-      oprot.writeBitSet(optionals, 6);
+      if (struct.is_set_resources_map()) {
+        optionals.set(6);
+      }
+      oprot.writeBitSet(optionals, 7);
       if (struct.is_set_assignment_id()) {
         oprot.writeString(struct.assignment_id);
       }
       if (struct.is_set_used_ports()) {
         {
           oprot.writeI32(struct.used_ports.size());
-          for (long _iter483 : struct.used_ports)
+          for (long _iter498 : struct.used_ports)
           {
-            oprot.writeI64(_iter483);
+            oprot.writeI64(_iter498);
           }
         }
       }
       if (struct.is_set_meta()) {
         {
           oprot.writeI32(struct.meta.size());
-          for (long _iter484 : struct.meta)
+          for (long _iter499 : struct.meta)
           {
-            oprot.writeI64(_iter484);
+            oprot.writeI64(_iter499);
           }
         }
       }
       if (struct.is_set_scheduler_meta()) {
         {
           oprot.writeI32(struct.scheduler_meta.size());
-          for (Map.Entry<String, String> _iter485 : struct.scheduler_meta.entrySet())
+          for (Map.Entry<String, String> _iter500 : struct.scheduler_meta.entrySet())
           {
-            oprot.writeString(_iter485.getKey());
-            oprot.writeString(_iter485.getValue());
+            oprot.writeString(_iter500.getKey());
+            oprot.writeString(_iter500.getValue());
           }
         }
       }
@@ -1218,6 +1351,16 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       if (struct.is_set_version()) {
         oprot.writeString(struct.version);
       }
+      if (struct.is_set_resources_map()) {
+        {
+          oprot.writeI32(struct.resources_map.size());
+          for (Map.Entry<String, Double> _iter501 : struct.resources_map.entrySet())
+          {
+            oprot.writeString(_iter501.getKey());
+            oprot.writeDouble(_iter501.getValue());
+          }
+        }
+      }
     }
 
     @Override
@@ -1227,48 +1370,48 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
       struct.set_time_secs_isSet(true);
       struct.hostname = iprot.readString();
       struct.set_hostname_isSet(true);
-      BitSet incoming = iprot.readBitSet(6);
+      BitSet incoming = iprot.readBitSet(7);
       if (incoming.get(0)) {
         struct.assignment_id = iprot.readString();
         struct.set_assignment_id_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list486 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.used_ports = new ArrayList<Long>(_list486.size);
-          long _elem487;
-          for (int _i488 = 0; _i488 < _list486.size; ++_i488)
+          org.apache.thrift.protocol.TList _list502 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.used_ports = new ArrayList<Long>(_list502.size);
+          long _elem503;
+          for (int _i504 = 0; _i504 < _list502.size; ++_i504)
           {
-            _elem487 = iprot.readI64();
-            struct.used_ports.add(_elem487);
+            _elem503 = iprot.readI64();
+            struct.used_ports.add(_elem503);
           }
         }
         struct.set_used_ports_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list489 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.meta = new ArrayList<Long>(_list489.size);
-          long _elem490;
-          for (int _i491 = 0; _i491 < _list489.size; ++_i491)
+          org.apache.thrift.protocol.TList _list505 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.meta = new ArrayList<Long>(_list505.size);
+          long _elem506;
+          for (int _i507 = 0; _i507 < _list505.size; ++_i507)
           {
-            _elem490 = iprot.readI64();
-            struct.meta.add(_elem490);
+            _elem506 = iprot.readI64();
+            struct.meta.add(_elem506);
           }
         }
         struct.set_meta_isSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map492 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.scheduler_meta = new HashMap<String,String>(2*_map492.size);
-          String _key493;
-          String _val494;
-          for (int _i495 = 0; _i495 < _map492.size; ++_i495)
+          org.apache.thrift.protocol.TMap _map508 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.scheduler_meta = new HashMap<String,String>(2*_map508.size);
+          String _key509;
+          String _val510;
+          for (int _i511 = 0; _i511 < _map508.size; ++_i511)
           {
-            _key493 = iprot.readString();
-            _val494 = iprot.readString();
-            struct.scheduler_meta.put(_key493, _val494);
+            _key509 = iprot.readString();
+            _val510 = iprot.readString();
+            struct.scheduler_meta.put(_key509, _val510);
           }
         }
         struct.set_scheduler_meta_isSet(true);
@@ -1281,6 +1424,21 @@ public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, S
         struct.version = iprot.readString();
         struct.set_version_isSet(true);
       }
+      if (incoming.get(6)) {
+        {
+          org.apache.thrift.protocol.TMap _map512 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.resources_map = new HashMap<String,Double>(2*_map512.size);
+          String _key513;
+          double _val514;
+          for (int _i515 = 0; _i515 < _map512.size; ++_i515)
+          {
+            _key513 = iprot.readString();
+            _val514 = iprot.readDouble();
+            struct.resources_map.put(_key513, _val514);
+          }
+        }
+        struct.set_resources_map_isSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
index 9bdf9f7..ed626b5 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-20")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
 public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSummary, SupervisorSummary._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorSummary");
 
@@ -61,6 +61,7 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
   private static final org.apache.thrift.protocol.TField NUM_USED_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_used_workers", org.apache.thrift.protocol.TType.I32, (short)4);
   private static final org.apache.thrift.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_id", org.apache.thrift.protocol.TType.STRING, (short)5);
   private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField TOTAL_RESOURCES_FIELD_DESC = new org.apache.thrift.protocol.TField("total_resources", org.apache.thrift.protocol.TType.MAP, (short)7);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -74,6 +75,7 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
   private int num_used_workers; // required
   private String supervisor_id; // required
   private String version; // optional
+  private Map<String,Double> total_resources; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -82,7 +84,8 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
     NUM_WORKERS((short)3, "num_workers"),
     NUM_USED_WORKERS((short)4, "num_used_workers"),
     SUPERVISOR_ID((short)5, "supervisor_id"),
-    VERSION((short)6, "version");
+    VERSION((short)6, "version"),
+    TOTAL_RESOURCES((short)7, "total_resources");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -109,6 +112,8 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
           return SUPERVISOR_ID;
         case 6: // VERSION
           return VERSION;
+        case 7: // TOTAL_RESOURCES
+          return TOTAL_RESOURCES;
         default:
           return null;
       }
@@ -153,7 +158,7 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
   private static final int __NUM_WORKERS_ISSET_ID = 1;
   private static final int __NUM_USED_WORKERS_ISSET_ID = 2;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.VERSION};
+  private static final _Fields optionals[] = {_Fields.VERSION,_Fields.TOTAL_RESOURCES};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -169,6 +174,10 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOTAL_RESOURCES, new org.apache.thrift.meta_data.FieldMetaData("total_resources", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap);
   }
@@ -213,6 +222,10 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
     if (other.is_set_version()) {
       this.version = other.version;
     }
+    if (other.is_set_total_resources()) {
+      Map<String,Double> __this__total_resources = new HashMap<String,Double>(other.total_resources);
+      this.total_resources = __this__total_resources;
+    }
   }
 
   public SupervisorSummary deepCopy() {
@@ -231,6 +244,7 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
     this.supervisor_id = null;
     this.version = "VERSION_NOT_PROVIDED";
 
+    this.total_resources = null;
   }
 
   public String get_host() {
@@ -368,6 +382,40 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
     }
   }
 
+  public int get_total_resources_size() {
+    return (this.total_resources == null) ? 0 : this.total_resources.size();
+  }
+
+  public void put_to_total_resources(String key, double val) {
+    if (this.total_resources == null) {
+      this.total_resources = new HashMap<String,Double>();
+    }
+    this.total_resources.put(key, val);
+  }
+
+  public Map<String,Double> get_total_resources() {
+    return this.total_resources;
+  }
+
+  public void set_total_resources(Map<String,Double> total_resources) {
+    this.total_resources = total_resources;
+  }
+
+  public void unset_total_resources() {
+    this.total_resources = null;
+  }
+
+  /** Returns true if field total_resources is set (has been assigned a value) and false otherwise */
+  public boolean is_set_total_resources() {
+    return this.total_resources != null;
+  }
+
+  public void set_total_resources_isSet(boolean value) {
+    if (!value) {
+      this.total_resources = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case HOST:
@@ -418,6 +466,14 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
       }
       break;
 
+    case TOTAL_RESOURCES:
+      if (value == null) {
+        unset_total_resources();
+      } else {
+        set_total_resources((Map<String,Double>)value);
+      }
+      break;
+
     }
   }
 
@@ -441,6 +497,9 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
     case VERSION:
       return get_version();
 
+    case TOTAL_RESOURCES:
+      return get_total_resources();
+
     }
     throw new IllegalStateException();
   }
@@ -464,6 +523,8 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
       return is_set_supervisor_id();
     case VERSION:
       return is_set_version();
+    case TOTAL_RESOURCES:
+      return is_set_total_resources();
     }
     throw new IllegalStateException();
   }
@@ -535,6 +596,15 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
         return false;
     }
 
+    boolean this_present_total_resources = true && this.is_set_total_resources();
+    boolean that_present_total_resources = true && that.is_set_total_resources();
+    if (this_present_total_resources || that_present_total_resources) {
+      if (!(this_present_total_resources && that_present_total_resources))
+        return false;
+      if (!this.total_resources.equals(that.total_resources))
+        return false;
+    }
+
     return true;
   }
 
@@ -572,6 +642,11 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
     if (present_version)
       list.add(version);
 
+    boolean present_total_resources = true && (is_set_total_resources());
+    list.add(present_total_resources);
+    if (present_total_resources)
+      list.add(total_resources);
+
     return list.hashCode();
   }
 
@@ -643,6 +718,16 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_total_resources()).compareTo(other.is_set_total_resources());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_total_resources()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.total_resources, other.total_resources);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -700,6 +785,16 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
       }
       first = false;
     }
+    if (is_set_total_resources()) {
+      if (!first) sb.append(", ");
+      sb.append("total_resources:");
+      if (this.total_resources == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.total_resources);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -813,6 +908,26 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 7: // TOTAL_RESOURCES
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map74 = iprot.readMapBegin();
+                struct.total_resources = new HashMap<String,Double>(2*_map74.size);
+                String _key75;
+                double _val76;
+                for (int _i77 = 0; _i77 < _map74.size; ++_i77)
+                {
+                  _key75 = iprot.readString();
+                  _val76 = iprot.readDouble();
+                  struct.total_resources.put(_key75, _val76);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_total_resources_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -852,6 +967,21 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
           oprot.writeFieldEnd();
         }
       }
+      if (struct.total_resources != null) {
+        if (struct.is_set_total_resources()) {
+          oprot.writeFieldBegin(TOTAL_RESOURCES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.total_resources.size()));
+            for (Map.Entry<String, Double> _iter78 : struct.total_resources.entrySet())
+            {
+              oprot.writeString(_iter78.getKey());
+              oprot.writeDouble(_iter78.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -878,10 +1008,23 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
       if (struct.is_set_version()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_total_resources()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.is_set_version()) {
         oprot.writeString(struct.version);
       }
+      if (struct.is_set_total_resources()) {
+        {
+          oprot.writeI32(struct.total_resources.size());
+          for (Map.Entry<String, Double> _iter79 : struct.total_resources.entrySet())
+          {
+            oprot.writeString(_iter79.getKey());
+            oprot.writeDouble(_iter79.getValue());
+          }
+        }
+      }
     }
 
     @Override
@@ -897,11 +1040,26 @@ public class SupervisorSummary implements org.apache.thrift.TBase<SupervisorSumm
       struct.set_num_used_workers_isSet(true);
       struct.supervisor_id = iprot.readString();
       struct.set_supervisor_id_isSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.version = iprot.readString();
         struct.set_version_isSet(true);
       }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map80 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+          struct.total_resources = new HashMap<String,Double>(2*_map80.size);
+          String _key81;
+          double _val82;
+          for (int _i83 = 0; _i83 < _map80.size; ++_i83)
+          {
+            _key81 = iprot.readString();
+            _val82 = iprot.readDouble();
+            struct.total_resources.put(_key81, _val82);
+          }
+        }
+        struct.set_total_resources_isSet(true);
+      }
     }
   }
 


[3/9] storm git commit: [STORM-893] - Resource Aware Scheduler implementation. [STORM-894] - Basic Resource Aware Scheduling implementation.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
new file mode 100644
index 0000000..3c86528
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -0,0 +1,547 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.resource;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Map.Entry;
+import java.util.ArrayList;
+
+import backtype.storm.Config;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.SchedulerAssignment;
+import backtype.storm.scheduler.SupervisorDetails;
+import backtype.storm.scheduler.WorkerSlot;
+
+/**
+ * Represents a single node in the cluster.
+ */
+public class RAS_Node {
+    private static final Logger LOG = LoggerFactory.getLogger(RAS_Node.class);
+    private Map<String, Set<WorkerSlot>> _topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
+    private Set<WorkerSlot> _freeSlots = new HashSet<WorkerSlot>();
+    private final String _nodeId;
+    private String _hostname;
+    private boolean _isAlive;
+    private SupervisorDetails _sup;
+    private Double _availMemory;
+    private Double _availCPU;
+    private List<WorkerSlot> _slots;
+    private List<ExecutorDetails> _execs;
+    private Map<WorkerSlot, List<ExecutorDetails>> _slotToExecs;
+
+    public RAS_Node(String nodeId, Set<Integer> allPorts, boolean isAlive,
+                    SupervisorDetails sup) {
+        _slots = new ArrayList<WorkerSlot>();
+        _execs = new ArrayList<ExecutorDetails>();
+        _slotToExecs = new HashMap<WorkerSlot, List<ExecutorDetails>>();
+        _nodeId = nodeId;
+        _isAlive = isAlive;
+        if (_isAlive && allPorts != null) {
+            for (int port : allPorts) {
+                _freeSlots.add(new WorkerSlot(_nodeId, port));
+            }
+            _sup = sup;
+            _hostname = sup.getHost();
+            _availMemory = this.getTotalMemoryResources();
+            _availCPU = this.getTotalCpuResources();
+            _slots.addAll(_freeSlots);
+            for (WorkerSlot ws : _slots) {
+                _slotToExecs.put(ws, new ArrayList<ExecutorDetails>());
+            }
+        }
+    }
+
+    public String getId() {
+        return _nodeId;
+    }
+
+    public String getHostname() {
+        return _hostname;
+    }
+
+    public Collection<WorkerSlot> getFreeSlots() {
+        return _freeSlots;
+    }
+
+    public boolean isAlive() {
+        return _isAlive;
+    }
+
+    /**
+     * @return a collection of the topology ids currently running on this node
+     */
+    public Collection<String> getRunningTopologies() {
+        return _topIdToUsedSlots.keySet();
+    }
+
+    public boolean isTotallyFree() {
+        return _topIdToUsedSlots.isEmpty();
+    }
+
+    public int totalSlotsFree() {
+        return _freeSlots.size();
+    }
+
+    public int totalSlotsUsed() {
+        int total = 0;
+        for (Set<WorkerSlot> slots : _topIdToUsedSlots.values()) {
+            total += slots.size();
+        }
+        return total;
+    }
+
+    public int totalSlots() {
+        return totalSlotsFree() + totalSlotsUsed();
+    }
+
+    public int totalSlotsUsed(String topId) {
+        int total = 0;
+        Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
+        if (slots != null) {
+            total = slots.size();
+        }
+        return total;
+    }
+
+    private void validateSlot(WorkerSlot ws) {
+        if (!_nodeId.equals(ws.getNodeId())) {
+            throw new IllegalArgumentException(
+                    "Trying to add a slot to the wrong node " + ws +
+                            " is not a part of " + _nodeId);
+        }
+    }
+
+    private void addOrphanedSlot(WorkerSlot ws) {
+        if (_isAlive) {
+            throw new IllegalArgumentException("Orphaned Slots " +
+                    "only are allowed on dead nodes.");
+        }
+        validateSlot(ws);
+        if (_freeSlots.contains(ws)) {
+            return;
+        }
+        for (Set<WorkerSlot> used : _topIdToUsedSlots.values()) {
+            if (used.contains(ws)) {
+                return;
+            }
+        }
+        _freeSlots.add(ws);
+        _slotToExecs.put(ws, new ArrayList<ExecutorDetails>());
+    }
+
+    boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) {
+        validateSlot(ws);
+        if (!_freeSlots.remove(ws)) {
+            if (dontThrow) {
+                return true;
+            }
+            throw new IllegalStateException("Assigning a slot that was not free " + ws);
+        }
+        Set<WorkerSlot> usedSlots = _topIdToUsedSlots.get(topId);
+        if (usedSlots == null) {
+            usedSlots = new HashSet<WorkerSlot>();
+            _topIdToUsedSlots.put(topId, usedSlots);
+        }
+        usedSlots.add(ws);
+        return false;
+    }
+
+    /**
+     * Free all slots on this node.  This will update the Cluster too.
+     * @param cluster the cluster to be updated
+     */
+    public void freeAllSlots(Cluster cluster) {
+        if (!_isAlive) {
+            LOG.warn("Freeing all slots on a dead node {} ", _nodeId);
+        }
+        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+            cluster.freeSlots(entry.getValue());
+            if (_isAlive) {
+                _freeSlots.addAll(entry.getValue());
+            }
+        }
+        _topIdToUsedSlots = new HashMap<String, Set<WorkerSlot>>();
+    }
+
+    /**
+     * Frees a single slot in this node
+     * @param ws the slot to free
+     * @param cluster the cluster to update
+     */
+    public void free(WorkerSlot ws, Cluster cluster) {
+        if (_freeSlots.contains(ws)) return;
+        for (Entry<String, Set<WorkerSlot>> entry : _topIdToUsedSlots.entrySet()) {
+            Set<WorkerSlot> slots = entry.getValue();
+            if (slots.remove(ws)) {
+                cluster.freeSlot(ws);
+                if (_isAlive) {
+                    _freeSlots.add(ws);
+                }
+                return;
+            }
+        }
+        throw new IllegalArgumentException("Tried to free a slot that was not" +
+                " part of this node " + _nodeId);
+    }
+
+    /**
+     * Frees all the slots for a topology.
+     * @param topId the topology to free slots for
+     * @param cluster the cluster to update
+     */
+    public void freeTopology(String topId, Cluster cluster) {
+        Set<WorkerSlot> slots = _topIdToUsedSlots.get(topId);
+        if (slots == null || slots.isEmpty()) {
+            return;
+        }
+        for (WorkerSlot ws : slots) {
+            cluster.freeSlot(ws);
+            if (_isAlive) {
+                _freeSlots.add(ws);
+            }
+        }
+        _topIdToUsedSlots.remove(topId);
+    }
+
+    public void assign(WorkerSlot target, TopologyDetails td, Collection<ExecutorDetails> executors,
+                       Cluster cluster) {
+        if (!_isAlive) {
+            throw new IllegalStateException("Trying to adding to a dead node " + _nodeId);
+        }
+        if (_freeSlots.isEmpty()) {
+            throw new IllegalStateException("Trying to assign to a full node " + _nodeId);
+        }
+        if (executors.size() == 0) {
+            LOG.warn("Trying to assign nothing from " + td.getId() + " to " + _nodeId + " (Ignored)");
+        }
+
+        if (target == null) {
+            target = _freeSlots.iterator().next();
+        }
+        if (!_freeSlots.contains(target)) {
+            throw new IllegalStateException("Trying to assign already used slot" + target.getPort() + "on node " + _nodeId);
+        } else {
+            cluster.assign(target, td.getId(), executors);
+            assignInternal(target, td.getId(), false);
+        }
+    }
+
+    /**
+     * Assign a free slot on the node to the following topology and executors.
+     * This will update the cluster too.
+     * @param td the TopologyDetails to assign a free slot to.
+     * @param executors the executors to run in that slot.
+     * @param cluster the cluster to be updated
+     */
+    public void assign(TopologyDetails td, Collection<ExecutorDetails> executors,
+                       Cluster cluster) {
+        this.assign(null, td, executors, cluster);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (other instanceof RAS_Node) {
+            return _nodeId.equals(((RAS_Node) other)._nodeId);
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode() {
+        return _nodeId.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return "{Node: " + _sup.getHost() + ", AvailMem: " + _availMemory.toString() + ", AvailCPU: " + _availCPU.toString() + "}";
+    }
+
+    public static int countSlotsUsed(String topId, Collection<RAS_Node> nodes) {
+        int total = 0;
+        for (RAS_Node n : nodes) {
+            total += n.totalSlotsUsed(topId);
+        }
+        return total;
+    }
+
+    public static int countSlotsUsed(Collection<RAS_Node> nodes) {
+        int total = 0;
+        for (RAS_Node n : nodes) {
+            total += n.totalSlotsUsed();
+        }
+        return total;
+    }
+
+    public static int countFreeSlotsAlive(Collection<RAS_Node> nodes) {
+        int total = 0;
+        for (RAS_Node n : nodes) {
+            if (n.isAlive()) {
+                total += n.totalSlotsFree();
+            }
+        }
+        return total;
+    }
+
+    public static int countTotalSlotsAlive(Collection<RAS_Node> nodes) {
+        int total = 0;
+        for (RAS_Node n : nodes) {
+            if (n.isAlive()) {
+                total += n.totalSlots();
+            }
+        }
+        return total;
+    }
+
+    public static Map<String, RAS_Node> getAllNodesFrom(Cluster cluster, Topologies topologies) {
+        Map<String, RAS_Node> nodeIdToNode = new HashMap<String, RAS_Node>();
+        for (SupervisorDetails sup : cluster.getSupervisors().values()) {
+            //Node ID and supervisor ID are the same.
+            String id = sup.getId();
+            boolean isAlive = !cluster.isBlackListed(id);
+            LOG.debug("Found a {} Node {} {}",
+                    isAlive ? "living" : "dead", id, sup.getAllPorts());
+            LOG.debug("resources_mem: {}, resources_CPU: {}", sup.getTotalMemory(), sup.getTotalCPU());
+            nodeIdToNode.put(sup.getId(), new RAS_Node(id, sup.getAllPorts(), isAlive, sup));
+        }
+        for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments().entrySet()) {
+            String topId = entry.getValue().getTopologyId();
+            for (WorkerSlot workerSlot : entry.getValue().getSlots()) {
+                String id = workerSlot.getNodeId();
+                RAS_Node node = nodeIdToNode.get(id);
+                if (node == null) {
+                    LOG.info("Found an assigned slot on a dead supervisor {} with executors {}",
+                            workerSlot, getExecutors(workerSlot, cluster));
+                    node = new RAS_Node(id, null, false, null);
+                    nodeIdToNode.put(id, node);
+                }
+                if (!node.isAlive()) {
+                    //The supervisor on the node down so add an orphaned slot to hold the unsupervised worker
+                    node.addOrphanedSlot(workerSlot);
+                }
+                if (node.assignInternal(workerSlot, topId, true)) {
+                    LOG.warn("Bad scheduling state, " + workerSlot + " assigned multiple workers, unassigning everything...");
+                    node.free(workerSlot, cluster);
+                }
+            }
+        }
+        RAS_Node.updateAvailableResources(cluster, topologies, nodeIdToNode);
+
+        for (Map.Entry<String, SchedulerAssignment> entry : cluster
+                .getAssignments().entrySet()) {
+            for (Map.Entry<ExecutorDetails, WorkerSlot> exec : entry.getValue()
+                    .getExecutorToSlot().entrySet()) {
+                ExecutorDetails ed = exec.getKey();
+                WorkerSlot ws = exec.getValue();
+                String node_id = ws.getNodeId();
+                if (nodeIdToNode.containsKey(node_id)) {
+                    RAS_Node node = nodeIdToNode.get(node_id);
+                    if (node._slotToExecs.containsKey(ws)) {
+                        node._slotToExecs.get(ws).add(ed);
+                        node._execs.add(ed);
+                    } else {
+                        LOG.info(
+                                "ERROR: should have node {} should have worker: {}",
+                                node_id, ed);
+                        return null;
+                    }
+                } else {
+                    LOG.info("ERROR: should have node {}", node_id);
+                    return null;
+                }
+            }
+        }
+        return nodeIdToNode;
+    }
+
+    //This function is only used for logging information
+    private static Collection<ExecutorDetails> getExecutors(WorkerSlot ws,
+                                                            Cluster cluster) {
+        Collection<ExecutorDetails> retList = new ArrayList<ExecutorDetails>();
+        for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments()
+                .entrySet()) {
+            Map<ExecutorDetails, WorkerSlot> executorToSlot = entry.getValue()
+                    .getExecutorToSlot();
+            for (Map.Entry<ExecutorDetails, WorkerSlot> execToSlot : executorToSlot
+                    .entrySet()) {
+                WorkerSlot slot = execToSlot.getValue();
+                if (ws.getPort() == slot.getPort()
+                        && ws.getNodeId().equals(slot.getNodeId())) {
+                    ExecutorDetails exec = execToSlot.getKey();
+                    retList.add(exec);
+                }
+            }
+        }
+        return retList;
+    }
+
+    /**
+     * updates the available resources for every node in a cluster
+     * by recalculating memory requirements.
+     * @param cluster the cluster used in this calculation
+     * @param topologies container of all topologies
+     * @param nodeIdToNode a map between node id and node
+     */
+    private static void updateAvailableResources(Cluster cluster,
+                                                 Topologies topologies,
+                                                 Map<String, RAS_Node> nodeIdToNode) {
+        //recompute memory
+        if (cluster.getAssignments().size() > 0) {
+            for (Entry<String, SchedulerAssignment> entry : cluster.getAssignments()
+                    .entrySet()) {
+                Map<ExecutorDetails, WorkerSlot> executorToSlot = entry.getValue()
+                        .getExecutorToSlot();
+                Map<ExecutorDetails, Double> topoMemoryResourceList = topologies.getById(entry.getKey()).getTotalMemoryResourceList();
+
+                if (topoMemoryResourceList == null || topoMemoryResourceList.size() == 0) {
+                    continue;
+                }
+                for (Map.Entry<ExecutorDetails, WorkerSlot> execToSlot : executorToSlot
+                        .entrySet()) {
+                    WorkerSlot slot = execToSlot.getValue();
+                    ExecutorDetails exec = execToSlot.getKey();
+                    RAS_Node node = nodeIdToNode.get(slot.getNodeId());
+                    if (!node.isAlive()) {
+                        continue;
+                        // We do not free the assigned slots (the orphaned slots) on the inactive supervisors
+                        // The inactive node will be treated as a 0-resource node and not available for other unassigned workers
+                    }
+                    if (topoMemoryResourceList.containsKey(exec)) {
+                        node.consumeResourcesforTask(exec, topologies.getById(entry.getKey()));
+                    } else {
+                        LOG.warn("Resource Req not found...Scheduling Task{} with memory requirement as on heap - {} and off heap - {} and CPU requirement as {}",
+                                exec,
+                                Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
+                                Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT);
+                        topologies.getById(entry.getKey()).addDefaultResforExec(exec);
+                        node.consumeResourcesforTask(exec, topologies.getById(entry.getKey()));
+                    }
+                }
+            }
+        } else {
+            for (RAS_Node n : nodeIdToNode.values()) {
+                n.setAvailableMemory(n.getAvailableMemoryResources());
+            }
+        }
+    }
+
+    /**
+     * Sets the Available Memory for a node
+     * @param amount the amount to set as available memory
+     */
+    public void setAvailableMemory(Double amount) {
+        _availMemory = amount;
+    }
+
+    /**
+     * Gets the available memory resources for this node
+     * @return the available memory for this node
+     */
+    public Double getAvailableMemoryResources() {
+        if (_availMemory == null) {
+            return 0.0;
+        }
+        return _availMemory;
+    }
+
+    /**
+     * Gets the total memory resources for this node
+     * @return the total memory for this node
+     */
+    public Double getTotalMemoryResources() {
+        if (_sup != null && _sup.getTotalMemory() != null) {
+            return _sup.getTotalMemory();
+        } else {
+            return 0.0;
+        }
+    }
+
+    /**
+     * Consumes a certain amount of memory for this node
+     * @param amount is the amount memory to consume from this node
+     * @return the current available memory for this node after consumption
+     */
+    public Double consumeMemory(Double amount) {
+        if (amount > _availMemory) {
+            LOG.error("Attempting to consume more memory than available! Needed: {}, we only have: {}", amount, _availMemory);
+            return null;
+        }
+        _availMemory = _availMemory - amount;
+        return _availMemory;
+    }
+
+    /**
+     * Gets the available cpu resources for this node
+     * @return the available cpu for this node
+     */
+    public Double getAvailableCpuResources() {
+        if (_availCPU == null) {
+            return 0.0;
+        }
+        return _availCPU;
+    }
+
+    /**
+     * Gets the total cpu resources for this node
+     * @return the total cpu for this node
+     */
+    public Double getTotalCpuResources() {
+        if (_sup != null && _sup.getTotalCPU() != null) {
+            return _sup.getTotalCPU();
+        } else {
+            return 0.0;
+        }
+    }
+
+    /**
+     * Consumes a certain amount of cpu for this node
+     * @param amount is the amount cpu to consume from this node
+     * @return the current available cpu for this node after consumption
+     */
+    public Double consumeCPU(Double amount) {
+        if (amount > _availCPU) {
+            LOG.error("Attempting to consume more CPU than available! Needed: {}, we only have: {}", amount, _availCPU);
+            return null;
+        }
+        _availCPU = _availCPU - amount;
+        return _availCPU;
+    }
+
+    /**
+     * Consumes a certain amount of resources for a executor in a topology.
+     * @param exec is the executor that is consuming resources on this node
+     * @param topo the topology the executor is a part
+     */
+    public void consumeResourcesforTask(ExecutorDetails exec, TopologyDetails topo) {
+        Double taskMemReq = topo.getTotalMemReqTask(exec);
+        Double taskCpuReq = topo.getTotalCpuReqTask(exec);
+        this.consumeCPU(taskCpuReq);
+        this.consumeMemory(taskMemReq);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
new file mode 100644
index 0000000..b3ff97b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -0,0 +1,149 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.resource;
+
+import backtype.storm.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.IScheduler;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.scheduler.resource.strategies.ResourceAwareStrategy;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+
+public class ResourceAwareScheduler implements IScheduler {
+    private static final Logger LOG = LoggerFactory
+            .getLogger(ResourceAwareScheduler.class);
+    @SuppressWarnings("rawtypes")
+    private Map _conf;
+
+    @Override
+    public void prepare(Map conf) {
+        _conf = conf;
+    }
+
+    @Override
+    public void schedule(Topologies topologies, Cluster cluster) {
+        LOG.debug("\n\n\nRerunning ResourceAwareScheduler...");
+
+        ResourceAwareStrategy RAStrategy = new ResourceAwareStrategy(cluster, topologies);
+        LOG.debug(printScheduling(cluster, topologies));
+
+        for (TopologyDetails td : topologies.getTopologies()) {
+            String topId = td.getId();
+            Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap;
+            if (cluster.needsScheduling(td) && cluster.getUnassignedExecutors(td).size() > 0) {
+                LOG.debug("/********Scheduling topology {} ************/", topId);
+
+                schedulerAssignmentMap = RAStrategy.schedule(td);
+
+                if (schedulerAssignmentMap != null) {
+                    try {
+                        Set<String> nodesUsed = new HashSet<String>();
+                        for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> taskToWorkerEntry : schedulerAssignmentMap.entrySet()) {
+                            WorkerSlot targetSlot = taskToWorkerEntry.getKey();
+                            Collection<ExecutorDetails> execsNeedScheduling = taskToWorkerEntry.getValue();
+                            RAS_Node targetNode = RAStrategy.idToNode(targetSlot.getNodeId());
+                            targetNode.assign(targetSlot, td, execsNeedScheduling, cluster);
+                            LOG.debug("ASSIGNMENT    TOPOLOGY: {}  TASKS: {} To Node: {} on Slot: {}",
+                                    td.getName(), execsNeedScheduling, targetNode.getHostname(), targetSlot.getPort());
+                            if (!nodesUsed.contains(targetNode.getId())) {
+                                nodesUsed.add(targetNode.getId());
+                            }
+                        }
+                        LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), schedulerAssignmentMap.keySet().size());
+                        cluster.setStatus(td.getId(), td.getId() + " Fully Scheduled");
+                    } catch (IllegalStateException ex) {
+                        LOG.error(ex.toString());
+                        LOG.error("Unsuccessfull in scheduling {}", td.getId());
+                        cluster.setStatus(td.getId(), "Unsuccessfull in scheduling " + td.getId());
+                    }
+                } else {
+                    LOG.error("Unsuccessfull in scheduling topology {}", td.getId());
+                    cluster.setStatus(td.getId(), "Unsuccessfull in scheduling " + td.getId());
+                }
+            } else {
+                cluster.setStatus(td.getId(), td.getId() + " Fully Scheduled");
+            }
+        }
+    }
+
+    private Map<String, Double> getUserConf() {
+        Map<String, Double> ret = new HashMap<String, Double>();
+        ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
+                (Double) _conf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB));
+        ret.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB,
+                (Double) _conf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB));
+        return ret;
+    }
+
+    /**
+     * print scheduling for debug purposes
+     * @param cluster
+     * @param topologies
+     */
+    public String printScheduling(Cluster cluster, Topologies topologies) {
+        StringBuilder str = new StringBuilder();
+        Map<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>> schedulingMap = new HashMap<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>>();
+        for (TopologyDetails topo : topologies.getTopologies()) {
+            if (cluster.getAssignmentById(topo.getId()) != null) {
+                for (Map.Entry<ExecutorDetails, WorkerSlot> entry : cluster.getAssignmentById(topo.getId()).getExecutorToSlot().entrySet()) {
+                    WorkerSlot slot = entry.getValue();
+                    String nodeId = slot.getNodeId();
+                    ExecutorDetails exec = entry.getKey();
+                    if (schedulingMap.containsKey(nodeId) == false) {
+                        schedulingMap.put(nodeId, new HashMap<String, Map<WorkerSlot, Collection<ExecutorDetails>>>());
+                    }
+                    if (schedulingMap.get(nodeId).containsKey(topo.getId()) == false) {
+                        schedulingMap.get(nodeId).put(topo.getId(), new HashMap<WorkerSlot, Collection<ExecutorDetails>>());
+                    }
+                    if (schedulingMap.get(nodeId).get(topo.getId()).containsKey(slot) == false) {
+                        schedulingMap.get(nodeId).get(topo.getId()).put(slot, new LinkedList<ExecutorDetails>());
+                    }
+                    schedulingMap.get(nodeId).get(topo.getId()).get(slot).add(exec);
+                }
+            }
+        }
+
+        for (Map.Entry<String, Map<String, Map<WorkerSlot, Collection<ExecutorDetails>>>> entry : schedulingMap.entrySet()) {
+            if (cluster.getSupervisorById(entry.getKey()) != null) {
+                str.append("/** Node: " + cluster.getSupervisorById(entry.getKey()).getHost() + "-" + entry.getKey() + " **/\n");
+            } else {
+                str.append("/** Node: Unknown may be dead -" + entry.getKey() + " **/\n");
+            }
+            for (Map.Entry<String, Map<WorkerSlot, Collection<ExecutorDetails>>> topo_sched : schedulingMap.get(entry.getKey()).entrySet()) {
+                str.append("\t-->Topology: " + topo_sched.getKey() + "\n");
+                for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> ws : topo_sched.getValue().entrySet()) {
+                    str.append("\t\t->Slot [" + ws.getKey().getPort() + "] -> " + ws.getValue() + "\n");
+                }
+            }
+        }
+        return str.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
new file mode 100644
index 0000000..c22d5bc
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceUtils.java
@@ -0,0 +1,133 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.resource;
+
+import backtype.storm.Config;
+import backtype.storm.generated.Bolt;
+import backtype.storm.generated.SpoutSpec;
+import backtype.storm.generated.StormTopology;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class ResourceUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(ResourceUtils.class);
+
+    public static Map<String, Map<String, Double>> getBoltsResources(StormTopology topology, Map topologyConf) {
+        Map<String, Map<String, Double>> boltResources = new HashMap<String, Map<String, Double>>();
+        if (topology.get_bolts() != null) {
+            for (Map.Entry<String, Bolt> bolt : topology.get_bolts().entrySet()) {
+                Map<String, Double> topology_resources = parseResources(bolt.getValue().get_common().get_json_conf());
+                checkIntialization(topology_resources, bolt.getValue().toString(), topologyConf);
+                boltResources.put(bolt.getKey(), topology_resources);
+            }
+        }
+        return boltResources;
+    }
+
+    public static Map<String, Map<String, Double>> getSpoutsResources(StormTopology topology, Map topologyConf) {
+        Map<String, Map<String, Double>> spoutResources = new HashMap<String, Map<String, Double>>();
+        if (topology.get_spouts() != null) {
+            for (Map.Entry<String, SpoutSpec> spout : topology.get_spouts().entrySet()) {
+                Map<String, Double> topology_resources = parseResources(spout.getValue().get_common().get_json_conf());
+                checkIntialization(topology_resources, spout.getValue().toString(), topologyConf);
+                spoutResources.put(spout.getKey(), topology_resources);
+            }
+        }
+        return spoutResources;
+    }
+
+    public static void checkIntialization(Map<String, Double> topology_resources, String Com, Map topologyConf) {
+        checkInitMem(topology_resources, Com, topologyConf);
+        checkInitCPU(topology_resources, Com, topologyConf);
+    }
+
+    public static void checkInitMem(Map<String, Double> topology_resources, String Com, Map topologyConf) {
+        if (!topology_resources.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB)) {
+            topology_resources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB,
+                    backtype.storm.utils.Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null));
+            debugMessage("ONHEAP", Com, topologyConf);
+        }
+        if (!topology_resources.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)) {
+            topology_resources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB,
+                    backtype.storm.utils.Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null));
+            debugMessage("OFFHEAP", Com, topologyConf);
+        }
+    }
+
+    public static void checkInitCPU(Map<String, Double> topology_resources, String Com, Map topologyConf) {
+        if (!topology_resources.containsKey(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT)) {
+            topology_resources.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT,
+                    backtype.storm.utils.Utils.getDouble(topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null));
+            debugMessage("CPU", Com, topologyConf);
+        }
+    }
+
+    public static Map<String, Double> parseResources(String input) {
+        Map<String, Double> topology_resources = new HashMap<String, Double>();
+        JSONParser parser = new JSONParser();
+        LOG.debug("Input to parseResources {}", input);
+        try {
+            if (input != null) {
+                Object obj = parser.parse(input);
+                JSONObject jsonObject = (JSONObject) obj;
+                if (jsonObject.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB)) {
+                    Double topoMemOnHeap = backtype.storm.utils.Utils
+                            .getDouble(jsonObject.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB), null);
+                    topology_resources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, topoMemOnHeap);
+                }
+                if (jsonObject.containsKey(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)) {
+                    Double topoMemOffHeap = backtype.storm.utils.Utils
+                            .getDouble(jsonObject.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB), null);
+                    topology_resources.put(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, topoMemOffHeap);
+                }
+                if (jsonObject.containsKey(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT)) {
+                    Double topoCPU = backtype.storm.utils.Utils.getDouble(jsonObject.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT), null);
+                    topology_resources.put(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, topoCPU);
+                }
+                LOG.debug("Topology Resources {}", topology_resources);
+            }
+        } catch (ParseException e) {
+            LOG.error("Failed to parse component resources is:" + e.toString(), e);
+            return null;
+        }
+        return topology_resources;
+    }
+
+    private static void debugMessage(String memoryType, String Com, Map topologyConf) {
+        if (memoryType.equals("ONHEAP")) {
+            LOG.debug(
+                    "Unable to extract resource requirement for Component {} \n Resource : Memory Type : On Heap set to default {}",
+                    Com, topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB));
+        } else if (memoryType.equals("OFFHEAP")) {
+            LOG.debug(
+                    "Unable to extract resource requirement for Component {} \n Resource : Memory Type : Off Heap set to default {}",
+                    Com, topologyConf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB));
+        } else {
+            LOG.debug(
+                    "Unable to extract resource requirement for Component {} \n Resource : CPU Pcore Percent set to default {}",
+                    Com, topologyConf.get(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
new file mode 100644
index 0000000..01f3223
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/IStrategy.java
@@ -0,0 +1,37 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scheduler.resource.strategies;
+
+import java.util.Collection;
+import java.util.Map;
+
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.scheduler.resource.RAS_Node;
+
+/**
+ * An interface to for implementing different scheduling strategies for the resource aware scheduling
+ * In the future stategies will be pluggable
+ */
+public interface IStrategy {
+
+    public Map<WorkerSlot, Collection<ExecutorDetails>> schedule(TopologyDetails td);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
new file mode 100644
index 0000000..22bac2d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
@@ -0,0 +1,480 @@
+/**
+ * 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.scheduler.resource.strategies;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.TreeMap;
+import java.util.HashSet;
+import java.util.Iterator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import backtype.storm.scheduler.Cluster;
+import backtype.storm.scheduler.ExecutorDetails;
+import backtype.storm.scheduler.Topologies;
+import backtype.storm.scheduler.TopologyDetails;
+import backtype.storm.scheduler.WorkerSlot;
+import backtype.storm.scheduler.resource.Component;
+import backtype.storm.scheduler.resource.RAS_Node;
+
+public class ResourceAwareStrategy implements IStrategy {
+    private Logger LOG = null;
+    private Topologies _topologies;
+    private Cluster _cluster;
+    //Map key is the supervisor id and the value is the corresponding RAS_Node Object 
+    private Map<String, RAS_Node> _availNodes;
+    private RAS_Node refNode = null;
+    /**
+     * supervisor id -> Node
+     */
+    private Map<String, RAS_Node> _nodes;
+    private Map<String, List<String>> _clusterInfo;
+
+    private final double CPU_WEIGHT = 1.0;
+    private final double MEM_WEIGHT = 1.0;
+    private final double NETWORK_WEIGHT = 1.0;
+
+    public ResourceAwareStrategy(Cluster cluster, Topologies topologies) {
+        _topologies = topologies;
+        _cluster = cluster;
+        _nodes = RAS_Node.getAllNodesFrom(cluster, _topologies);
+        _availNodes = this.getAvailNodes();
+        this.LOG = LoggerFactory.getLogger(this.getClass());
+        _clusterInfo = cluster.getNetworkTopography();
+        LOG.debug(this.getClusterInfo());
+    }
+
+    //the returned TreeMap keeps the Components sorted
+    private TreeMap<Integer, List<ExecutorDetails>> getPriorityToExecutorDetailsListMap(
+            Queue<Component> ordered__Component_list, Collection<ExecutorDetails> unassignedExecutors) {
+        TreeMap<Integer, List<ExecutorDetails>> retMap = new TreeMap<Integer, List<ExecutorDetails>>();
+        Integer rank = 0;
+        for (Component ras_comp : ordered__Component_list) {
+            retMap.put(rank, new ArrayList<ExecutorDetails>());
+            for(ExecutorDetails exec : ras_comp.execs) {
+                if(unassignedExecutors.contains(exec)) {
+                    retMap.get(rank).add(exec);
+                }
+            }
+            rank++;
+        }
+        return retMap;
+    }
+
+    public Map<WorkerSlot, Collection<ExecutorDetails>> schedule(TopologyDetails td) {
+        if (_availNodes.size() <= 0) {
+            LOG.warn("No available nodes to schedule tasks on!");
+            return null;
+        }
+        Collection<ExecutorDetails> unassignedExecutors = _cluster.getUnassignedExecutors(td);
+        Map<WorkerSlot, Collection<ExecutorDetails>> schedulerAssignmentMap = new HashMap<WorkerSlot, Collection<ExecutorDetails>>();
+        LOG.debug("ExecutorsNeedScheduling: {}", unassignedExecutors);
+        Collection<ExecutorDetails> scheduledTasks = new ArrayList<ExecutorDetails>();
+        List<Component> spouts = this.getSpouts(_topologies, td);
+
+        if (spouts.size() == 0) {
+            LOG.error("Cannot find a Spout!");
+            return null;
+        }
+
+        Queue<Component> ordered__Component_list = bfs(_topologies, td, spouts);
+
+        Map<Integer, List<ExecutorDetails>> priorityToExecutorMap = getPriorityToExecutorDetailsListMap(ordered__Component_list, unassignedExecutors);
+        Collection<ExecutorDetails> executorsNotScheduled = new HashSet<ExecutorDetails>(unassignedExecutors);
+        Integer longestPriorityListSize = this.getLongestPriorityListSize(priorityToExecutorMap);
+        //Pick the first executor with priority one, then the 1st exec with priority 2, so on an so forth. 
+        //Once we reach the last priority, we go back to priority 1 and schedule the second task with priority 1.
+        for (int i = 0; i < longestPriorityListSize; i++) {
+            for (Entry<Integer, List<ExecutorDetails>> entry : priorityToExecutorMap.entrySet()) {
+                Iterator<ExecutorDetails> it = entry.getValue().iterator();
+                if (it.hasNext()) {
+                    ExecutorDetails exec = it.next();
+                    LOG.debug("\n\nAttempting to schedule: {} of component {}[avail {}] with rank {}",
+                            new Object[] { exec, td.getExecutorToComponent().get(exec),
+                    td.getTaskResourceReqList(exec), entry.getKey() });
+                    WorkerSlot targetSlot = this.findWorkerForExec(exec, td, schedulerAssignmentMap);
+                    if (targetSlot != null) {
+                        RAS_Node targetNode = this.idToNode(targetSlot.getNodeId());
+                        if(!schedulerAssignmentMap.containsKey(targetSlot)) {
+                            schedulerAssignmentMap.put(targetSlot, new LinkedList<ExecutorDetails>());
+                        }
+                       
+                        schedulerAssignmentMap.get(targetSlot).add(exec);
+                        targetNode.consumeResourcesforTask(exec, td);
+                        scheduledTasks.add(exec);
+                        LOG.debug("TASK {} assigned to Node: {} avail [mem: {} cpu: {}] total [mem: {} cpu: {}] on slot: {}", exec,
+                                targetNode, targetNode.getAvailableMemoryResources(),
+                                targetNode.getAvailableCpuResources(), targetNode.getTotalMemoryResources(),
+                                targetNode.getTotalCpuResources(), targetSlot);
+                    } else {
+                        LOG.error("Not Enough Resources to schedule Task {}", exec);
+                    }
+                    it.remove();
+                }
+            }
+        }
+
+        executorsNotScheduled.removeAll(scheduledTasks);
+        LOG.debug("/* Scheduling left over task (most likely sys tasks) */");
+        // schedule left over system tasks
+        for (ExecutorDetails exec : executorsNotScheduled) {
+            WorkerSlot targetSlot = this.findWorkerForExec(exec, td, schedulerAssignmentMap);
+            if (targetSlot != null) {
+                RAS_Node targetNode = this.idToNode(targetSlot.getNodeId());
+                if(schedulerAssignmentMap.containsKey(targetSlot) == false) {
+                    schedulerAssignmentMap.put(targetSlot, new LinkedList<ExecutorDetails>());
+                }
+               
+                schedulerAssignmentMap.get(targetSlot).add(exec);
+                targetNode.consumeResourcesforTask(exec, td);
+                scheduledTasks.add(exec);
+                LOG.debug("TASK {} assigned to Node: {} avail [mem: {} cpu: {}] total [mem: {} cpu: {}] on slot: {}", exec,
+                        targetNode, targetNode.getAvailableMemoryResources(),
+                        targetNode.getAvailableCpuResources(), targetNode.getTotalMemoryResources(),
+                        targetNode.getTotalCpuResources(), targetSlot);
+            } else {
+                LOG.error("Not Enough Resources to schedule Task {}", exec);
+            }
+        }
+        executorsNotScheduled.removeAll(scheduledTasks);
+        if (executorsNotScheduled.size() > 0) {
+            LOG.error("Not all executors successfully scheduled: {}",
+                    executorsNotScheduled);
+            schedulerAssignmentMap = null;
+        } else {
+            LOG.debug("All resources successfully scheduled!");
+        }
+        if (schedulerAssignmentMap == null) {
+            LOG.error("Topology {} not successfully scheduled!", td.getId());
+        }
+        return schedulerAssignmentMap;
+    }
+
+    private WorkerSlot findWorkerForExec(ExecutorDetails exec, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+      WorkerSlot ws = null;
+      // first scheduling
+      if (this.refNode == null) {
+          String clus = this.getBestClustering();
+          ws = this.getBestWorker(exec, td, clus, scheduleAssignmentMap);
+      } else {
+          ws = this.getBestWorker(exec, td, scheduleAssignmentMap);
+      }
+      if(ws != null) {
+          this.refNode = this.idToNode(ws.getNodeId());
+      }
+      LOG.debug("reference node for the resource aware scheduler is: {}", this.refNode);
+      return ws;
+    }
+
+    private WorkerSlot getBestWorker(ExecutorDetails exec, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        return this.getBestWorker(exec, td, null, scheduleAssignmentMap);
+    }
+
+    private WorkerSlot getBestWorker(ExecutorDetails exec, TopologyDetails td, String clusterId, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        double taskMem = td.getTotalMemReqTask(exec);
+        double taskCPU = td.getTotalCpuReqTask(exec);
+        List<RAS_Node> nodes;
+        if(clusterId != null) {
+            nodes = this.getAvailableNodesFromCluster(clusterId);
+            
+        } else {
+            nodes = this.getAvailableNodes();
+        }
+        //First sort nodes by distance
+        TreeMap<Double, RAS_Node> nodeRankMap = new TreeMap<Double, RAS_Node>();
+        for (RAS_Node n : nodes) {
+            if(n.getFreeSlots().size()>0) {
+                if (n.getAvailableMemoryResources() >= taskMem
+                      && n.getAvailableCpuResources() >= taskCPU) {
+                  double a = Math.pow((taskCPU - n.getAvailableCpuResources())
+                          * this.CPU_WEIGHT, 2);
+                  double b = Math.pow((taskMem - n.getAvailableMemoryResources())
+                          * this.MEM_WEIGHT, 2);
+                  double c = 0.0;
+                  if(this.refNode != null) {
+                      c = Math.pow(this.distToNode(this.refNode, n)
+                              * this.NETWORK_WEIGHT, 2);
+                  }
+                  double distance = Math.sqrt(a + b + c);
+                  nodeRankMap.put(distance, n);
+                }
+            }
+        }
+        //Then, pick worker from closest node that satisfy constraints
+        for(Map.Entry<Double, RAS_Node> entry : nodeRankMap.entrySet()) {
+            RAS_Node n = entry.getValue();
+            for(WorkerSlot ws : n.getFreeSlots()) {
+                if(checkWorkerConstraints(exec, ws, td, scheduleAssignmentMap)) {
+                    return ws;
+                }
+            }
+        }
+        return null;
+    }
+
+    private String getBestClustering() {
+        String bestCluster = null;
+        Double mostRes = 0.0;
+        for (Entry<String, List<String>> cluster : _clusterInfo
+                .entrySet()) {
+            Double clusterTotalRes = this.getTotalClusterRes(cluster.getValue());
+            if (clusterTotalRes > mostRes) {
+                mostRes = clusterTotalRes;
+                bestCluster = cluster.getKey();
+            }
+        }
+        return bestCluster;
+    }
+
+    private Double getTotalClusterRes(List<String> cluster) {
+        Double res = 0.0;
+        for (String node : cluster) {
+            res += _availNodes.get(this.NodeHostnameToId(node))
+                    .getAvailableMemoryResources()
+                    + _availNodes.get(this.NodeHostnameToId(node))
+                    .getAvailableCpuResources();
+        }
+        return res;
+    }
+
+    private Double distToNode(RAS_Node src, RAS_Node dest) {
+        if (src.getId().equals(dest.getId())==true) {
+            return 1.0;
+        }else if (this.NodeToCluster(src) == this.NodeToCluster(dest)) {
+            return 2.0;
+        } else {
+            return 3.0;
+        }
+    }
+
+    private String NodeToCluster(RAS_Node node) {
+        for (Entry<String, List<String>> entry : _clusterInfo
+                .entrySet()) {
+            if (entry.getValue().contains(node.getHostname())) {
+                return entry.getKey();
+            }
+        }
+        LOG.error("Node: {} not found in any clusters", node.getHostname());
+        return null;
+    }
+    
+    private List<RAS_Node> getAvailableNodes() {
+        LinkedList<RAS_Node> nodes = new LinkedList<RAS_Node>();
+        for (String clusterId : _clusterInfo.keySet()) {
+            nodes.addAll(this.getAvailableNodesFromCluster(clusterId));
+        }
+        return nodes;
+    }
+
+    private List<RAS_Node> getAvailableNodesFromCluster(String clus) {
+        List<RAS_Node> retList = new ArrayList<RAS_Node>();
+        for (String node_id : _clusterInfo.get(clus)) {
+            retList.add(_availNodes.get(this
+                    .NodeHostnameToId(node_id)));
+        }
+        return retList;
+    }
+
+    private List<WorkerSlot> getAvailableWorkersFromCluster(String clusterId) {
+        List<RAS_Node> nodes = this.getAvailableNodesFromCluster(clusterId);
+        List<WorkerSlot> workers = new LinkedList<WorkerSlot>();
+        for(RAS_Node node : nodes) {
+            workers.addAll(node.getFreeSlots());
+        }
+        return workers;
+    }
+
+    private List<WorkerSlot> getAvailableWorker() {
+        List<WorkerSlot> workers = new LinkedList<WorkerSlot>();
+        for (String clusterId : _clusterInfo.keySet()) {
+            workers.addAll(this.getAvailableWorkersFromCluster(clusterId));
+        }
+        return workers;
+    }
+
+    /**
+     * In case in the future RAS can only use a subset of nodes
+     */
+    private Map<String, RAS_Node> getAvailNodes() {
+        return _nodes;
+    }
+
+    /**
+     * Breadth first traversal of the topology DAG
+     * @param topologies
+     * @param td
+     * @param spouts
+     * @return A partial ordering of components
+     */
+    private Queue<Component> bfs(Topologies topologies, TopologyDetails td, List<Component> spouts) {
+        // Since queue is a interface
+        Queue<Component> ordered__Component_list = new LinkedList<Component>();
+        HashMap<String, Component> visited = new HashMap<String, Component>();
+
+        /* start from each spout that is not visited, each does a breadth-first traverse */
+        for (Component spout : spouts) {
+            if (!visited.containsKey(spout.id)) {
+                Queue<Component> queue = new LinkedList<Component>();
+                queue.offer(spout);
+                while (!queue.isEmpty()) {
+                    Component comp = queue.poll();
+                    visited.put(comp.id, comp);
+                    ordered__Component_list.add(comp);
+                    List<String> neighbors = new ArrayList<String>();
+                    neighbors.addAll(comp.children);
+                    neighbors.addAll(comp.parents);
+                    for (String nbID : neighbors) {
+                        if (!visited.containsKey(nbID)) {
+                            Component child = topologies.getAllComponents().get(td.getId()).get(nbID);
+                            queue.offer(child);
+                        }
+                    }
+                }
+            }
+        }
+        return ordered__Component_list;
+    }
+
+    private List<Component> getSpouts(Topologies topologies, TopologyDetails td) {
+        List<Component> spouts = new ArrayList<Component>();
+        for (Component c : topologies.getAllComponents().get(td.getId())
+                .values()) {
+            if (c.type == Component.ComponentType.SPOUT) {
+                spouts.add(c);
+            }
+        }
+        return spouts;
+    }
+
+    private Integer getLongestPriorityListSize(Map<Integer, List<ExecutorDetails>> priorityToExecutorMap) {
+        Integer mostNum = 0;
+        for (List<ExecutorDetails> execs : priorityToExecutorMap.values()) {
+            Integer numExecs = execs.size();
+            if (mostNum < numExecs) {
+                mostNum = numExecs;
+            }
+        }
+        return mostNum;
+    }
+
+    /**
+     * Get the remaining amount memory that can be assigned to a worker given the set worker max heap size
+     * @param ws
+     * @param td
+     * @param scheduleAssignmentMap
+     * @return The remaining amount of memory
+     */
+    private Double getWorkerScheduledMemoryAvailable(WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        Double memScheduleUsed = this.getWorkerScheduledMemoryUse(ws, td, scheduleAssignmentMap);
+        return td.getTopologyWorkerMaxHeapSize() - memScheduleUsed;
+    }
+
+    /**
+     * Get the amount of memory already assigned to a worker
+     * @param ws
+     * @param td
+     * @param scheduleAssignmentMap
+     * @return the amount of memory
+     */
+    private Double getWorkerScheduledMemoryUse(WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        Double totalMem = 0.0;
+        Collection<ExecutorDetails> execs = scheduleAssignmentMap.get(ws);
+        if(execs != null) {
+            for(ExecutorDetails exec : execs) {
+                totalMem += td.getTotalMemReqTask(exec);
+            }
+        } 
+        return totalMem;
+    }
+
+    /**
+     * Checks whether we can schedule an Executor exec on the worker slot ws
+     * Only considers memory currenlty.  May include CPU in the future
+     * @param exec
+     * @param ws
+     * @param td
+     * @param scheduleAssignmentMap
+     * @return a boolean: True denoting the exec can be scheduled on ws and false if it cannot
+     */
+    private boolean checkWorkerConstraints(ExecutorDetails exec, WorkerSlot ws, TopologyDetails td, Map<WorkerSlot, Collection<ExecutorDetails>> scheduleAssignmentMap) {
+        boolean retVal = false;
+        if(this.getWorkerScheduledMemoryAvailable(ws, td, scheduleAssignmentMap) >= td.getTotalMemReqTask(exec)) {
+            retVal = true;
+        }
+        return retVal;
+    }
+
+    /**
+     * Get the amount of resources available and total for each node
+     * @return a String with cluster resource info for debug
+     */
+    private String getClusterInfo() {
+        String retVal = "Cluster info:\n";
+        for(Entry<String, List<String>> clusterEntry : _clusterInfo.entrySet()) {
+            String clusterId = clusterEntry.getKey();
+            retVal += "Rack: " + clusterId + "\n";
+            for(String nodeHostname : clusterEntry.getValue()) {
+                RAS_Node node = this.idToNode(this.NodeHostnameToId(nodeHostname));
+                retVal += "-> Node: " + node.getHostname() + " " + node.getId() + "\n";
+                retVal += "--> Avail Resources: {Mem " + node.getAvailableMemoryResources() + ", CPU " + node.getAvailableCpuResources() + "}\n";
+                retVal += "--> Total Resources: {Mem " + node.getTotalMemoryResources() + ", CPU " + node.getTotalCpuResources() + "}\n";
+            }
+        }
+        return retVal;
+    }
+
+    /**
+     * hostname to Id
+     * @param hostname
+     * @return the id of a node
+     */
+    public String NodeHostnameToId(String hostname) {
+        for (RAS_Node n : _nodes.values()) {
+            if (n.getHostname() == null) {
+                continue;
+            }
+            if (n.getHostname().equals(hostname)) {
+                return n.getId();
+            }
+        }
+        LOG.error("Cannot find Node with hostname {}", hostname);
+        return null;
+    }
+
+    /**
+     * Find RAS_Node for specified node id
+     * @param id
+     * @return a RAS_Node object
+     */
+    public RAS_Node idToNode(String id) {
+        if(_nodes.containsKey(id) == false) {
+            LOG.error("Cannot find Node with Id: {}", id);
+            return null;
+        }
+        return _nodes.get(id);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/testing/AlternateRackDNSToSwitchMapping.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/testing/AlternateRackDNSToSwitchMapping.java b/storm-core/src/jvm/backtype/storm/testing/AlternateRackDNSToSwitchMapping.java
new file mode 100644
index 0000000..8ed5f7b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/testing/AlternateRackDNSToSwitchMapping.java
@@ -0,0 +1,48 @@
+package backtype.storm.testing;
+
+import backtype.storm.networktopography.AbstractDNSToSwitchMapping;
+import backtype.storm.networktopography.DNSToSwitchMapping;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * This class implements the {@link DNSToSwitchMapping} interface
+ *    It alternates bewteen RACK1 and RACK2 for the hosts.
+ */
+public final class AlternateRackDNSToSwitchMapping extends AbstractDNSToSwitchMapping {
+
+  private Map<String, String> mappingCache = new ConcurrentHashMap<String, String>();
+
+  @Override
+  public Map<String, String> resolve(List<String> names) {
+    TreeSet<String> sortedNames = new TreeSet<String>(names);
+    Map <String, String> m = new HashMap<String, String>();
+    if (names.isEmpty()) {
+      //name list is empty, return an empty map
+      return m;
+    }
+
+    Boolean odd = true;
+    for (String name : sortedNames) {
+      if (odd) {
+        m.put(name, "RACK1");
+        mappingCache.put(name, "RACK1");
+        odd = false;
+      } else {
+        m.put(name, "RACK2");
+        mappingCache.put(name, "RACK2");
+        odd = true;
+      }
+    }
+    return m;
+  }
+
+  @Override
+  public String toString() {
+    return "defaultRackDNSToSwitchMapping (" + mappingCache.size() + " mappings cached)";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/topology/BaseConfigurationDeclarer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/BaseConfigurationDeclarer.java b/storm-core/src/jvm/backtype/storm/topology/BaseConfigurationDeclarer.java
index 465f04d..b6f3b53 100644
--- a/storm-core/src/jvm/backtype/storm/topology/BaseConfigurationDeclarer.java
+++ b/storm-core/src/jvm/backtype/storm/topology/BaseConfigurationDeclarer.java
@@ -20,8 +20,10 @@ package backtype.storm.topology;
 import backtype.storm.Config;
 import java.util.HashMap;
 import java.util.Map;
+import backtype.storm.utils.Utils;
 
 public abstract class BaseConfigurationDeclarer<T extends ComponentConfigurationDeclarer> implements ComponentConfigurationDeclarer<T> {
+    private Map conf = Utils.readStormConfig();
     @Override
     public T addConfiguration(String config, Object value) {
         Map<String, Object> configMap = new HashMap<>();
@@ -48,7 +50,34 @@ public abstract class BaseConfigurationDeclarer<T extends ComponentConfiguration
     
     @Override
     public T setNumTasks(Number val) {
-        if(val!=null) val = val.intValue();
+        if (val != null) val = val.intValue();
         return addConfiguration(Config.TOPOLOGY_TASKS, val);
     }
+
+    @Override
+    public T setMemoryLoad(Number onHeap) {
+        return setMemoryLoad(onHeap, Utils.getDouble(conf.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)));
+    }
+
+    @Override
+    public T setMemoryLoad(Number onHeap, Number offHeap) {
+        T ret = null;
+        if (onHeap != null) {
+            onHeap = onHeap.doubleValue();
+            ret = addConfiguration(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB, onHeap);
+        }
+        if (offHeap!=null) {
+            offHeap = offHeap.doubleValue();
+            ret = addConfiguration(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB, offHeap);
+        }
+        return ret;
+    }
+
+    @Override
+    public T setCPULoad(Number amount) {
+        if(amount != null) {
+            return addConfiguration(Config.TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT, amount);
+        }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/topology/ComponentConfigurationDeclarer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/topology/ComponentConfigurationDeclarer.java b/storm-core/src/jvm/backtype/storm/topology/ComponentConfigurationDeclarer.java
index 3a433b9..2003c4e 100644
--- a/storm-core/src/jvm/backtype/storm/topology/ComponentConfigurationDeclarer.java
+++ b/storm-core/src/jvm/backtype/storm/topology/ComponentConfigurationDeclarer.java
@@ -26,4 +26,7 @@ public interface ComponentConfigurationDeclarer<T extends ComponentConfiguration
     T setMaxTaskParallelism(Number val);
     T setMaxSpoutPending(Number val);
     T setNumTasks(Number val);
+    T setMemoryLoad(Number onHeap);
+    T setMemoryLoad(Number onHeap, Number offHeap);
+    T setCPULoad(Number amount);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/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 4cc6d11..6cb5a79 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -452,6 +452,25 @@ public class Utils {
       throw new IllegalArgumentException("Don't know how to convert " + o + " to int");
     }
 
+    public static Double getDouble(Object o) {
+        Double result = getDouble(o, null);
+        if (null == result) {
+            throw new IllegalArgumentException("Don't know how to convert null to double");
+        }
+        return result;
+    }
+
+    public static Double getDouble(Object o, Double defaultValue) {
+        if (null == o) {
+            return defaultValue;
+        }
+        if (o instanceof Number) {
+            return ((Number) o).doubleValue();
+        } else {
+            throw new IllegalArgumentException("Don't know how to convert " + o + " + to double");
+        }
+    }
+
     public static boolean getBoolean(Object o, boolean defaultValue) {
       if (null == o) {
         return defaultValue;


[8/9] storm git commit: Merge branch 'opensource_ras' of https://github.com/jerrypeng/storm into STORM-893

Posted by bo...@apache.org.
Merge branch 'opensource_ras' of https://github.com/jerrypeng/storm into STORM-893

STORM-893: Resource Aware Scheduling


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

Branch: refs/heads/master
Commit: 7c5f0feffb50eff529b0d7fa8eb82da7c0ace8e3
Parents: 3f79c2f 121d022
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Oct 9 09:43:08 2015 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Oct 9 09:43:08 2015 -0500

----------------------------------------------------------------------
 conf/defaults.yaml                              |  14 +
 .../starter/ResourceAwareExampleTopology.java   |  87 ++
 storm-core/src/clj/backtype/storm/converter.clj |   4 +-
 .../src/clj/backtype/storm/daemon/common.clj    |   2 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  29 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |   9 +-
 storm-core/src/jvm/backtype/storm/Config.java   |  63 ++
 .../jvm/backtype/storm/ConfigValidation.java    |  22 +
 .../src/jvm/backtype/storm/StormSubmitter.java  |  35 +-
 .../backtype/storm/generated/Assignment.java    | 194 ++--
 .../storm/generated/BoltAggregateStats.java     |   2 +-
 .../jvm/backtype/storm/generated/BoltStats.java | 442 ++++-----
 .../storm/generated/ClusterSummary.java         | 110 +-
 .../storm/generated/ClusterWorkerHeartbeat.java |  54 +-
 .../storm/generated/CommonAggregateStats.java   |   2 +-
 .../generated/ComponentAggregateStats.java      |   2 +-
 .../storm/generated/ComponentPageInfo.java      | 222 ++---
 .../backtype/storm/generated/Credentials.java   |  46 +-
 .../storm/generated/ExecutorAggregateStats.java |   2 +-
 .../backtype/storm/generated/ExecutorStats.java | 170 ++--
 .../storm/generated/LSApprovedWorkers.java      |  46 +-
 .../generated/LSSupervisorAssignments.java      |  50 +-
 .../storm/generated/LSWorkerHeartbeat.java      |  38 +-
 .../storm/generated/LocalAssignment.java        |  38 +-
 .../storm/generated/LocalStateData.java         |  50 +-
 .../jvm/backtype/storm/generated/LogConfig.java |  50 +-
 .../jvm/backtype/storm/generated/LogLevel.java  |   2 +-
 .../jvm/backtype/storm/generated/Nimbus.java    |   2 +-
 .../jvm/backtype/storm/generated/NodeInfo.java  |  34 +-
 .../storm/generated/RebalanceOptions.java       |  46 +-
 .../storm/generated/SpoutAggregateStats.java    |   2 +-
 .../backtype/storm/generated/SpoutStats.java    | 254 ++---
 .../jvm/backtype/storm/generated/StormBase.java |  94 +-
 .../storm/generated/SupervisorInfo.java         | 276 +++--
 .../storm/generated/SupervisorSummary.java      | 168 +++-
 .../backtype/storm/generated/TopologyInfo.java  | 162 +--
 .../storm/generated/TopologyPageInfo.java       |  98 +-
 .../backtype/storm/generated/TopologyStats.java | 222 ++---
 .../AbstractDNSToSwitchMapping.java             |  95 ++
 .../networktopography/DNSToSwitchMapping.java   |  50 +
 .../DefaultRackDNSToSwitchMapping.java          |  35 +
 .../jvm/backtype/storm/scheduler/Cluster.java   |  46 +-
 .../storm/scheduler/SupervisorDetails.java      |  61 +-
 .../backtype/storm/scheduler/Topologies.java    |  15 +-
 .../storm/scheduler/TopologyDetails.java        | 330 +++++-
 .../storm/scheduler/resource/Component.java     |  54 +
 .../storm/scheduler/resource/RAS_Node.java      | 547 ++++++++++
 .../resource/ResourceAwareScheduler.java        | 149 +++
 .../storm/scheduler/resource/ResourceUtils.java | 133 +++
 .../resource/strategies/IStrategy.java          |  37 +
 .../strategies/ResourceAwareStrategy.java       | 480 +++++++++
 .../AlternateRackDNSToSwitchMapping.java        |  48 +
 .../topology/BaseConfigurationDeclarer.java     |  31 +-
 .../ComponentConfigurationDeclarer.java         |   3 +
 .../src/jvm/backtype/storm/utils/Utils.java     |  19 +
 storm-core/src/py/storm/ttypes.py               | 994 ++++++++++---------
 storm-core/src/storm.thrift                     |   2 +
 .../test/clj/backtype/storm/cluster_test.clj    |   4 +-
 .../scheduler/multitenant_scheduler_test.clj    |  34 +-
 .../scheduler/resource_aware_scheduler_test.clj | 669 +++++++++++++
 .../test/clj/backtype/storm/scheduler_test.clj  |   3 +-
 61 files changed, 5161 insertions(+), 1821 deletions(-)
----------------------------------------------------------------------


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


[6/9] storm git commit: [STORM-893] - Resource Aware Scheduler implementation. [STORM-894] - Basic Resource Aware Scheduling implementation.

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
index e5651df..57543db 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-3-2")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
 public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, ClusterSummary._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterSummary");
 
@@ -581,14 +581,14 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
           case 1: // SUPERVISORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list74 = iprot.readListBegin();
-                struct.supervisors = new ArrayList<SupervisorSummary>(_list74.size);
-                SupervisorSummary _elem75;
-                for (int _i76 = 0; _i76 < _list74.size; ++_i76)
+                org.apache.thrift.protocol.TList _list84 = iprot.readListBegin();
+                struct.supervisors = new ArrayList<SupervisorSummary>(_list84.size);
+                SupervisorSummary _elem85;
+                for (int _i86 = 0; _i86 < _list84.size; ++_i86)
                 {
-                  _elem75 = new SupervisorSummary();
-                  _elem75.read(iprot);
-                  struct.supervisors.add(_elem75);
+                  _elem85 = new SupervisorSummary();
+                  _elem85.read(iprot);
+                  struct.supervisors.add(_elem85);
                 }
                 iprot.readListEnd();
               }
@@ -600,14 +600,14 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
           case 3: // TOPOLOGIES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list77 = iprot.readListBegin();
-                struct.topologies = new ArrayList<TopologySummary>(_list77.size);
-                TopologySummary _elem78;
-                for (int _i79 = 0; _i79 < _list77.size; ++_i79)
+                org.apache.thrift.protocol.TList _list87 = iprot.readListBegin();
+                struct.topologies = new ArrayList<TopologySummary>(_list87.size);
+                TopologySummary _elem88;
+                for (int _i89 = 0; _i89 < _list87.size; ++_i89)
                 {
-                  _elem78 = new TopologySummary();
-                  _elem78.read(iprot);
-                  struct.topologies.add(_elem78);
+                  _elem88 = new TopologySummary();
+                  _elem88.read(iprot);
+                  struct.topologies.add(_elem88);
                 }
                 iprot.readListEnd();
               }
@@ -619,14 +619,14 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
           case 4: // NIMBUSES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list80 = iprot.readListBegin();
-                struct.nimbuses = new ArrayList<NimbusSummary>(_list80.size);
-                NimbusSummary _elem81;
-                for (int _i82 = 0; _i82 < _list80.size; ++_i82)
+                org.apache.thrift.protocol.TList _list90 = iprot.readListBegin();
+                struct.nimbuses = new ArrayList<NimbusSummary>(_list90.size);
+                NimbusSummary _elem91;
+                for (int _i92 = 0; _i92 < _list90.size; ++_i92)
                 {
-                  _elem81 = new NimbusSummary();
-                  _elem81.read(iprot);
-                  struct.nimbuses.add(_elem81);
+                  _elem91 = new NimbusSummary();
+                  _elem91.read(iprot);
+                  struct.nimbuses.add(_elem91);
                 }
                 iprot.readListEnd();
               }
@@ -652,9 +652,9 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
         oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.supervisors.size()));
-          for (SupervisorSummary _iter83 : struct.supervisors)
+          for (SupervisorSummary _iter93 : struct.supervisors)
           {
-            _iter83.write(oprot);
+            _iter93.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -664,9 +664,9 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
         oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topologies.size()));
-          for (TopologySummary _iter84 : struct.topologies)
+          for (TopologySummary _iter94 : struct.topologies)
           {
-            _iter84.write(oprot);
+            _iter94.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -676,9 +676,9 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
         oprot.writeFieldBegin(NIMBUSES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.nimbuses.size()));
-          for (NimbusSummary _iter85 : struct.nimbuses)
+          for (NimbusSummary _iter95 : struct.nimbuses)
           {
-            _iter85.write(oprot);
+            _iter95.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -703,23 +703,23 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.supervisors.size());
-        for (SupervisorSummary _iter86 : struct.supervisors)
+        for (SupervisorSummary _iter96 : struct.supervisors)
         {
-          _iter86.write(oprot);
+          _iter96.write(oprot);
         }
       }
       {
         oprot.writeI32(struct.topologies.size());
-        for (TopologySummary _iter87 : struct.topologies)
+        for (TopologySummary _iter97 : struct.topologies)
         {
-          _iter87.write(oprot);
+          _iter97.write(oprot);
         }
       }
       {
         oprot.writeI32(struct.nimbuses.size());
-        for (NimbusSummary _iter88 : struct.nimbuses)
+        for (NimbusSummary _iter98 : struct.nimbuses)
         {
-          _iter88.write(oprot);
+          _iter98.write(oprot);
         }
       }
     }
@@ -728,38 +728,38 @@ public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, C
     public void read(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list89 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.supervisors = new ArrayList<SupervisorSummary>(_list89.size);
-        SupervisorSummary _elem90;
-        for (int _i91 = 0; _i91 < _list89.size; ++_i91)
+        org.apache.thrift.protocol.TList _list99 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.supervisors = new ArrayList<SupervisorSummary>(_list99.size);
+        SupervisorSummary _elem100;
+        for (int _i101 = 0; _i101 < _list99.size; ++_i101)
         {
-          _elem90 = new SupervisorSummary();
-          _elem90.read(iprot);
-          struct.supervisors.add(_elem90);
+          _elem100 = new SupervisorSummary();
+          _elem100.read(iprot);
+          struct.supervisors.add(_elem100);
         }
       }
       struct.set_supervisors_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list92 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.topologies = new ArrayList<TopologySummary>(_list92.size);
-        TopologySummary _elem93;
-        for (int _i94 = 0; _i94 < _list92.size; ++_i94)
+        org.apache.thrift.protocol.TList _list102 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topologies = new ArrayList<TopologySummary>(_list102.size);
+        TopologySummary _elem103;
+        for (int _i104 = 0; _i104 < _list102.size; ++_i104)
         {
-          _elem93 = new TopologySummary();
-          _elem93.read(iprot);
-          struct.topologies.add(_elem93);
+          _elem103 = new TopologySummary();
+          _elem103.read(iprot);
+          struct.topologies.add(_elem103);
         }
       }
       struct.set_topologies_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list95 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.nimbuses = new ArrayList<NimbusSummary>(_list95.size);
-        NimbusSummary _elem96;
-        for (int _i97 = 0; _i97 < _list95.size; ++_i97)
+        org.apache.thrift.protocol.TList _list105 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.nimbuses = new ArrayList<NimbusSummary>(_list105.size);
+        NimbusSummary _elem106;
+        for (int _i107 = 0; _i107 < _list105.size; ++_i107)
         {
-          _elem96 = new NimbusSummary();
-          _elem96.read(iprot);
-          struct.nimbuses.add(_elem96);
+          _elem106 = new NimbusSummary();
+          _elem106.read(iprot);
+          struct.nimbuses.add(_elem106);
         }
       }
       struct.set_nimbuses_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
index 8c74ca2..0b79fc3 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWorkerHeartbeat, ClusterWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterWorkerHeartbeat> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterWorkerHeartbeat");
 
@@ -635,17 +635,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
           case 2: // EXECUTOR_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map570 = iprot.readMapBegin();
-                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map570.size);
-                ExecutorInfo _key571;
-                ExecutorStats _val572;
-                for (int _i573 = 0; _i573 < _map570.size; ++_i573)
+                org.apache.thrift.protocol.TMap _map590 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map590.size);
+                ExecutorInfo _key591;
+                ExecutorStats _val592;
+                for (int _i593 = 0; _i593 < _map590.size; ++_i593)
                 {
-                  _key571 = new ExecutorInfo();
-                  _key571.read(iprot);
-                  _val572 = new ExecutorStats();
-                  _val572.read(iprot);
-                  struct.executor_stats.put(_key571, _val572);
+                  _key591 = new ExecutorInfo();
+                  _key591.read(iprot);
+                  _val592 = new ExecutorStats();
+                  _val592.read(iprot);
+                  struct.executor_stats.put(_key591, _val592);
                 }
                 iprot.readMapEnd();
               }
@@ -692,10 +692,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
         oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
-          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter574 : struct.executor_stats.entrySet())
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter594 : struct.executor_stats.entrySet())
           {
-            _iter574.getKey().write(oprot);
-            _iter574.getValue().write(oprot);
+            _iter594.getKey().write(oprot);
+            _iter594.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -727,10 +727,10 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       oprot.writeString(struct.storm_id);
       {
         oprot.writeI32(struct.executor_stats.size());
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter575 : struct.executor_stats.entrySet())
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter595 : struct.executor_stats.entrySet())
         {
-          _iter575.getKey().write(oprot);
-          _iter575.getValue().write(oprot);
+          _iter595.getKey().write(oprot);
+          _iter595.getValue().write(oprot);
         }
       }
       oprot.writeI32(struct.time_secs);
@@ -743,17 +743,17 @@ public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWo
       struct.storm_id = iprot.readString();
       struct.set_storm_id_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map576 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map576.size);
-        ExecutorInfo _key577;
-        ExecutorStats _val578;
-        for (int _i579 = 0; _i579 < _map576.size; ++_i579)
+        org.apache.thrift.protocol.TMap _map596 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map596.size);
+        ExecutorInfo _key597;
+        ExecutorStats _val598;
+        for (int _i599 = 0; _i599 < _map596.size; ++_i599)
         {
-          _key577 = new ExecutorInfo();
-          _key577.read(iprot);
-          _val578 = new ExecutorStats();
-          _val578.read(iprot);
-          struct.executor_stats.put(_key577, _val578);
+          _key597 = new ExecutorInfo();
+          _key597.read(iprot);
+          _val598 = new ExecutorStats();
+          _val598.read(iprot);
+          struct.executor_stats.put(_key597, _val598);
         }
       }
       struct.set_executor_stats_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
index 0eea7c2..5aa1c6e 100644
--- a/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class CommonAggregateStats implements org.apache.thrift.TBase<CommonAggregateStats, CommonAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<CommonAggregateStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CommonAggregateStats");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
index ae74b84..0f7336a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class ComponentAggregateStats implements org.apache.thrift.TBase<ComponentAggregateStats, ComponentAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<ComponentAggregateStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentAggregateStats");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
index 1bb080c..b6b6862 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-29")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageInfo, ComponentPageInfo._Fields>, java.io.Serializable, Cloneable, Comparable<ComponentPageInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentPageInfo");
 
@@ -1657,16 +1657,16 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           case 7: // WINDOW_TO_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map404 = iprot.readMapBegin();
-                struct.window_to_stats = new HashMap<String,ComponentAggregateStats>(2*_map404.size);
-                String _key405;
-                ComponentAggregateStats _val406;
-                for (int _i407 = 0; _i407 < _map404.size; ++_i407)
+                org.apache.thrift.protocol.TMap _map414 = iprot.readMapBegin();
+                struct.window_to_stats = new HashMap<String,ComponentAggregateStats>(2*_map414.size);
+                String _key415;
+                ComponentAggregateStats _val416;
+                for (int _i417 = 0; _i417 < _map414.size; ++_i417)
                 {
-                  _key405 = iprot.readString();
-                  _val406 = new ComponentAggregateStats();
-                  _val406.read(iprot);
-                  struct.window_to_stats.put(_key405, _val406);
+                  _key415 = iprot.readString();
+                  _val416 = new ComponentAggregateStats();
+                  _val416.read(iprot);
+                  struct.window_to_stats.put(_key415, _val416);
                 }
                 iprot.readMapEnd();
               }
@@ -1678,17 +1678,17 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           case 8: // GSID_TO_INPUT_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map408 = iprot.readMapBegin();
-                struct.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(2*_map408.size);
-                GlobalStreamId _key409;
-                ComponentAggregateStats _val410;
-                for (int _i411 = 0; _i411 < _map408.size; ++_i411)
+                org.apache.thrift.protocol.TMap _map418 = iprot.readMapBegin();
+                struct.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(2*_map418.size);
+                GlobalStreamId _key419;
+                ComponentAggregateStats _val420;
+                for (int _i421 = 0; _i421 < _map418.size; ++_i421)
                 {
-                  _key409 = new GlobalStreamId();
-                  _key409.read(iprot);
-                  _val410 = new ComponentAggregateStats();
-                  _val410.read(iprot);
-                  struct.gsid_to_input_stats.put(_key409, _val410);
+                  _key419 = new GlobalStreamId();
+                  _key419.read(iprot);
+                  _val420 = new ComponentAggregateStats();
+                  _val420.read(iprot);
+                  struct.gsid_to_input_stats.put(_key419, _val420);
                 }
                 iprot.readMapEnd();
               }
@@ -1700,16 +1700,16 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           case 9: // SID_TO_OUTPUT_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map412 = iprot.readMapBegin();
-                struct.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(2*_map412.size);
-                String _key413;
-                ComponentAggregateStats _val414;
-                for (int _i415 = 0; _i415 < _map412.size; ++_i415)
+                org.apache.thrift.protocol.TMap _map422 = iprot.readMapBegin();
+                struct.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(2*_map422.size);
+                String _key423;
+                ComponentAggregateStats _val424;
+                for (int _i425 = 0; _i425 < _map422.size; ++_i425)
                 {
-                  _key413 = iprot.readString();
-                  _val414 = new ComponentAggregateStats();
-                  _val414.read(iprot);
-                  struct.sid_to_output_stats.put(_key413, _val414);
+                  _key423 = iprot.readString();
+                  _val424 = new ComponentAggregateStats();
+                  _val424.read(iprot);
+                  struct.sid_to_output_stats.put(_key423, _val424);
                 }
                 iprot.readMapEnd();
               }
@@ -1721,14 +1721,14 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           case 10: // EXEC_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list416 = iprot.readListBegin();
-                struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list416.size);
-                ExecutorAggregateStats _elem417;
-                for (int _i418 = 0; _i418 < _list416.size; ++_i418)
+                org.apache.thrift.protocol.TList _list426 = iprot.readListBegin();
+                struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list426.size);
+                ExecutorAggregateStats _elem427;
+                for (int _i428 = 0; _i428 < _list426.size; ++_i428)
                 {
-                  _elem417 = new ExecutorAggregateStats();
-                  _elem417.read(iprot);
-                  struct.exec_stats.add(_elem417);
+                  _elem427 = new ExecutorAggregateStats();
+                  _elem427.read(iprot);
+                  struct.exec_stats.add(_elem427);
                 }
                 iprot.readListEnd();
               }
@@ -1740,14 +1740,14 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           case 11: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list419 = iprot.readListBegin();
-                struct.errors = new ArrayList<ErrorInfo>(_list419.size);
-                ErrorInfo _elem420;
-                for (int _i421 = 0; _i421 < _list419.size; ++_i421)
+                org.apache.thrift.protocol.TList _list429 = iprot.readListBegin();
+                struct.errors = new ArrayList<ErrorInfo>(_list429.size);
+                ErrorInfo _elem430;
+                for (int _i431 = 0; _i431 < _list429.size; ++_i431)
                 {
-                  _elem420 = new ErrorInfo();
-                  _elem420.read(iprot);
-                  struct.errors.add(_elem420);
+                  _elem430 = new ErrorInfo();
+                  _elem430.read(iprot);
+                  struct.errors.add(_elem430);
                 }
                 iprot.readListEnd();
               }
@@ -1841,10 +1841,10 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           oprot.writeFieldBegin(WINDOW_TO_STATS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.window_to_stats.size()));
-            for (Map.Entry<String, ComponentAggregateStats> _iter422 : struct.window_to_stats.entrySet())
+            for (Map.Entry<String, ComponentAggregateStats> _iter432 : struct.window_to_stats.entrySet())
             {
-              oprot.writeString(_iter422.getKey());
-              _iter422.getValue().write(oprot);
+              oprot.writeString(_iter432.getKey());
+              _iter432.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1856,10 +1856,10 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           oprot.writeFieldBegin(GSID_TO_INPUT_STATS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.gsid_to_input_stats.size()));
-            for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter423 : struct.gsid_to_input_stats.entrySet())
+            for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter433 : struct.gsid_to_input_stats.entrySet())
             {
-              _iter423.getKey().write(oprot);
-              _iter423.getValue().write(oprot);
+              _iter433.getKey().write(oprot);
+              _iter433.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1871,10 +1871,10 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           oprot.writeFieldBegin(SID_TO_OUTPUT_STATS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.sid_to_output_stats.size()));
-            for (Map.Entry<String, ComponentAggregateStats> _iter424 : struct.sid_to_output_stats.entrySet())
+            for (Map.Entry<String, ComponentAggregateStats> _iter434 : struct.sid_to_output_stats.entrySet())
             {
-              oprot.writeString(_iter424.getKey());
-              _iter424.getValue().write(oprot);
+              oprot.writeString(_iter434.getKey());
+              _iter434.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1886,9 +1886,9 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           oprot.writeFieldBegin(EXEC_STATS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.exec_stats.size()));
-            for (ExecutorAggregateStats _iter425 : struct.exec_stats)
+            for (ExecutorAggregateStats _iter435 : struct.exec_stats)
             {
-              _iter425.write(oprot);
+              _iter435.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1900,9 +1900,9 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.errors.size()));
-            for (ErrorInfo _iter426 : struct.errors)
+            for (ErrorInfo _iter436 : struct.errors)
             {
-              _iter426.write(oprot);
+              _iter436.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -2010,48 +2010,48 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
       if (struct.is_set_window_to_stats()) {
         {
           oprot.writeI32(struct.window_to_stats.size());
-          for (Map.Entry<String, ComponentAggregateStats> _iter427 : struct.window_to_stats.entrySet())
+          for (Map.Entry<String, ComponentAggregateStats> _iter437 : struct.window_to_stats.entrySet())
           {
-            oprot.writeString(_iter427.getKey());
-            _iter427.getValue().write(oprot);
+            oprot.writeString(_iter437.getKey());
+            _iter437.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_gsid_to_input_stats()) {
         {
           oprot.writeI32(struct.gsid_to_input_stats.size());
-          for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter428 : struct.gsid_to_input_stats.entrySet())
+          for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter438 : struct.gsid_to_input_stats.entrySet())
           {
-            _iter428.getKey().write(oprot);
-            _iter428.getValue().write(oprot);
+            _iter438.getKey().write(oprot);
+            _iter438.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_sid_to_output_stats()) {
         {
           oprot.writeI32(struct.sid_to_output_stats.size());
-          for (Map.Entry<String, ComponentAggregateStats> _iter429 : struct.sid_to_output_stats.entrySet())
+          for (Map.Entry<String, ComponentAggregateStats> _iter439 : struct.sid_to_output_stats.entrySet())
           {
-            oprot.writeString(_iter429.getKey());
-            _iter429.getValue().write(oprot);
+            oprot.writeString(_iter439.getKey());
+            _iter439.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_exec_stats()) {
         {
           oprot.writeI32(struct.exec_stats.size());
-          for (ExecutorAggregateStats _iter430 : struct.exec_stats)
+          for (ExecutorAggregateStats _iter440 : struct.exec_stats)
           {
-            _iter430.write(oprot);
+            _iter440.write(oprot);
           }
         }
       }
       if (struct.is_set_errors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (ErrorInfo _iter431 : struct.errors)
+          for (ErrorInfo _iter441 : struct.errors)
           {
-            _iter431.write(oprot);
+            _iter441.write(oprot);
           }
         }
       }
@@ -2095,77 +2095,77 @@ public class ComponentPageInfo implements org.apache.thrift.TBase<ComponentPageI
       }
       if (incoming.get(4)) {
         {
-          org.apache.thrift.protocol.TMap _map432 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.window_to_stats = new HashMap<String,ComponentAggregateStats>(2*_map432.size);
-          String _key433;
-          ComponentAggregateStats _val434;
-          for (int _i435 = 0; _i435 < _map432.size; ++_i435)
+          org.apache.thrift.protocol.TMap _map442 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.window_to_stats = new HashMap<String,ComponentAggregateStats>(2*_map442.size);
+          String _key443;
+          ComponentAggregateStats _val444;
+          for (int _i445 = 0; _i445 < _map442.size; ++_i445)
           {
-            _key433 = iprot.readString();
-            _val434 = new ComponentAggregateStats();
-            _val434.read(iprot);
-            struct.window_to_stats.put(_key433, _val434);
+            _key443 = iprot.readString();
+            _val444 = new ComponentAggregateStats();
+            _val444.read(iprot);
+            struct.window_to_stats.put(_key443, _val444);
           }
         }
         struct.set_window_to_stats_isSet(true);
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map436 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(2*_map436.size);
-          GlobalStreamId _key437;
-          ComponentAggregateStats _val438;
-          for (int _i439 = 0; _i439 < _map436.size; ++_i439)
+          org.apache.thrift.protocol.TMap _map446 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(2*_map446.size);
+          GlobalStreamId _key447;
+          ComponentAggregateStats _val448;
+          for (int _i449 = 0; _i449 < _map446.size; ++_i449)
           {
-            _key437 = new GlobalStreamId();
-            _key437.read(iprot);
-            _val438 = new ComponentAggregateStats();
-            _val438.read(iprot);
-            struct.gsid_to_input_stats.put(_key437, _val438);
+            _key447 = new GlobalStreamId();
+            _key447.read(iprot);
+            _val448 = new ComponentAggregateStats();
+            _val448.read(iprot);
+            struct.gsid_to_input_stats.put(_key447, _val448);
           }
         }
         struct.set_gsid_to_input_stats_isSet(true);
       }
       if (incoming.get(6)) {
         {
-          org.apache.thrift.protocol.TMap _map440 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(2*_map440.size);
-          String _key441;
-          ComponentAggregateStats _val442;
-          for (int _i443 = 0; _i443 < _map440.size; ++_i443)
+          org.apache.thrift.protocol.TMap _map450 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(2*_map450.size);
+          String _key451;
+          ComponentAggregateStats _val452;
+          for (int _i453 = 0; _i453 < _map450.size; ++_i453)
           {
-            _key441 = iprot.readString();
-            _val442 = new ComponentAggregateStats();
-            _val442.read(iprot);
-            struct.sid_to_output_stats.put(_key441, _val442);
+            _key451 = iprot.readString();
+            _val452 = new ComponentAggregateStats();
+            _val452.read(iprot);
+            struct.sid_to_output_stats.put(_key451, _val452);
           }
         }
         struct.set_sid_to_output_stats_isSet(true);
       }
       if (incoming.get(7)) {
         {
-          org.apache.thrift.protocol.TList _list444 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list444.size);
-          ExecutorAggregateStats _elem445;
-          for (int _i446 = 0; _i446 < _list444.size; ++_i446)
+          org.apache.thrift.protocol.TList _list454 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list454.size);
+          ExecutorAggregateStats _elem455;
+          for (int _i456 = 0; _i456 < _list454.size; ++_i456)
           {
-            _elem445 = new ExecutorAggregateStats();
-            _elem445.read(iprot);
-            struct.exec_stats.add(_elem445);
+            _elem455 = new ExecutorAggregateStats();
+            _elem455.read(iprot);
+            struct.exec_stats.add(_elem455);
           }
         }
         struct.set_exec_stats_isSet(true);
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TList _list447 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.errors = new ArrayList<ErrorInfo>(_list447.size);
-          ErrorInfo _elem448;
-          for (int _i449 = 0; _i449 < _list447.size; ++_i449)
+          org.apache.thrift.protocol.TList _list457 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.errors = new ArrayList<ErrorInfo>(_list457.size);
+          ErrorInfo _elem458;
+          for (int _i459 = 0; _i459 < _list457.size; ++_i459)
           {
-            _elem448 = new ErrorInfo();
-            _elem448.read(iprot);
-            struct.errors.add(_elem448);
+            _elem458 = new ErrorInfo();
+            _elem458.read(iprot);
+            struct.errors.add(_elem458);
           }
         }
         struct.set_errors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Credentials.java b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
index b7d0ee6..afd9e2f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Credentials.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class Credentials implements org.apache.thrift.TBase<Credentials, Credentials._Fields>, java.io.Serializable, Cloneable, Comparable<Credentials> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials");
 
@@ -365,15 +365,15 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
           case 1: // CREDS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map460 = iprot.readMapBegin();
-                struct.creds = new HashMap<String,String>(2*_map460.size);
-                String _key461;
-                String _val462;
-                for (int _i463 = 0; _i463 < _map460.size; ++_i463)
+                org.apache.thrift.protocol.TMap _map470 = iprot.readMapBegin();
+                struct.creds = new HashMap<String,String>(2*_map470.size);
+                String _key471;
+                String _val472;
+                for (int _i473 = 0; _i473 < _map470.size; ++_i473)
                 {
-                  _key461 = iprot.readString();
-                  _val462 = iprot.readString();
-                  struct.creds.put(_key461, _val462);
+                  _key471 = iprot.readString();
+                  _val472 = iprot.readString();
+                  struct.creds.put(_key471, _val472);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
         oprot.writeFieldBegin(CREDS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.creds.size()));
-          for (Map.Entry<String, String> _iter464 : struct.creds.entrySet())
+          for (Map.Entry<String, String> _iter474 : struct.creds.entrySet())
           {
-            oprot.writeString(_iter464.getKey());
-            oprot.writeString(_iter464.getValue());
+            oprot.writeString(_iter474.getKey());
+            oprot.writeString(_iter474.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.creds.size());
-        for (Map.Entry<String, String> _iter465 : struct.creds.entrySet())
+        for (Map.Entry<String, String> _iter475 : struct.creds.entrySet())
         {
-          oprot.writeString(_iter465.getKey());
-          oprot.writeString(_iter465.getValue());
+          oprot.writeString(_iter475.getKey());
+          oprot.writeString(_iter475.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@ public class Credentials implements org.apache.thrift.TBase<Credentials, Credent
     public void read(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map466 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.creds = new HashMap<String,String>(2*_map466.size);
-        String _key467;
-        String _val468;
-        for (int _i469 = 0; _i469 < _map466.size; ++_i469)
+        org.apache.thrift.protocol.TMap _map476 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.creds = new HashMap<String,String>(2*_map476.size);
+        String _key477;
+        String _val478;
+        for (int _i479 = 0; _i479 < _map476.size; ++_i479)
         {
-          _key467 = iprot.readString();
-          _val468 = iprot.readString();
-          struct.creds.put(_key467, _val468);
+          _key477 = iprot.readString();
+          _val478 = iprot.readString();
+          struct.creds.put(_key477, _val478);
         }
       }
       struct.set_creds_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
index 629caa6..b07febf 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class ExecutorAggregateStats implements org.apache.thrift.TBase<ExecutorAggregateStats, ExecutorAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorAggregateStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorAggregateStats");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
index 58a7936..1c78f2d 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
 public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, ExecutorStats._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorStats");
 
@@ -660,27 +660,27 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
           case 1: // EMITTED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map258 = iprot.readMapBegin();
-                struct.emitted = new HashMap<String,Map<String,Long>>(2*_map258.size);
-                String _key259;
-                Map<String,Long> _val260;
-                for (int _i261 = 0; _i261 < _map258.size; ++_i261)
+                org.apache.thrift.protocol.TMap _map268 = iprot.readMapBegin();
+                struct.emitted = new HashMap<String,Map<String,Long>>(2*_map268.size);
+                String _key269;
+                Map<String,Long> _val270;
+                for (int _i271 = 0; _i271 < _map268.size; ++_i271)
                 {
-                  _key259 = iprot.readString();
+                  _key269 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map262 = iprot.readMapBegin();
-                    _val260 = new HashMap<String,Long>(2*_map262.size);
-                    String _key263;
-                    long _val264;
-                    for (int _i265 = 0; _i265 < _map262.size; ++_i265)
+                    org.apache.thrift.protocol.TMap _map272 = iprot.readMapBegin();
+                    _val270 = new HashMap<String,Long>(2*_map272.size);
+                    String _key273;
+                    long _val274;
+                    for (int _i275 = 0; _i275 < _map272.size; ++_i275)
                     {
-                      _key263 = iprot.readString();
-                      _val264 = iprot.readI64();
-                      _val260.put(_key263, _val264);
+                      _key273 = iprot.readString();
+                      _val274 = iprot.readI64();
+                      _val270.put(_key273, _val274);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.emitted.put(_key259, _val260);
+                  struct.emitted.put(_key269, _val270);
                 }
                 iprot.readMapEnd();
               }
@@ -692,27 +692,27 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
           case 2: // TRANSFERRED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map266 = iprot.readMapBegin();
-                struct.transferred = new HashMap<String,Map<String,Long>>(2*_map266.size);
-                String _key267;
-                Map<String,Long> _val268;
-                for (int _i269 = 0; _i269 < _map266.size; ++_i269)
+                org.apache.thrift.protocol.TMap _map276 = iprot.readMapBegin();
+                struct.transferred = new HashMap<String,Map<String,Long>>(2*_map276.size);
+                String _key277;
+                Map<String,Long> _val278;
+                for (int _i279 = 0; _i279 < _map276.size; ++_i279)
                 {
-                  _key267 = iprot.readString();
+                  _key277 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map270 = iprot.readMapBegin();
-                    _val268 = new HashMap<String,Long>(2*_map270.size);
-                    String _key271;
-                    long _val272;
-                    for (int _i273 = 0; _i273 < _map270.size; ++_i273)
+                    org.apache.thrift.protocol.TMap _map280 = iprot.readMapBegin();
+                    _val278 = new HashMap<String,Long>(2*_map280.size);
+                    String _key281;
+                    long _val282;
+                    for (int _i283 = 0; _i283 < _map280.size; ++_i283)
                     {
-                      _key271 = iprot.readString();
-                      _val272 = iprot.readI64();
-                      _val268.put(_key271, _val272);
+                      _key281 = iprot.readString();
+                      _val282 = iprot.readI64();
+                      _val278.put(_key281, _val282);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.transferred.put(_key267, _val268);
+                  struct.transferred.put(_key277, _val278);
                 }
                 iprot.readMapEnd();
               }
@@ -755,15 +755,15 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         oprot.writeFieldBegin(EMITTED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.emitted.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter274 : struct.emitted.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter284 : struct.emitted.entrySet())
           {
-            oprot.writeString(_iter274.getKey());
+            oprot.writeString(_iter284.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter274.getValue().size()));
-              for (Map.Entry<String, Long> _iter275 : _iter274.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter284.getValue().size()));
+              for (Map.Entry<String, Long> _iter285 : _iter284.getValue().entrySet())
               {
-                oprot.writeString(_iter275.getKey());
-                oprot.writeI64(_iter275.getValue());
+                oprot.writeString(_iter285.getKey());
+                oprot.writeI64(_iter285.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -776,15 +776,15 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
         oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.transferred.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter276 : struct.transferred.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter286 : struct.transferred.entrySet())
           {
-            oprot.writeString(_iter276.getKey());
+            oprot.writeString(_iter286.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter276.getValue().size()));
-              for (Map.Entry<String, Long> _iter277 : _iter276.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, _iter286.getValue().size()));
+              for (Map.Entry<String, Long> _iter287 : _iter286.getValue().entrySet())
               {
-                oprot.writeString(_iter277.getKey());
-                oprot.writeI64(_iter277.getValue());
+                oprot.writeString(_iter287.getKey());
+                oprot.writeI64(_iter287.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -820,30 +820,30 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.emitted.size());
-        for (Map.Entry<String, Map<String,Long>> _iter278 : struct.emitted.entrySet())
+        for (Map.Entry<String, Map<String,Long>> _iter288 : struct.emitted.entrySet())
         {
-          oprot.writeString(_iter278.getKey());
+          oprot.writeString(_iter288.getKey());
           {
-            oprot.writeI32(_iter278.getValue().size());
-            for (Map.Entry<String, Long> _iter279 : _iter278.getValue().entrySet())
+            oprot.writeI32(_iter288.getValue().size());
+            for (Map.Entry<String, Long> _iter289 : _iter288.getValue().entrySet())
             {
-              oprot.writeString(_iter279.getKey());
-              oprot.writeI64(_iter279.getValue());
+              oprot.writeString(_iter289.getKey());
+              oprot.writeI64(_iter289.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.transferred.size());
-        for (Map.Entry<String, Map<String,Long>> _iter280 : struct.transferred.entrySet())
+        for (Map.Entry<String, Map<String,Long>> _iter290 : struct.transferred.entrySet())
         {
-          oprot.writeString(_iter280.getKey());
+          oprot.writeString(_iter290.getKey());
           {
-            oprot.writeI32(_iter280.getValue().size());
-            for (Map.Entry<String, Long> _iter281 : _iter280.getValue().entrySet())
+            oprot.writeI32(_iter290.getValue().size());
+            for (Map.Entry<String, Long> _iter291 : _iter290.getValue().entrySet())
             {
-              oprot.writeString(_iter281.getKey());
-              oprot.writeI64(_iter281.getValue());
+              oprot.writeString(_iter291.getKey());
+              oprot.writeI64(_iter291.getValue());
             }
           }
         }
@@ -856,50 +856,50 @@ public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, Exe
     public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorStats struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map282 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.emitted = new HashMap<String,Map<String,Long>>(2*_map282.size);
-        String _key283;
-        Map<String,Long> _val284;
-        for (int _i285 = 0; _i285 < _map282.size; ++_i285)
+        org.apache.thrift.protocol.TMap _map292 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.emitted = new HashMap<String,Map<String,Long>>(2*_map292.size);
+        String _key293;
+        Map<String,Long> _val294;
+        for (int _i295 = 0; _i295 < _map292.size; ++_i295)
         {
-          _key283 = iprot.readString();
+          _key293 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map286 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-            _val284 = new HashMap<String,Long>(2*_map286.size);
-            String _key287;
-            long _val288;
-            for (int _i289 = 0; _i289 < _map286.size; ++_i289)
+            org.apache.thrift.protocol.TMap _map296 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val294 = new HashMap<String,Long>(2*_map296.size);
+            String _key297;
+            long _val298;
+            for (int _i299 = 0; _i299 < _map296.size; ++_i299)
             {
-              _key287 = iprot.readString();
-              _val288 = iprot.readI64();
-              _val284.put(_key287, _val288);
+              _key297 = iprot.readString();
+              _val298 = iprot.readI64();
+              _val294.put(_key297, _val298);
             }
           }
-          struct.emitted.put(_key283, _val284);
+          struct.emitted.put(_key293, _val294);
         }
       }
       struct.set_emitted_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map290 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.transferred = new HashMap<String,Map<String,Long>>(2*_map290.size);
-        String _key291;
-        Map<String,Long> _val292;
-        for (int _i293 = 0; _i293 < _map290.size; ++_i293)
+        org.apache.thrift.protocol.TMap _map300 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.transferred = new HashMap<String,Map<String,Long>>(2*_map300.size);
+        String _key301;
+        Map<String,Long> _val302;
+        for (int _i303 = 0; _i303 < _map300.size; ++_i303)
         {
-          _key291 = iprot.readString();
+          _key301 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map294 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-            _val292 = new HashMap<String,Long>(2*_map294.size);
-            String _key295;
-            long _val296;
-            for (int _i297 = 0; _i297 < _map294.size; ++_i297)
+            org.apache.thrift.protocol.TMap _map304 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val302 = new HashMap<String,Long>(2*_map304.size);
+            String _key305;
+            long _val306;
+            for (int _i307 = 0; _i307 < _map304.size; ++_i307)
             {
-              _key295 = iprot.readString();
-              _val296 = iprot.readI64();
-              _val292.put(_key295, _val296);
+              _key305 = iprot.readString();
+              _val306 = iprot.readI64();
+              _val302.put(_key305, _val306);
             }
           }
-          struct.transferred.put(_key291, _val292);
+          struct.transferred.put(_key301, _val302);
         }
       }
       struct.set_transferred_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
index b7af5ca..a38724d 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWorkers, LSApprovedWorkers._Fields>, java.io.Serializable, Cloneable, Comparable<LSApprovedWorkers> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSApprovedWorkers");
 
@@ -365,15 +365,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
           case 1: // APPROVED_WORKERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map598 = iprot.readMapBegin();
-                struct.approved_workers = new HashMap<String,Integer>(2*_map598.size);
-                String _key599;
-                int _val600;
-                for (int _i601 = 0; _i601 < _map598.size; ++_i601)
+                org.apache.thrift.protocol.TMap _map618 = iprot.readMapBegin();
+                struct.approved_workers = new HashMap<String,Integer>(2*_map618.size);
+                String _key619;
+                int _val620;
+                for (int _i621 = 0; _i621 < _map618.size; ++_i621)
                 {
-                  _key599 = iprot.readString();
-                  _val600 = iprot.readI32();
-                  struct.approved_workers.put(_key599, _val600);
+                  _key619 = iprot.readString();
+                  _val620 = iprot.readI32();
+                  struct.approved_workers.put(_key619, _val620);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
         oprot.writeFieldBegin(APPROVED_WORKERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.approved_workers.size()));
-          for (Map.Entry<String, Integer> _iter602 : struct.approved_workers.entrySet())
+          for (Map.Entry<String, Integer> _iter622 : struct.approved_workers.entrySet())
           {
-            oprot.writeString(_iter602.getKey());
-            oprot.writeI32(_iter602.getValue());
+            oprot.writeString(_iter622.getKey());
+            oprot.writeI32(_iter622.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.approved_workers.size());
-        for (Map.Entry<String, Integer> _iter603 : struct.approved_workers.entrySet())
+        for (Map.Entry<String, Integer> _iter623 : struct.approved_workers.entrySet())
         {
-          oprot.writeString(_iter603.getKey());
-          oprot.writeI32(_iter603.getValue());
+          oprot.writeString(_iter623.getKey());
+          oprot.writeI32(_iter623.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@ public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWork
     public void read(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map604 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.approved_workers = new HashMap<String,Integer>(2*_map604.size);
-        String _key605;
-        int _val606;
-        for (int _i607 = 0; _i607 < _map604.size; ++_i607)
+        org.apache.thrift.protocol.TMap _map624 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.approved_workers = new HashMap<String,Integer>(2*_map624.size);
+        String _key625;
+        int _val626;
+        for (int _i627 = 0; _i627 < _map624.size; ++_i627)
         {
-          _key605 = iprot.readString();
-          _val606 = iprot.readI32();
-          struct.approved_workers.put(_key605, _val606);
+          _key625 = iprot.readString();
+          _val626 = iprot.readI32();
+          struct.approved_workers.put(_key625, _val626);
         }
       }
       struct.set_approved_workers_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
index 728a533..82a9fad 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSupervisorAssignments, LSSupervisorAssignments._Fields>, java.io.Serializable, Cloneable, Comparable<LSSupervisorAssignments> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorAssignments");
 
@@ -376,16 +376,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
           case 1: // ASSIGNMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map608 = iprot.readMapBegin();
-                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map608.size);
-                int _key609;
-                LocalAssignment _val610;
-                for (int _i611 = 0; _i611 < _map608.size; ++_i611)
+                org.apache.thrift.protocol.TMap _map628 = iprot.readMapBegin();
+                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map628.size);
+                int _key629;
+                LocalAssignment _val630;
+                for (int _i631 = 0; _i631 < _map628.size; ++_i631)
                 {
-                  _key609 = iprot.readI32();
-                  _val610 = new LocalAssignment();
-                  _val610.read(iprot);
-                  struct.assignments.put(_key609, _val610);
+                  _key629 = iprot.readI32();
+                  _val630 = new LocalAssignment();
+                  _val630.read(iprot);
+                  struct.assignments.put(_key629, _val630);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
         oprot.writeFieldBegin(ASSIGNMENTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.assignments.size()));
-          for (Map.Entry<Integer, LocalAssignment> _iter612 : struct.assignments.entrySet())
+          for (Map.Entry<Integer, LocalAssignment> _iter632 : struct.assignments.entrySet())
           {
-            oprot.writeI32(_iter612.getKey());
-            _iter612.getValue().write(oprot);
+            oprot.writeI32(_iter632.getKey());
+            _iter632.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.assignments.size());
-        for (Map.Entry<Integer, LocalAssignment> _iter613 : struct.assignments.entrySet())
+        for (Map.Entry<Integer, LocalAssignment> _iter633 : struct.assignments.entrySet())
         {
-          oprot.writeI32(_iter613.getKey());
-          _iter613.getValue().write(oprot);
+          oprot.writeI32(_iter633.getKey());
+          _iter633.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSuperv
     public void read(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map614 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map614.size);
-        int _key615;
-        LocalAssignment _val616;
-        for (int _i617 = 0; _i617 < _map614.size; ++_i617)
+        org.apache.thrift.protocol.TMap _map634 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map634.size);
+        int _key635;
+        LocalAssignment _val636;
+        for (int _i637 = 0; _i637 < _map634.size; ++_i637)
         {
-          _key615 = iprot.readI32();
-          _val616 = new LocalAssignment();
-          _val616.read(iprot);
-          struct.assignments.put(_key615, _val616);
+          _key635 = iprot.readI32();
+          _val636 = new LocalAssignment();
+          _val636.read(iprot);
+          struct.assignments.put(_key635, _val636);
         }
       }
       struct.set_assignments_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
index 6c88563..053707d 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartbeat, LSWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<LSWorkerHeartbeat> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSWorkerHeartbeat");
 
@@ -638,14 +638,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
           case 3: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list618.size);
-                ExecutorInfo _elem619;
-                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
+                org.apache.thrift.protocol.TList _list638 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list638.size);
+                ExecutorInfo _elem639;
+                for (int _i640 = 0; _i640 < _list638.size; ++_i640)
                 {
-                  _elem619 = new ExecutorInfo();
-                  _elem619.read(iprot);
-                  struct.executors.add(_elem619);
+                  _elem639 = new ExecutorInfo();
+                  _elem639.read(iprot);
+                  struct.executors.add(_elem639);
                 }
                 iprot.readListEnd();
               }
@@ -687,9 +687,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter621 : struct.executors)
+          for (ExecutorInfo _iter641 : struct.executors)
           {
-            _iter621.write(oprot);
+            _iter641.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter622 : struct.executors)
+        for (ExecutorInfo _iter642 : struct.executors)
         {
-          _iter622.write(oprot);
+          _iter642.write(oprot);
         }
       }
       oprot.writeI32(struct.port);
@@ -735,14 +735,14 @@ public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartb
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list623.size);
-        ExecutorInfo _elem624;
-        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
+        org.apache.thrift.protocol.TList _list643 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list643.size);
+        ExecutorInfo _elem644;
+        for (int _i645 = 0; _i645 < _list643.size; ++_i645)
         {
-          _elem624 = new ExecutorInfo();
-          _elem624.read(iprot);
-          struct.executors.add(_elem624);
+          _elem644 = new ExecutorInfo();
+          _elem644.read(iprot);
+          struct.executors.add(_elem644);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
index 65068bc..f7b034f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment, LocalAssignment._Fields>, java.io.Serializable, Cloneable, Comparable<LocalAssignment> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalAssignment");
 
@@ -464,14 +464,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
           case 2: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list590 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list590.size);
-                ExecutorInfo _elem591;
-                for (int _i592 = 0; _i592 < _list590.size; ++_i592)
+                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list610.size);
+                ExecutorInfo _elem611;
+                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
                 {
-                  _elem591 = new ExecutorInfo();
-                  _elem591.read(iprot);
-                  struct.executors.add(_elem591);
+                  _elem611 = new ExecutorInfo();
+                  _elem611.read(iprot);
+                  struct.executors.add(_elem611);
                 }
                 iprot.readListEnd();
               }
@@ -502,9 +502,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter593 : struct.executors)
+          for (ExecutorInfo _iter613 : struct.executors)
           {
-            _iter593.write(oprot);
+            _iter613.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -530,9 +530,9 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter594 : struct.executors)
+        for (ExecutorInfo _iter614 : struct.executors)
         {
-          _iter594.write(oprot);
+          _iter614.write(oprot);
         }
       }
     }
@@ -543,14 +543,14 @@ public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment,
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list595 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list595.size);
-        ExecutorInfo _elem596;
-        for (int _i597 = 0; _i597 < _list595.size; ++_i597)
+        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list615.size);
+        ExecutorInfo _elem616;
+        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
         {
-          _elem596 = new ExecutorInfo();
-          _elem596.read(iprot);
-          struct.executors.add(_elem596);
+          _elem616 = new ExecutorInfo();
+          _elem616.read(iprot);
+          struct.executors.add(_elem616);
         }
       }
       struct.set_executors_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
index 1589379..eb1ca97 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, LocalStateData._Fields>, java.io.Serializable, Cloneable, Comparable<LocalStateData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalStateData");
 
@@ -376,16 +376,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
           case 1: // SERIALIZED_PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map580 = iprot.readMapBegin();
-                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map580.size);
-                String _key581;
-                ThriftSerializedObject _val582;
-                for (int _i583 = 0; _i583 < _map580.size; ++_i583)
+                org.apache.thrift.protocol.TMap _map600 = iprot.readMapBegin();
+                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map600.size);
+                String _key601;
+                ThriftSerializedObject _val602;
+                for (int _i603 = 0; _i603 < _map600.size; ++_i603)
                 {
-                  _key581 = iprot.readString();
-                  _val582 = new ThriftSerializedObject();
-                  _val582.read(iprot);
-                  struct.serialized_parts.put(_key581, _val582);
+                  _key601 = iprot.readString();
+                  _val602 = new ThriftSerializedObject();
+                  _val602.read(iprot);
+                  struct.serialized_parts.put(_key601, _val602);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
         oprot.writeFieldBegin(SERIALIZED_PARTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.serialized_parts.size()));
-          for (Map.Entry<String, ThriftSerializedObject> _iter584 : struct.serialized_parts.entrySet())
+          for (Map.Entry<String, ThriftSerializedObject> _iter604 : struct.serialized_parts.entrySet())
           {
-            oprot.writeString(_iter584.getKey());
-            _iter584.getValue().write(oprot);
+            oprot.writeString(_iter604.getKey());
+            _iter604.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.serialized_parts.size());
-        for (Map.Entry<String, ThriftSerializedObject> _iter585 : struct.serialized_parts.entrySet())
+        for (Map.Entry<String, ThriftSerializedObject> _iter605 : struct.serialized_parts.entrySet())
         {
-          oprot.writeString(_iter585.getKey());
-          _iter585.getValue().write(oprot);
+          oprot.writeString(_iter605.getKey());
+          _iter605.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@ public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, L
     public void read(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map586 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map586.size);
-        String _key587;
-        ThriftSerializedObject _val588;
-        for (int _i589 = 0; _i589 < _map586.size; ++_i589)
+        org.apache.thrift.protocol.TMap _map606 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map606.size);
+        String _key607;
+        ThriftSerializedObject _val608;
+        for (int _i609 = 0; _i609 < _map606.size; ++_i609)
         {
-          _key587 = iprot.readString();
-          _val588 = new ThriftSerializedObject();
-          _val588.read(iprot);
-          struct.serialized_parts.put(_key587, _val588);
+          _key607 = iprot.readString();
+          _val608 = new ThriftSerializedObject();
+          _val608.read(iprot);
+          struct.serialized_parts.put(_key607, _val608);
         }
       }
       struct.set_serialized_parts_isSet(true);


[9/9] storm git commit: Added STORM-893 to Changelog

Posted by bo...@apache.org.
Added STORM-893 to Changelog


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

Branch: refs/heads/master
Commit: 86f2d03c2060f25ccdaf7580cfee5d9f1a9ac2e3
Parents: 7c5f0fe
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Oct 9 09:52:44 2015 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Oct 9 09:52:44 2015 -0500

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


http://git-wip-us.apache.org/repos/asf/storm/blob/86f2d03c/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 1fb0633..fe367a4 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-893: Resource Aware Scheduling (Experimental)
  * STORM-1095: Tuple.getSourceGlobalStreamid() has wrong camel-case naming
  * STORM-1091: Add unit test for tick tuples to HiveBolt and HdfsBolt
  * STORM-1090: Nimbus HA should support `storm.local.hostname`


[7/9] storm git commit: [STORM-893] - Resource Aware Scheduler implementation. [STORM-894] - Basic Resource Aware Scheduling implementation.

Posted by bo...@apache.org.
[STORM-893] - Resource Aware Scheduler implementation.
[STORM-894] - Basic Resource Aware Scheduling implementation.

Added functionality for users to limit the amount of memory resources allocated to a worker (JVM) process when scheduling with resource aware scheduler. This allows users to potentially spread executors more evenly across workers.
Also refactored some code


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

Branch: refs/heads/master
Commit: 121d022b9f11146f6dadc2cd402c747472cac0d1
Parents: 1822491
Author: Boyang Jerry Peng <je...@yahoo-inc.com>
Authored: Tue Oct 6 22:23:18 2015 -0500
Committer: Boyang Jerry Peng <je...@yahoo-inc.com>
Committed: Fri Oct 9 08:44:20 2015 -0500

----------------------------------------------------------------------
 conf/defaults.yaml                              |  14 +
 .../starter/ResourceAwareExampleTopology.java   |  87 ++
 storm-core/src/clj/backtype/storm/converter.clj |   4 +-
 .../src/clj/backtype/storm/daemon/common.clj    |   2 +-
 .../src/clj/backtype/storm/daemon/nimbus.clj    |  29 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |   9 +-
 storm-core/src/jvm/backtype/storm/Config.java   |  63 ++
 .../jvm/backtype/storm/ConfigValidation.java    |  22 +
 .../src/jvm/backtype/storm/StormSubmitter.java  |  35 +-
 .../backtype/storm/generated/Assignment.java    | 194 ++--
 .../storm/generated/BoltAggregateStats.java     |   2 +-
 .../jvm/backtype/storm/generated/BoltStats.java | 442 ++++-----
 .../storm/generated/ClusterSummary.java         | 110 +-
 .../storm/generated/ClusterWorkerHeartbeat.java |  54 +-
 .../storm/generated/CommonAggregateStats.java   |   2 +-
 .../generated/ComponentAggregateStats.java      |   2 +-
 .../storm/generated/ComponentPageInfo.java      | 222 ++---
 .../backtype/storm/generated/Credentials.java   |  46 +-
 .../storm/generated/ExecutorAggregateStats.java |   2 +-
 .../backtype/storm/generated/ExecutorStats.java | 170 ++--
 .../storm/generated/LSApprovedWorkers.java      |  46 +-
 .../generated/LSSupervisorAssignments.java      |  50 +-
 .../storm/generated/LSWorkerHeartbeat.java      |  38 +-
 .../storm/generated/LocalAssignment.java        |  38 +-
 .../storm/generated/LocalStateData.java         |  50 +-
 .../jvm/backtype/storm/generated/LogConfig.java |  50 +-
 .../jvm/backtype/storm/generated/LogLevel.java  |   2 +-
 .../jvm/backtype/storm/generated/Nimbus.java    |   2 +-
 .../jvm/backtype/storm/generated/NodeInfo.java  |  34 +-
 .../storm/generated/RebalanceOptions.java       |  46 +-
 .../storm/generated/SpoutAggregateStats.java    |   2 +-
 .../backtype/storm/generated/SpoutStats.java    | 254 ++---
 .../jvm/backtype/storm/generated/StormBase.java |  94 +-
 .../storm/generated/SupervisorInfo.java         | 276 +++--
 .../storm/generated/SupervisorSummary.java      | 168 +++-
 .../backtype/storm/generated/TopologyInfo.java  | 162 +--
 .../storm/generated/TopologyPageInfo.java       |  98 +-
 .../backtype/storm/generated/TopologyStats.java | 222 ++---
 .../AbstractDNSToSwitchMapping.java             |  95 ++
 .../networktopography/DNSToSwitchMapping.java   |  50 +
 .../DefaultRackDNSToSwitchMapping.java          |  35 +
 .../jvm/backtype/storm/scheduler/Cluster.java   |  46 +-
 .../storm/scheduler/SupervisorDetails.java      |  61 +-
 .../backtype/storm/scheduler/Topologies.java    |  15 +-
 .../storm/scheduler/TopologyDetails.java        | 330 +++++-
 .../storm/scheduler/resource/Component.java     |  54 +
 .../storm/scheduler/resource/RAS_Node.java      | 547 ++++++++++
 .../resource/ResourceAwareScheduler.java        | 149 +++
 .../storm/scheduler/resource/ResourceUtils.java | 133 +++
 .../resource/strategies/IStrategy.java          |  37 +
 .../strategies/ResourceAwareStrategy.java       | 480 +++++++++
 .../AlternateRackDNSToSwitchMapping.java        |  48 +
 .../topology/BaseConfigurationDeclarer.java     |  31 +-
 .../ComponentConfigurationDeclarer.java         |   3 +
 .../src/jvm/backtype/storm/utils/Utils.java     |  19 +
 storm-core/src/py/storm/ttypes.py               | 994 ++++++++++---------
 storm-core/src/storm.thrift                     |   2 +
 .../test/clj/backtype/storm/cluster_test.clj    |   4 +-
 .../scheduler/multitenant_scheduler_test.clj    |  34 +-
 .../scheduler/resource_aware_scheduler_test.clj | 669 +++++++++++++
 .../test/clj/backtype/storm/scheduler_test.clj  |   3 +-
 61 files changed, 5161 insertions(+), 1821 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 4a8e354..a41d805 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -132,6 +132,11 @@ supervisor.heartbeat.frequency.secs: 5
 supervisor.enable: true
 supervisor.supervisors: []
 supervisor.supervisors.commands: []
+supervisor.memory.capacity.mb: 3072.0
+#By convention 1 cpu core should be about 100, but this can be adjusted if needed
+# using 100 makes it simple to set the desired value to the capacity measurement
+# for single threaded bolts
+supervisor.cpu.capacity: 400.0
 
 ### worker.* configs are for task workers
 worker.childopts: "-Xmx768m"
@@ -174,6 +179,9 @@ storm.messaging.netty.socket.backlog: 500
 # By default, the Netty SASL authentication is set to false.  Users can override and set it true for a specific topology.
 storm.messaging.netty.authentication: false
 
+# Default plugin to use for automatic network topology discovery
+storm.network.topography.plugin: backtype.storm.networktopography.DefaultRackDNSToSwitchMapping
+
 # default number of seconds group mapping service will cache user group
 storm.group.mapping.service.cache.duration.secs: 120
 
@@ -215,4 +223,10 @@ topology.environment: null
 topology.bolts.outgoing.overflow.buffer.enable: false
 topology.disruptor.wait.timeout.millis: 1000
 
+# Configs for Resource Aware Scheduler
+topology.component.resources.onheap.memory.mb: 128.0
+topology.component.resources.offheap.memory.mb: 0.0
+topology.component.cpu.pcore.percent: 10.0
+topology.worker.max.heap.size.mb: 768.0
+
 dev.zookeeper.path: "/tmp/dev-storm-zookeeper"

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
new file mode 100644
index 0000000..96e300f
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
@@ -0,0 +1,87 @@
+package storm.starter;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.testing.TestWordSpout;
+import backtype.storm.topology.BoltDeclarer;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.SpoutDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
+
+import java.util.Map;
+
+/**
+ * Created by jerrypeng on 8/19/15.
+ */
+public class ResourceAwareExampleTopology {
+  public static class ExclamationBolt extends BaseRichBolt {
+    OutputCollector _collector;
+
+    @Override
+    public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+      _collector = collector;
+    }
+
+    @Override
+    public void execute(Tuple tuple) {
+      _collector.emit(tuple, new Values(tuple.getString(0) + "!!!"));
+      _collector.ack(tuple);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("word"));
+    }
+
+
+  }
+
+  public static void main(String[] args) throws Exception {
+    TopologyBuilder builder = new TopologyBuilder();
+
+    SpoutDeclarer spout =  builder.setSpout("word", new TestWordSpout(), 10);
+    //set cpu requirement
+    spout.setCPULoad(20);
+    //set onheap and offheap memory requirement
+    spout.setMemoryLoad(64, 16);
+
+    BoltDeclarer bolt1 = builder.setBolt("exclaim1", new ExclamationBolt(), 3).shuffleGrouping("word");
+    //sets cpu requirement.  Not neccessary to set both CPU and memory.
+    //For requirements not set, a default value will be used
+    bolt1.setCPULoad(15);
+
+    BoltDeclarer bolt2 = builder.setBolt("exclaim2", new ExclamationBolt(), 2).shuffleGrouping("exclaim1");
+    bolt2.setMemoryLoad(100);
+
+    Config conf = new Config();
+    conf.setDebug(true);
+
+    /**
+     * Use to limit the maximum amount of memory (in MB) allocated to one worker process.
+     * Can be used to spread executors to to multiple workers
+     */
+    conf.setTopologyWorkerMaxHeapSize(512.0);
+
+    if (args != null && args.length > 0) {
+      conf.setNumWorkers(3);
+
+      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+    }
+    else {
+
+      LocalCluster cluster = new LocalCluster();
+      cluster.submitTopology("test", conf, builder.createTopology());
+      Utils.sleep(10000);
+      cluster.killTopology("test");
+      cluster.shutdown();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/clj/backtype/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index c571fa1..27336f0 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -30,6 +30,7 @@
     (.set_scheduler_meta (:scheduler-meta supervisor-info))
     (.set_uptime_secs (long (:uptime-secs supervisor-info)))
     (.set_version (:version supervisor-info))
+    (.set_resources_map (:resources-map supervisor-info))
     ))
 
 (defn clojurify-supervisor-info [^SupervisorInfo supervisor-info]
@@ -42,7 +43,8 @@
       (if (.get_meta supervisor-info) (into [] (.get_meta supervisor-info)))
       (if (.get_scheduler_meta supervisor-info) (into {} (.get_scheduler_meta supervisor-info)))
       (.get_uptime_secs supervisor-info)
-      (.get_version supervisor-info))))
+      (.get_version supervisor-info)
+      (if-let [res-map (.get_resources_map supervisor-info)] (into {} res-map)))))
 
 (defn thriftify-assignment [assignment]
   (doto (Assignment.)

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

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 70bd197..61aba6b 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -333,7 +333,7 @@
         supervisor-infos (all-supervisor-info storm-cluster-state nil)
 
         supervisor-details (dofor [[id info] supervisor-infos]
-                             (SupervisorDetails. id (:meta info)))
+                             (SupervisorDetails. id (:meta info) (:resources-map info)))
 
         ret (.allSlotsAvailableForScheduling inimbus
                      supervisor-details
@@ -568,7 +568,8 @@
                                                     all-ports (-> (get all-scheduling-slots sid)
                                                                   (set/difference dead-ports)
                                                                   ((fn [ports] (map int ports))))
-                                                    supervisor-details (SupervisorDetails. sid hostname scheduler-meta all-ports)]]
+                                                    supervisor-details (SupervisorDetails. sid hostname scheduler-meta all-ports (:resources-map supervisor-info))
+                                                    ]]
                                           {sid supervisor-details}))]
     (merge all-supervisor-details
            (into {}
@@ -652,7 +653,7 @@
                                   (apply merge-with set/union))
 
         supervisors (read-all-supervisor-details nimbus all-scheduling-slots supervisor->dead-ports)
-        cluster (Cluster. (:inimbus nimbus) supervisors topology->scheduler-assignment)
+        cluster (Cluster. (:inimbus nimbus) supervisors topology->scheduler-assignment conf)
 
         ;; call scheduler.schedule to schedule all the topologies
         ;; the new assignments for all the topologies are in the cluster object.
@@ -701,7 +702,7 @@
   (let [infos (all-supervisor-info storm-cluster-state)]
     (->> infos
          (map (fn [[id info]]
-                 [id (SupervisorDetails. id (:hostname info) (:scheduler-meta info) nil)]))
+                 [id (SupervisorDetails. id (:hostname info) (:scheduler-meta info) nil (:resources-map info))]))
          (into {}))))
 
 (defn- to-worker-slot [[node port]]
@@ -1433,16 +1434,16 @@
               ;; TODO: need to get the port info about supervisors...
               ;; in standalone just look at metadata, otherwise just say N/A?
               supervisor-summaries (dofor [[id info] supervisor-infos]
-                                          (let [ports (set (:meta info)) ;;TODO: this is only true for standalone
-
-                                            sup-sum (SupervisorSummary. (:hostname info)
-                                                                (:uptime-secs info)
-                                                                (count ports)
-                                                                (count (:used-ports info))
-                                                                id) ]
-                                            (when-let [version (:version info)] (.set_version sup-sum version))
-                                            sup-sum
-                                            ))
+                                     (let [ports (set (:meta info)) ;;TODO: this is only true for standalone
+                                           sup-sum (SupervisorSummary. (:hostname info)
+                                                     (:uptime-secs info)
+                                                     (count ports)
+                                                     (count (:used-ports info))
+                                                     id) ]
+                                       (.set_total_resources sup-sum (map-val double (:resources-map info)))
+                                       (when-let [version (:version info)] (.set_version sup-sum version))
+                                       sup-sum))
+              nimbus-uptime ((:uptime nimbus))
               bases (topology-bases storm-cluster-state)
               nimbuses (.nimbuses storm-cluster-state)
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index c78e865..3eed36e 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -25,6 +25,7 @@
            [java.io File])
   (:use [backtype.storm config util log timer local-state])
   (:import [backtype.storm.utils VersionInfo])
+  (:import [backtype.storm Config])
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm.daemon [worker :as worker]]
             [backtype.storm [process-simulator :as psim] [cluster :as cluster] [event :as event]]
@@ -495,6 +496,11 @@
       (.add processes-event-manager sync-processes)
       )))
 
+(defn mk-supervisor-capacities
+  [conf]
+  {Config/SUPERVISOR_MEMORY_CAPACITY_MB (double (conf SUPERVISOR-MEMORY-CAPACITY-MB))
+   Config/SUPERVISOR_CPU_CAPACITY (double (conf SUPERVISOR-CPU-CAPACITY))})
+
 ;; in local state, supervisor stores who its current assignments are
 ;; another thread launches events to restart any dead processes if necessary
 (defserverfn mk-supervisor [conf shared-context ^ISupervisor isupervisor]
@@ -516,7 +522,8 @@
                                                  (.getMetadata isupervisor)
                                                  (conf SUPERVISOR-SCHEDULER-META)
                                                  ((:uptime supervisor))
-                                                 (:version supervisor))))]
+                                                 (:version supervisor)
+                                                 (mk-supervisor-capacities conf))))]
     (heartbeat-fn)
 
     ;; should synchronize supervisor so it doesn't launch anything after being down (optimization)

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/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 c805d03..26a6681 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -169,6 +169,15 @@ public class Config extends HashMap<String, Object> {
     public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class;
 
     /**
+     * What Network Topography detection classes should we use.
+     * Given a list of supervisor hostnames (or IP addresses), this class would return a list of
+     * rack names that correspond to the supervisors. This information is stored in Cluster.java, and
+     * is used in the resource aware scheduler.
+     */
+    public static final String STORM_NETWORK_TOPOGRAPHY_PLUGIN = "storm.network.topography.plugin";
+    public static final Object STORM_NETWORK_TOPOGRAPHY_PLUGIN_SCHEMA = String.class;
+
+    /**
      * The hostname the supervisors/workers should report to nimbus. If unset, Storm will
      * get the hostname to report by calling <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
      *
@@ -993,6 +1002,22 @@ public class Config extends HashMap<String, Object> {
     public static final Object SUPERVISOR_WORKER_LAUNCHER_SCHEMA = String.class;
 
     /**
+     * The total amount of memory (in MiB) a supervisor is allowed to give to its workers.
+     *  A default value will be set for this config if user does not override
+     */
+    public static final String SUPERVISOR_MEMORY_CAPACITY_MB = "supervisor.memory.capacity.mb";
+    public static final Object SUPERVISOR_MEMORY_CAPACITY_MB_SCHEMA = ConfigValidation.PositiveNumberValidator;
+
+    /**
+     * The total amount of CPU resources a supervisor is allowed to give to its workers.
+     * By convention 1 cpu core should be about 100, but this can be adjusted if needed
+     * using 100 makes it simple to set the desired value to the capacity measurement
+     * for single threaded bolts.  A default value will be set for this config if user does not override
+     */
+    public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
+    public static final Object SUPERVISOR_CPU_CAPACITY_SCHEMA = ConfigValidation.PositiveNumberValidator;
+
+    /**
      * The jvm opts provided to workers launched by this supervisor. All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%"
      * and "%WORKER-PORT%" substrings are replaced with:
      * %ID%          -> port (for backward compatibility),
@@ -1133,6 +1158,34 @@ public class Config extends HashMap<String, Object> {
     public static final Object TOPOLOGY_TASKS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
+     * The maximum amount of memory an instance of a spout/bolt will take on heap. This enables the scheduler
+     * to allocate slots on machines with enough available memory. A default value will be set for this config if user does not override
+     */
+    public static final String TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB = "topology.component.resources.onheap.memory.mb";
+    public static final Object TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB_SCHEMA = ConfigValidation.NonNegativeNumberValidator;
+
+    /**
+     * The maximum amount of memory an instance of a spout/bolt will take off heap. This enables the scheduler
+     * to allocate slots on machines with enough available memory.  A default value will be set for this config if user does not override
+     */
+    public static final String TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB = "topology.component.resources.offheap.memory.mb";
+    public static final Object TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB_SCHEMA = ConfigValidation.NonNegativeNumberValidator;
+
+    /**
+     * The config indicates the percentage of cpu for a core an instance(executor) of a component will use.
+     * Assuming the a core value to be 100, a value of 10 indicates 10% of the core.
+     * The P in PCORE represents the term "physical".  A default value will be set for this config if user does not override
+     */
+    public static final String TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT = "topology.component.cpu.pcore.percent";
+    public static final Object TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT_SCHEMA = ConfigValidation.NonNegativeNumberValidator;
+
+    /**
+     * A per topology config that specifies the maximum amount of memory a worker can use for that specific topology
+     */
+    public static final String TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB = "topology.worker.max.heap.size.mb";
+    public static final Object TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB_SCHEMA = ConfigValidation.PositiveNumberValidator;
+
+    /**
      * How many executors to spawn for ackers.
      *
      * <p>By not setting this variable or setting it as null, Storm will set the number of acker executors
@@ -1755,4 +1808,14 @@ public class Config extends HashMap<String, Object> {
         conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret);
         return ret;
     }
+
+    /**
+     * set the max heap size allow per worker for this topology
+     * @param size
+     */
+    public void setTopologyWorkerMaxHeapSize(Number size) {
+        if(size != null) {
+            this.put(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB, size);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
index fd9dae7..e6c0986 100644
--- a/storm-core/src/jvm/backtype/storm/ConfigValidation.java
+++ b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
@@ -261,6 +261,28 @@ public class ConfigValidation {
     };
 
     /**
+     * Validates a non negative double value.
+     */
+    public static Object NonNegativeNumberValidator = new FieldValidator() {
+        @Override
+        public void validateField(String name, Object o) throws IllegalArgumentException {
+            if (o == null) {
+                // A null value is acceptable.
+                return;
+            }
+            final double double_value;
+            if (o instanceof Number)
+            {
+                double_value = ((Number)o).doubleValue();
+                if (double_value >= 0.0) {
+                    return;
+                }
+            }
+            throw new IllegalArgumentException("Field " + name + " must be a non-negative double.");
+        }
+    };
+
+    /**
      * Validates a Positive Number
      */
     public static Object PositiveNumberValidator = new FieldValidator() {

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/StormSubmitter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/StormSubmitter.java b/storm-core/src/jvm/backtype/storm/StormSubmitter.java
index a4ccf5f..8afbbc6 100644
--- a/storm-core/src/jvm/backtype/storm/StormSubmitter.java
+++ b/storm-core/src/jvm/backtype/storm/StormSubmitter.java
@@ -24,6 +24,7 @@ import java.util.regex.Pattern;
 import java.util.HashMap;
 import java.util.Map;
 
+import backtype.storm.scheduler.resource.ResourceUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.thrift.TException;
 import org.json.simple.JSONValue;
@@ -185,9 +186,10 @@ public class StormSubmitter {
      * @throws AlreadyAliveException
      * @throws InvalidTopologyException
      * @throws AuthorizationException
+     * @throws IllegalArgumentException thrown if configs will yield an unschedulable topology. validateConfs validates confs
      */
     public static void submitTopologyAs(String name, Map stormConf, StormTopology topology, SubmitOptions opts, ProgressListener progressListener, String asUser)
-            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
+            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, IllegalArgumentException {
         if(!Utils.isValidConf(stormConf)) {
             throw new IllegalArgumentException("Storm conf is not valid. Must be json-serializable");
         }
@@ -197,6 +199,8 @@ public class StormSubmitter {
         conf.putAll(stormConf);
         stormConf.putAll(prepareZookeeperAuthentication(conf));
 
+        validateConfs(conf, topology);
+
         Map<String,String> passedCreds = new HashMap<String, String>();
         if (opts != null) {
             Credentials tmpCreds = opts.get_creds();
@@ -442,4 +446,33 @@ public class StormSubmitter {
          */
         public void onCompleted(String srcFile, String targetFile, long totalBytes);
     }
+
+    private static void validateConfs(Map stormConf, StormTopology topology) throws IllegalArgumentException {
+        double largestMemReq = getMaxExecutorMemoryUsageForTopo(topology, stormConf);
+        Double topologyWorkerMaxHeapSize = Utils.getDouble(stormConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB));
+        if(topologyWorkerMaxHeapSize < largestMemReq) {
+            throw new IllegalArgumentException("Topology will not be able to be successfully scheduled: Config TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB="
+                    +Utils.getDouble(stormConf.get(Config.TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB)) + " < " 
+                            + largestMemReq + " (Largest memory requirement of a component in the topology). Perhaps set TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB to a larger amount");
+        }
+    }
+
+    private static double getMaxExecutorMemoryUsageForTopo(StormTopology topology, Map topologyConf) {
+        double largestMemoryOperator = 0.0;
+        for(Map<String, Double> entry : ResourceUtils.getBoltsResources(topology, topologyConf).values()) {
+            double memoryRequirement = entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)
+                    + entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
+            if(memoryRequirement > largestMemoryOperator) {
+                largestMemoryOperator = memoryRequirement;
+            }
+        }
+        for(Map<String, Double> entry : ResourceUtils.getSpoutsResources(topology, topologyConf).values()) {
+            double memoryRequirement = entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB)
+                    + entry.get(Config.TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB);
+            if(memoryRequirement > largestMemoryOperator) {
+                largestMemoryOperator = memoryRequirement;
+            }
+        }
+        return largestMemoryOperator;
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/Assignment.java b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
index 7555b54..6405c05 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Assignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-28")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable, Comparable<Assignment> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment");
 
@@ -678,15 +678,15 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 2: // NODE_HOST
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map504 = iprot.readMapBegin();
-                struct.node_host = new HashMap<String,String>(2*_map504.size);
-                String _key505;
-                String _val506;
-                for (int _i507 = 0; _i507 < _map504.size; ++_i507)
+                org.apache.thrift.protocol.TMap _map524 = iprot.readMapBegin();
+                struct.node_host = new HashMap<String,String>(2*_map524.size);
+                String _key525;
+                String _val526;
+                for (int _i527 = 0; _i527 < _map524.size; ++_i527)
                 {
-                  _key505 = iprot.readString();
-                  _val506 = iprot.readString();
-                  struct.node_host.put(_key505, _val506);
+                  _key525 = iprot.readString();
+                  _val526 = iprot.readString();
+                  struct.node_host.put(_key525, _val526);
                 }
                 iprot.readMapEnd();
               }
@@ -698,26 +698,26 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 3: // EXECUTOR_NODE_PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map508 = iprot.readMapBegin();
-                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map508.size);
-                List<Long> _key509;
-                NodeInfo _val510;
-                for (int _i511 = 0; _i511 < _map508.size; ++_i511)
+                org.apache.thrift.protocol.TMap _map528 = iprot.readMapBegin();
+                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map528.size);
+                List<Long> _key529;
+                NodeInfo _val530;
+                for (int _i531 = 0; _i531 < _map528.size; ++_i531)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list512 = iprot.readListBegin();
-                    _key509 = new ArrayList<Long>(_list512.size);
-                    long _elem513;
-                    for (int _i514 = 0; _i514 < _list512.size; ++_i514)
+                    org.apache.thrift.protocol.TList _list532 = iprot.readListBegin();
+                    _key529 = new ArrayList<Long>(_list532.size);
+                    long _elem533;
+                    for (int _i534 = 0; _i534 < _list532.size; ++_i534)
                     {
-                      _elem513 = iprot.readI64();
-                      _key509.add(_elem513);
+                      _elem533 = iprot.readI64();
+                      _key529.add(_elem533);
                     }
                     iprot.readListEnd();
                   }
-                  _val510 = new NodeInfo();
-                  _val510.read(iprot);
-                  struct.executor_node_port.put(_key509, _val510);
+                  _val530 = new NodeInfo();
+                  _val530.read(iprot);
+                  struct.executor_node_port.put(_key529, _val530);
                 }
                 iprot.readMapEnd();
               }
@@ -729,25 +729,25 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           case 4: // EXECUTOR_START_TIME_SECS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map515 = iprot.readMapBegin();
-                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map515.size);
-                List<Long> _key516;
-                long _val517;
-                for (int _i518 = 0; _i518 < _map515.size; ++_i518)
+                org.apache.thrift.protocol.TMap _map535 = iprot.readMapBegin();
+                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map535.size);
+                List<Long> _key536;
+                long _val537;
+                for (int _i538 = 0; _i538 < _map535.size; ++_i538)
                 {
                   {
-                    org.apache.thrift.protocol.TList _list519 = iprot.readListBegin();
-                    _key516 = new ArrayList<Long>(_list519.size);
-                    long _elem520;
-                    for (int _i521 = 0; _i521 < _list519.size; ++_i521)
+                    org.apache.thrift.protocol.TList _list539 = iprot.readListBegin();
+                    _key536 = new ArrayList<Long>(_list539.size);
+                    long _elem540;
+                    for (int _i541 = 0; _i541 < _list539.size; ++_i541)
                     {
-                      _elem520 = iprot.readI64();
-                      _key516.add(_elem520);
+                      _elem540 = iprot.readI64();
+                      _key536.add(_elem540);
                     }
                     iprot.readListEnd();
                   }
-                  _val517 = iprot.readI64();
-                  struct.executor_start_time_secs.put(_key516, _val517);
+                  _val537 = iprot.readI64();
+                  struct.executor_start_time_secs.put(_key536, _val537);
                 }
                 iprot.readMapEnd();
               }
@@ -779,10 +779,10 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.node_host.size()));
-            for (Map.Entry<String, String> _iter522 : struct.node_host.entrySet())
+            for (Map.Entry<String, String> _iter542 : struct.node_host.entrySet())
             {
-              oprot.writeString(_iter522.getKey());
-              oprot.writeString(_iter522.getValue());
+              oprot.writeString(_iter542.getKey());
+              oprot.writeString(_iter542.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -794,17 +794,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, struct.executor_node_port.size()));
-            for (Map.Entry<List<Long>, NodeInfo> _iter523 : struct.executor_node_port.entrySet())
+            for (Map.Entry<List<Long>, NodeInfo> _iter543 : struct.executor_node_port.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter523.getKey().size()));
-                for (long _iter524 : _iter523.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter543.getKey().size()));
+                for (long _iter544 : _iter543.getKey())
                 {
-                  oprot.writeI64(_iter524);
+                  oprot.writeI64(_iter544);
                 }
                 oprot.writeListEnd();
               }
-              _iter523.getValue().write(oprot);
+              _iter543.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -816,17 +816,17 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
           oprot.writeFieldBegin(EXECUTOR_START_TIME_SECS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, struct.executor_start_time_secs.size()));
-            for (Map.Entry<List<Long>, Long> _iter525 : struct.executor_start_time_secs.entrySet())
+            for (Map.Entry<List<Long>, Long> _iter545 : struct.executor_start_time_secs.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter525.getKey().size()));
-                for (long _iter526 : _iter525.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter545.getKey().size()));
+                for (long _iter546 : _iter545.getKey())
                 {
-                  oprot.writeI64(_iter526);
+                  oprot.writeI64(_iter546);
                 }
                 oprot.writeListEnd();
               }
-              oprot.writeI64(_iter525.getValue());
+              oprot.writeI64(_iter545.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -865,42 +865,42 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       if (struct.is_set_node_host()) {
         {
           oprot.writeI32(struct.node_host.size());
-          for (Map.Entry<String, String> _iter527 : struct.node_host.entrySet())
+          for (Map.Entry<String, String> _iter547 : struct.node_host.entrySet())
           {
-            oprot.writeString(_iter527.getKey());
-            oprot.writeString(_iter527.getValue());
+            oprot.writeString(_iter547.getKey());
+            oprot.writeString(_iter547.getValue());
           }
         }
       }
       if (struct.is_set_executor_node_port()) {
         {
           oprot.writeI32(struct.executor_node_port.size());
-          for (Map.Entry<List<Long>, NodeInfo> _iter528 : struct.executor_node_port.entrySet())
+          for (Map.Entry<List<Long>, NodeInfo> _iter548 : struct.executor_node_port.entrySet())
           {
             {
-              oprot.writeI32(_iter528.getKey().size());
-              for (long _iter529 : _iter528.getKey())
+              oprot.writeI32(_iter548.getKey().size());
+              for (long _iter549 : _iter548.getKey())
               {
-                oprot.writeI64(_iter529);
+                oprot.writeI64(_iter549);
               }
             }
-            _iter528.getValue().write(oprot);
+            _iter548.getValue().write(oprot);
           }
         }
       }
       if (struct.is_set_executor_start_time_secs()) {
         {
           oprot.writeI32(struct.executor_start_time_secs.size());
-          for (Map.Entry<List<Long>, Long> _iter530 : struct.executor_start_time_secs.entrySet())
+          for (Map.Entry<List<Long>, Long> _iter550 : struct.executor_start_time_secs.entrySet())
           {
             {
-              oprot.writeI32(_iter530.getKey().size());
-              for (long _iter531 : _iter530.getKey())
+              oprot.writeI32(_iter550.getKey().size());
+              for (long _iter551 : _iter550.getKey())
               {
-                oprot.writeI64(_iter531);
+                oprot.writeI64(_iter551);
               }
             }
-            oprot.writeI64(_iter530.getValue());
+            oprot.writeI64(_iter550.getValue());
           }
         }
       }
@@ -914,64 +914,64 @@ public class Assignment implements org.apache.thrift.TBase<Assignment, Assignmen
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map532 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.node_host = new HashMap<String,String>(2*_map532.size);
-          String _key533;
-          String _val534;
-          for (int _i535 = 0; _i535 < _map532.size; ++_i535)
+          org.apache.thrift.protocol.TMap _map552 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.node_host = new HashMap<String,String>(2*_map552.size);
+          String _key553;
+          String _val554;
+          for (int _i555 = 0; _i555 < _map552.size; ++_i555)
           {
-            _key533 = iprot.readString();
-            _val534 = iprot.readString();
-            struct.node_host.put(_key533, _val534);
+            _key553 = iprot.readString();
+            _val554 = iprot.readString();
+            struct.node_host.put(_key553, _val554);
           }
         }
         struct.set_node_host_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map536 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map536.size);
-          List<Long> _key537;
-          NodeInfo _val538;
-          for (int _i539 = 0; _i539 < _map536.size; ++_i539)
+          org.apache.thrift.protocol.TMap _map556 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map556.size);
+          List<Long> _key557;
+          NodeInfo _val558;
+          for (int _i559 = 0; _i559 < _map556.size; ++_i559)
           {
             {
-              org.apache.thrift.protocol.TList _list540 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key537 = new ArrayList<Long>(_list540.size);
-              long _elem541;
-              for (int _i542 = 0; _i542 < _list540.size; ++_i542)
+              org.apache.thrift.protocol.TList _list560 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key557 = new ArrayList<Long>(_list560.size);
+              long _elem561;
+              for (int _i562 = 0; _i562 < _list560.size; ++_i562)
               {
-                _elem541 = iprot.readI64();
-                _key537.add(_elem541);
+                _elem561 = iprot.readI64();
+                _key557.add(_elem561);
               }
             }
-            _val538 = new NodeInfo();
-            _val538.read(iprot);
-            struct.executor_node_port.put(_key537, _val538);
+            _val558 = new NodeInfo();
+            _val558.read(iprot);
+            struct.executor_node_port.put(_key557, _val558);
           }
         }
         struct.set_executor_node_port_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map543 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map543.size);
-          List<Long> _key544;
-          long _val545;
-          for (int _i546 = 0; _i546 < _map543.size; ++_i546)
+          org.apache.thrift.protocol.TMap _map563 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map563.size);
+          List<Long> _key564;
+          long _val565;
+          for (int _i566 = 0; _i566 < _map563.size; ++_i566)
           {
             {
-              org.apache.thrift.protocol.TList _list547 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key544 = new ArrayList<Long>(_list547.size);
-              long _elem548;
-              for (int _i549 = 0; _i549 < _list547.size; ++_i549)
+              org.apache.thrift.protocol.TList _list567 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key564 = new ArrayList<Long>(_list567.size);
+              long _elem568;
+              for (int _i569 = 0; _i569 < _list567.size; ++_i569)
               {
-                _elem548 = iprot.readI64();
-                _key544.add(_elem548);
+                _elem568 = iprot.readI64();
+                _key564.add(_elem568);
               }
             }
-            _val545 = iprot.readI64();
-            struct.executor_start_time_secs.put(_key544, _val545);
+            _val565 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key564, _val565);
           }
         }
         struct.set_executor_start_time_secs_isSet(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
index 3234d2b..0ddcaba 100644
--- a/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-5-22")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-10-6")
 public class BoltAggregateStats implements org.apache.thrift.TBase<BoltAggregateStats, BoltAggregateStats._Fields>, java.io.Serializable, Cloneable, Comparable<BoltAggregateStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltAggregateStats");
 

http://git-wip-us.apache.org/repos/asf/storm/blob/121d022b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/generated/BoltStats.java b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
index 26ef5d8..315f955 100644
--- a/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
@@ -51,7 +51,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-30")
 public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._Fields>, java.io.Serializable, Cloneable, Comparable<BoltStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltStats");
 
@@ -881,28 +881,28 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
           case 1: // ACKED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map98 = iprot.readMapBegin();
-                struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map98.size);
-                String _key99;
-                Map<GlobalStreamId,Long> _val100;
-                for (int _i101 = 0; _i101 < _map98.size; ++_i101)
+                org.apache.thrift.protocol.TMap _map108 = iprot.readMapBegin();
+                struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map108.size);
+                String _key109;
+                Map<GlobalStreamId,Long> _val110;
+                for (int _i111 = 0; _i111 < _map108.size; ++_i111)
                 {
-                  _key99 = iprot.readString();
+                  _key109 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map102 = iprot.readMapBegin();
-                    _val100 = new HashMap<GlobalStreamId,Long>(2*_map102.size);
-                    GlobalStreamId _key103;
-                    long _val104;
-                    for (int _i105 = 0; _i105 < _map102.size; ++_i105)
+                    org.apache.thrift.protocol.TMap _map112 = iprot.readMapBegin();
+                    _val110 = new HashMap<GlobalStreamId,Long>(2*_map112.size);
+                    GlobalStreamId _key113;
+                    long _val114;
+                    for (int _i115 = 0; _i115 < _map112.size; ++_i115)
                     {
-                      _key103 = new GlobalStreamId();
-                      _key103.read(iprot);
-                      _val104 = iprot.readI64();
-                      _val100.put(_key103, _val104);
+                      _key113 = new GlobalStreamId();
+                      _key113.read(iprot);
+                      _val114 = iprot.readI64();
+                      _val110.put(_key113, _val114);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.acked.put(_key99, _val100);
+                  struct.acked.put(_key109, _val110);
                 }
                 iprot.readMapEnd();
               }
@@ -914,28 +914,28 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
           case 2: // FAILED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map106 = iprot.readMapBegin();
-                struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map106.size);
-                String _key107;
-                Map<GlobalStreamId,Long> _val108;
-                for (int _i109 = 0; _i109 < _map106.size; ++_i109)
+                org.apache.thrift.protocol.TMap _map116 = iprot.readMapBegin();
+                struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map116.size);
+                String _key117;
+                Map<GlobalStreamId,Long> _val118;
+                for (int _i119 = 0; _i119 < _map116.size; ++_i119)
                 {
-                  _key107 = iprot.readString();
+                  _key117 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map110 = iprot.readMapBegin();
-                    _val108 = new HashMap<GlobalStreamId,Long>(2*_map110.size);
-                    GlobalStreamId _key111;
-                    long _val112;
-                    for (int _i113 = 0; _i113 < _map110.size; ++_i113)
+                    org.apache.thrift.protocol.TMap _map120 = iprot.readMapBegin();
+                    _val118 = new HashMap<GlobalStreamId,Long>(2*_map120.size);
+                    GlobalStreamId _key121;
+                    long _val122;
+                    for (int _i123 = 0; _i123 < _map120.size; ++_i123)
                     {
-                      _key111 = new GlobalStreamId();
-                      _key111.read(iprot);
-                      _val112 = iprot.readI64();
-                      _val108.put(_key111, _val112);
+                      _key121 = new GlobalStreamId();
+                      _key121.read(iprot);
+                      _val122 = iprot.readI64();
+                      _val118.put(_key121, _val122);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.failed.put(_key107, _val108);
+                  struct.failed.put(_key117, _val118);
                 }
                 iprot.readMapEnd();
               }
@@ -947,28 +947,28 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
           case 3: // PROCESS_MS_AVG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map114 = iprot.readMapBegin();
-                struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map114.size);
-                String _key115;
-                Map<GlobalStreamId,Double> _val116;
-                for (int _i117 = 0; _i117 < _map114.size; ++_i117)
+                org.apache.thrift.protocol.TMap _map124 = iprot.readMapBegin();
+                struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map124.size);
+                String _key125;
+                Map<GlobalStreamId,Double> _val126;
+                for (int _i127 = 0; _i127 < _map124.size; ++_i127)
                 {
-                  _key115 = iprot.readString();
+                  _key125 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map118 = iprot.readMapBegin();
-                    _val116 = new HashMap<GlobalStreamId,Double>(2*_map118.size);
-                    GlobalStreamId _key119;
-                    double _val120;
-                    for (int _i121 = 0; _i121 < _map118.size; ++_i121)
+                    org.apache.thrift.protocol.TMap _map128 = iprot.readMapBegin();
+                    _val126 = new HashMap<GlobalStreamId,Double>(2*_map128.size);
+                    GlobalStreamId _key129;
+                    double _val130;
+                    for (int _i131 = 0; _i131 < _map128.size; ++_i131)
                     {
-                      _key119 = new GlobalStreamId();
-                      _key119.read(iprot);
-                      _val120 = iprot.readDouble();
-                      _val116.put(_key119, _val120);
+                      _key129 = new GlobalStreamId();
+                      _key129.read(iprot);
+                      _val130 = iprot.readDouble();
+                      _val126.put(_key129, _val130);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.process_ms_avg.put(_key115, _val116);
+                  struct.process_ms_avg.put(_key125, _val126);
                 }
                 iprot.readMapEnd();
               }
@@ -980,28 +980,28 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
           case 4: // EXECUTED
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map122 = iprot.readMapBegin();
-                struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map122.size);
-                String _key123;
-                Map<GlobalStreamId,Long> _val124;
-                for (int _i125 = 0; _i125 < _map122.size; ++_i125)
+                org.apache.thrift.protocol.TMap _map132 = iprot.readMapBegin();
+                struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map132.size);
+                String _key133;
+                Map<GlobalStreamId,Long> _val134;
+                for (int _i135 = 0; _i135 < _map132.size; ++_i135)
                 {
-                  _key123 = iprot.readString();
+                  _key133 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map126 = iprot.readMapBegin();
-                    _val124 = new HashMap<GlobalStreamId,Long>(2*_map126.size);
-                    GlobalStreamId _key127;
-                    long _val128;
-                    for (int _i129 = 0; _i129 < _map126.size; ++_i129)
+                    org.apache.thrift.protocol.TMap _map136 = iprot.readMapBegin();
+                    _val134 = new HashMap<GlobalStreamId,Long>(2*_map136.size);
+                    GlobalStreamId _key137;
+                    long _val138;
+                    for (int _i139 = 0; _i139 < _map136.size; ++_i139)
                     {
-                      _key127 = new GlobalStreamId();
-                      _key127.read(iprot);
-                      _val128 = iprot.readI64();
-                      _val124.put(_key127, _val128);
+                      _key137 = new GlobalStreamId();
+                      _key137.read(iprot);
+                      _val138 = iprot.readI64();
+                      _val134.put(_key137, _val138);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.executed.put(_key123, _val124);
+                  struct.executed.put(_key133, _val134);
                 }
                 iprot.readMapEnd();
               }
@@ -1013,28 +1013,28 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
           case 5: // EXECUTE_MS_AVG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map130 = iprot.readMapBegin();
-                struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map130.size);
-                String _key131;
-                Map<GlobalStreamId,Double> _val132;
-                for (int _i133 = 0; _i133 < _map130.size; ++_i133)
+                org.apache.thrift.protocol.TMap _map140 = iprot.readMapBegin();
+                struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map140.size);
+                String _key141;
+                Map<GlobalStreamId,Double> _val142;
+                for (int _i143 = 0; _i143 < _map140.size; ++_i143)
                 {
-                  _key131 = iprot.readString();
+                  _key141 = iprot.readString();
                   {
-                    org.apache.thrift.protocol.TMap _map134 = iprot.readMapBegin();
-                    _val132 = new HashMap<GlobalStreamId,Double>(2*_map134.size);
-                    GlobalStreamId _key135;
-                    double _val136;
-                    for (int _i137 = 0; _i137 < _map134.size; ++_i137)
+                    org.apache.thrift.protocol.TMap _map144 = iprot.readMapBegin();
+                    _val142 = new HashMap<GlobalStreamId,Double>(2*_map144.size);
+                    GlobalStreamId _key145;
+                    double _val146;
+                    for (int _i147 = 0; _i147 < _map144.size; ++_i147)
                     {
-                      _key135 = new GlobalStreamId();
-                      _key135.read(iprot);
-                      _val136 = iprot.readDouble();
-                      _val132.put(_key135, _val136);
+                      _key145 = new GlobalStreamId();
+                      _key145.read(iprot);
+                      _val146 = iprot.readDouble();
+                      _val142.put(_key145, _val146);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.execute_ms_avg.put(_key131, _val132);
+                  struct.execute_ms_avg.put(_key141, _val142);
                 }
                 iprot.readMapEnd();
               }
@@ -1060,15 +1060,15 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
         oprot.writeFieldBegin(ACKED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter138 : struct.acked.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter148 : struct.acked.entrySet())
           {
-            oprot.writeString(_iter138.getKey());
+            oprot.writeString(_iter148.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter138.getValue().size()));
-              for (Map.Entry<GlobalStreamId, Long> _iter139 : _iter138.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter148.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter149 : _iter148.getValue().entrySet())
               {
-                _iter139.getKey().write(oprot);
-                oprot.writeI64(_iter139.getValue());
+                _iter149.getKey().write(oprot);
+                oprot.writeI64(_iter149.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1081,15 +1081,15 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
         oprot.writeFieldBegin(FAILED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter140 : struct.failed.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter150 : struct.failed.entrySet())
           {
-            oprot.writeString(_iter140.getKey());
+            oprot.writeString(_iter150.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter140.getValue().size()));
-              for (Map.Entry<GlobalStreamId, Long> _iter141 : _iter140.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter150.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter151 : _iter150.getValue().entrySet())
               {
-                _iter141.getKey().write(oprot);
-                oprot.writeI64(_iter141.getValue());
+                _iter151.getKey().write(oprot);
+                oprot.writeI64(_iter151.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1102,15 +1102,15 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
         oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.process_ms_avg.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter142 : struct.process_ms_avg.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter152 : struct.process_ms_avg.entrySet())
           {
-            oprot.writeString(_iter142.getKey());
+            oprot.writeString(_iter152.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter142.getValue().size()));
-              for (Map.Entry<GlobalStreamId, Double> _iter143 : _iter142.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter152.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Double> _iter153 : _iter152.getValue().entrySet())
               {
-                _iter143.getKey().write(oprot);
-                oprot.writeDouble(_iter143.getValue());
+                _iter153.getKey().write(oprot);
+                oprot.writeDouble(_iter153.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1123,15 +1123,15 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
         oprot.writeFieldBegin(EXECUTED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.executed.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter144 : struct.executed.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter154 : struct.executed.entrySet())
           {
-            oprot.writeString(_iter144.getKey());
+            oprot.writeString(_iter154.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter144.getValue().size()));
-              for (Map.Entry<GlobalStreamId, Long> _iter145 : _iter144.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, _iter154.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter155 : _iter154.getValue().entrySet())
               {
-                _iter145.getKey().write(oprot);
-                oprot.writeI64(_iter145.getValue());
+                _iter155.getKey().write(oprot);
+                oprot.writeI64(_iter155.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1144,15 +1144,15 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
         oprot.writeFieldBegin(EXECUTE_MS_AVG_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.execute_ms_avg.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter146 : struct.execute_ms_avg.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter156 : struct.execute_ms_avg.entrySet())
           {
-            oprot.writeString(_iter146.getKey());
+            oprot.writeString(_iter156.getKey());
             {
-              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter146.getValue().size()));
-              for (Map.Entry<GlobalStreamId, Double> _iter147 : _iter146.getValue().entrySet())
+              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, _iter156.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Double> _iter157 : _iter156.getValue().entrySet())
               {
-                _iter147.getKey().write(oprot);
-                oprot.writeDouble(_iter147.getValue());
+                _iter157.getKey().write(oprot);
+                oprot.writeDouble(_iter157.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1180,75 +1180,75 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.acked.size());
-        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter148 : struct.acked.entrySet())
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter158 : struct.acked.entrySet())
         {
-          oprot.writeString(_iter148.getKey());
+          oprot.writeString(_iter158.getKey());
           {
-            oprot.writeI32(_iter148.getValue().size());
-            for (Map.Entry<GlobalStreamId, Long> _iter149 : _iter148.getValue().entrySet())
+            oprot.writeI32(_iter158.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter159 : _iter158.getValue().entrySet())
             {
-              _iter149.getKey().write(oprot);
-              oprot.writeI64(_iter149.getValue());
+              _iter159.getKey().write(oprot);
+              oprot.writeI64(_iter159.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.failed.size());
-        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter150 : struct.failed.entrySet())
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter160 : struct.failed.entrySet())
         {
-          oprot.writeString(_iter150.getKey());
+          oprot.writeString(_iter160.getKey());
           {
-            oprot.writeI32(_iter150.getValue().size());
-            for (Map.Entry<GlobalStreamId, Long> _iter151 : _iter150.getValue().entrySet())
+            oprot.writeI32(_iter160.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter161 : _iter160.getValue().entrySet())
             {
-              _iter151.getKey().write(oprot);
-              oprot.writeI64(_iter151.getValue());
+              _iter161.getKey().write(oprot);
+              oprot.writeI64(_iter161.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.process_ms_avg.size());
-        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter152 : struct.process_ms_avg.entrySet())
+        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter162 : struct.process_ms_avg.entrySet())
         {
-          oprot.writeString(_iter152.getKey());
+          oprot.writeString(_iter162.getKey());
           {
-            oprot.writeI32(_iter152.getValue().size());
-            for (Map.Entry<GlobalStreamId, Double> _iter153 : _iter152.getValue().entrySet())
+            oprot.writeI32(_iter162.getValue().size());
+            for (Map.Entry<GlobalStreamId, Double> _iter163 : _iter162.getValue().entrySet())
             {
-              _iter153.getKey().write(oprot);
-              oprot.writeDouble(_iter153.getValue());
+              _iter163.getKey().write(oprot);
+              oprot.writeDouble(_iter163.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.executed.size());
-        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter154 : struct.executed.entrySet())
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter164 : struct.executed.entrySet())
         {
-          oprot.writeString(_iter154.getKey());
+          oprot.writeString(_iter164.getKey());
           {
-            oprot.writeI32(_iter154.getValue().size());
-            for (Map.Entry<GlobalStreamId, Long> _iter155 : _iter154.getValue().entrySet())
+            oprot.writeI32(_iter164.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter165 : _iter164.getValue().entrySet())
             {
-              _iter155.getKey().write(oprot);
-              oprot.writeI64(_iter155.getValue());
+              _iter165.getKey().write(oprot);
+              oprot.writeI64(_iter165.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.execute_ms_avg.size());
-        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter156 : struct.execute_ms_avg.entrySet())
+        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter166 : struct.execute_ms_avg.entrySet())
         {
-          oprot.writeString(_iter156.getKey());
+          oprot.writeString(_iter166.getKey());
           {
-            oprot.writeI32(_iter156.getValue().size());
-            for (Map.Entry<GlobalStreamId, Double> _iter157 : _iter156.getValue().entrySet())
+            oprot.writeI32(_iter166.getValue().size());
+            for (Map.Entry<GlobalStreamId, Double> _iter167 : _iter166.getValue().entrySet())
             {
-              _iter157.getKey().write(oprot);
-              oprot.writeDouble(_iter157.getValue());
+              _iter167.getKey().write(oprot);
+              oprot.writeDouble(_iter167.getValue());
             }
           }
         }
@@ -1259,127 +1259,127 @@ public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._
     public void read(org.apache.thrift.protocol.TProtocol prot, BoltStats struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map158.size);
-        String _key159;
-        Map<GlobalStreamId,Long> _val160;
-        for (int _i161 = 0; _i161 < _map158.size; ++_i161)
+        org.apache.thrift.protocol.TMap _map168 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map168.size);
+        String _key169;
+        Map<GlobalStreamId,Long> _val170;
+        for (int _i171 = 0; _i171 < _map168.size; ++_i171)
         {
-          _key159 = iprot.readString();
+          _key169 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map162 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-            _val160 = new HashMap<GlobalStreamId,Long>(2*_map162.size);
-            GlobalStreamId _key163;
-            long _val164;
-            for (int _i165 = 0; _i165 < _map162.size; ++_i165)
+            org.apache.thrift.protocol.TMap _map172 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val170 = new HashMap<GlobalStreamId,Long>(2*_map172.size);
+            GlobalStreamId _key173;
+            long _val174;
+            for (int _i175 = 0; _i175 < _map172.size; ++_i175)
             {
-              _key163 = new GlobalStreamId();
-              _key163.read(iprot);
-              _val164 = iprot.readI64();
-              _val160.put(_key163, _val164);
+              _key173 = new GlobalStreamId();
+              _key173.read(iprot);
+              _val174 = iprot.readI64();
+              _val170.put(_key173, _val174);
             }
           }
-          struct.acked.put(_key159, _val160);
+          struct.acked.put(_key169, _val170);
         }
       }
       struct.set_acked_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map166 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map166.size);
-        String _key167;
-        Map<GlobalStreamId,Long> _val168;
-        for (int _i169 = 0; _i169 < _map166.size; ++_i169)
+        org.apache.thrift.protocol.TMap _map176 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map176.size);
+        String _key177;
+        Map<GlobalStreamId,Long> _val178;
+        for (int _i179 = 0; _i179 < _map176.size; ++_i179)
         {
-          _key167 = iprot.readString();
+          _key177 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map170 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-            _val168 = new HashMap<GlobalStreamId,Long>(2*_map170.size);
-            GlobalStreamId _key171;
-            long _val172;
-            for (int _i173 = 0; _i173 < _map170.size; ++_i173)
+            org.apache.thrift.protocol.TMap _map180 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val178 = new HashMap<GlobalStreamId,Long>(2*_map180.size);
+            GlobalStreamId _key181;
+            long _val182;
+            for (int _i183 = 0; _i183 < _map180.size; ++_i183)
             {
-              _key171 = new GlobalStreamId();
-              _key171.read(iprot);
-              _val172 = iprot.readI64();
-              _val168.put(_key171, _val172);
+              _key181 = new GlobalStreamId();
+              _key181.read(iprot);
+              _val182 = iprot.readI64();
+              _val178.put(_key181, _val182);
             }
           }
-          struct.failed.put(_key167, _val168);
+          struct.failed.put(_key177, _val178);
         }
       }
       struct.set_failed_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map174 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map174.size);
-        String _key175;
-        Map<GlobalStreamId,Double> _val176;
-        for (int _i177 = 0; _i177 < _map174.size; ++_i177)
+        org.apache.thrift.protocol.TMap _map184 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map184.size);
+        String _key185;
+        Map<GlobalStreamId,Double> _val186;
+        for (int _i187 = 0; _i187 < _map184.size; ++_i187)
         {
-          _key175 = iprot.readString();
+          _key185 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map178 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-            _val176 = new HashMap<GlobalStreamId,Double>(2*_map178.size);
-            GlobalStreamId _key179;
-            double _val180;
-            for (int _i181 = 0; _i181 < _map178.size; ++_i181)
+            org.apache.thrift.protocol.TMap _map188 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val186 = new HashMap<GlobalStreamId,Double>(2*_map188.size);
+            GlobalStreamId _key189;
+            double _val190;
+            for (int _i191 = 0; _i191 < _map188.size; ++_i191)
             {
-              _key179 = new GlobalStreamId();
-              _key179.read(iprot);
-              _val180 = iprot.readDouble();
-              _val176.put(_key179, _val180);
+              _key189 = new GlobalStreamId();
+              _key189.read(iprot);
+              _val190 = iprot.readDouble();
+              _val186.put(_key189, _val190);
             }
           }
-          struct.process_ms_avg.put(_key175, _val176);
+          struct.process_ms_avg.put(_key185, _val186);
         }
       }
       struct.set_process_ms_avg_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map182 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map182.size);
-        String _key183;
-        Map<GlobalStreamId,Long> _val184;
-        for (int _i185 = 0; _i185 < _map182.size; ++_i185)
+        org.apache.thrift.protocol.TMap _map192 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map192.size);
+        String _key193;
+        Map<GlobalStreamId,Long> _val194;
+        for (int _i195 = 0; _i195 < _map192.size; ++_i195)
         {
-          _key183 = iprot.readString();
+          _key193 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map186 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-            _val184 = new HashMap<GlobalStreamId,Long>(2*_map186.size);
-            GlobalStreamId _key187;
-            long _val188;
-            for (int _i189 = 0; _i189 < _map186.size; ++_i189)
+            org.apache.thrift.protocol.TMap _map196 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val194 = new HashMap<GlobalStreamId,Long>(2*_map196.size);
+            GlobalStreamId _key197;
+            long _val198;
+            for (int _i199 = 0; _i199 < _map196.size; ++_i199)
             {
-              _key187 = new GlobalStreamId();
-              _key187.read(iprot);
-              _val188 = iprot.readI64();
-              _val184.put(_key187, _val188);
+              _key197 = new GlobalStreamId();
+              _key197.read(iprot);
+              _val198 = iprot.readI64();
+              _val194.put(_key197, _val198);
             }
           }
-          struct.executed.put(_key183, _val184);
+          struct.executed.put(_key193, _val194);
         }
       }
       struct.set_executed_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map190 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
-        struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map190.size);
-        String _key191;
-        Map<GlobalStreamId,Double> _val192;
-        for (int _i193 = 0; _i193 < _map190.size; ++_i193)
+        org.apache.thrift.protocol.TMap _map200 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32());
+        struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map200.size);
+        String _key201;
+        Map<GlobalStreamId,Double> _val202;
+        for (int _i203 = 0; _i203 < _map200.size; ++_i203)
         {
-          _key191 = iprot.readString();
+          _key201 = iprot.readString();
           {
-            org.apache.thrift.protocol.TMap _map194 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
-            _val192 = new HashMap<GlobalStreamId,Double>(2*_map194.size);
-            GlobalStreamId _key195;
-            double _val196;
-            for (int _i197 = 0; _i197 < _map194.size; ++_i197)
+            org.apache.thrift.protocol.TMap _map204 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val202 = new HashMap<GlobalStreamId,Double>(2*_map204.size);
+            GlobalStreamId _key205;
+            double _val206;
+            for (int _i207 = 0; _i207 < _map204.size; ++_i207)
             {
-              _key195 = new GlobalStreamId();
-              _key195.read(iprot);
-              _val196 = iprot.readDouble();
-              _val192.put(_key195, _val196);
+              _key205 = new GlobalStreamId();
+              _key205.read(iprot);
+              _val206 = iprot.readDouble();
+              _val202.put(_key205, _val206);
             }
           }
-          struct.execute_ms_avg.put(_key191, _val192);
+          struct.execute_ms_avg.put(_key201, _val202);
         }
       }
       struct.set_execute_ms_avg_isSet(true);