Merge branch 'master' into STORM-1040
diff --git a/.gitignore b/.gitignore
index 4e08a2e..4b4c52f 100644
--- a/.gitignore
+++ b/.gitignore
@@ -35,3 +35,6 @@
 dependency-reduced-pom.xml
 derby.log
 metastore_db
+.settings/
+**/.project
+**/.classpath
diff --git a/.travis.yml b/.travis.yml
index 6d864d5..05e24fe 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -9,17 +9,24 @@
 #  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.
-language: java
 
+env:
+  - MODULES=storm-core
+  - MODULES='!storm-core'
+
+language: java
 jdk:
   - oraclejdk7
   - oraclejdk8
-
 before_install:
   - rvm use 2.1.5 --install
   - nvm install 0.12.2
   - nvm use 0.12.2
-
 install: /bin/bash ./dev-tools/travis/travis-install.sh `pwd`
-
-script: /bin/bash ./dev-tools/travis/travis-script.sh `pwd`
+script: /bin/bash ./dev-tools/travis/travis-script.sh `pwd` $MODULES
+sudo: false
+cache:
+  directories:
+    - "$HOME/.m2/repository"
+    - "$HOME/.rvm"
+    - "$NVM_DIR"
diff --git a/CHANGELOG.md b/CHANGELOG.md
index c5d052b..f513a7f 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,6 +1,101 @@
 ## 0.11.0
  * STORM-1060: Serialize Calcite plans into JSON format.
  * STORM-1062: Establish the basic structure of the code generator.
+ * STORM-1341: Let topology have own heartbeat timeout for multilang subprocess
+ * STORM-1207: Added flux support for IWindowedBolt
+ * STORM-1352: Trident should support writing to multiple Kafka clusters.
+ * STORM-1220: Avoid double copying in the Kafka spout.
+ * STORM-1340: Use Travis-CI build matrix to improve test execution times
+ * STORM-1126: Allow a configMethod that takes no arguments (Flux)
+ * STORM-1203: worker metadata file creation doesn't use storm.log.dir config
+ * STORM-1349: [Flux] Allow constructorArgs to take Maps as arguments
+ * STORM-126: Add Lifecycle support API for worker nodes
+ * STORM-1213: Remove sigar binaries from source tree
+ * STORM-885:  Heartbeat Server (Pacemaker)
+ * STORM-1221: Create a common interface for all Trident spout.
+ * STORM-1198: Web UI to show resource usages and Total Resources on all supervisors
+ * STORM-1167: Add windowing support for storm core.
+ * STORM-1215: Use Async Loggers to avoid locking  and logging overhead
+ * STORM-1204: Logviewer should graceful report page-not-found instead of 500 for bad topo-id etc
+ * STORM-831: Add BugTracker and Central Logging URL to UI
+ * STORM-1208: UI: NPE seen when aggregating bolt streams stats
+ * STORM-1016: Generate trident bolt ids with sorted group names
+ * STORM-1190: System Load too high after recent changes
+ * STORM-1098: Nimbus hook for topology actions.
+ * STORM-1145: Have IConnection push tuples instead of pull them
+ * STORM-1191: bump timeout by 50% due to intermittent travis build failures
+ * STORM-794: Modify Spout async loop to treat activate/deactivate ASAP
+ * STORM-1196: Upgrade to thrift 0.9.3
+ * STORM-1155: Supervisor recurring health checks
+ * STORM-1189: Maintain wire compatability with 0.10.x versions of storm.
+ * STORM-1185: replace nimbus.host with nimbus.seeds
+ * STORM-1164: Code cleanup for typos, warnings and conciseness.
+ * STORM-902: Simple Log Search.
+ * STORM-1052: TridentKafkaState uses new Kafka Producer API.
+ * STORM-1182: Removing and wrapping some exceptions in ConfigValidation to make code cleaner
+ * STORM-1134. Windows: Fix log4j config.
+ * STORM-1127: allow for boolean arguments (Flux)
+ * STORM-1180: FLUX logo wasn't appearing quite right
+ * STORM-1138: Storm-hdfs README should be updated with Avro Bolt information
+ * STORM-1154: SequenceFileBolt needs unit tests
+ * STORM-162: Load Aware Shuffle Grouping
+ * STORM-1158: Storm metrics to profile various storm functions
+ * STORM-1161: Add License headers and add rat checks to builds
+ * STORM-1165: normalize the scales of CPU/Mem/Net when choosing the best node for Resource Aware Scheduler
+ * STORM-1163: use rmr rather than rmpath for remove worker-root
+ * STORM-1170: Fix the producer alive issue in DisruptorQueueTest
+ * STORM-1168: removes noisy log message & a TODO
+ * STORM-1143: Validate topology Configs during topology submission
+ * STORM-1157: Adding dynamic profiling for worker, restarting worker, jstack, heap dump, and profiling
+ * STORM-1123: TupleImpl - Unnecessary variable initialization.
+ * STORM-1153: Use static final instead of just static for class members.
+ * STORM-817: Kafka Wildcard Topic Support.
+ * STORM-40: Turn worker garbage collection and heapdump on by default.
+ * STORM-1152: Change map keySet iteration to entrySet iteration for efficiency.
+ * STORM-1147: Storm JDBCBolt should add validation to ensure either insertQuery or table name is specified and not both.
+ * STORM-1151: Batching in DisruptorQueue
+ * STORM-350: Update disruptor to latest version (3.3.2)
+ * STORM-697: Support for Emitting Kafka Message Offset and Partition
+ * STORM-1074: Add Avro HDFS bolt
+ * STORM-566: Improve documentation including incorrect Kryo ser. framework docs
+ * STORM-1073: Refactor AbstractHdfsBolt
+ * STORM-1128: Make metrics fast
+ * STORM-1122: Fix the format issue in Utils.java
+ * STORM-1111: Fix Validation for lots of different configs
+ * STORM-1125: Adding separate ZK client for read in Nimbus ZK State
+ * STORM-1121: Remove method call to avoid overhead during topology submission time
+ * STORM-1120: Fix keyword (schema -> scheme) from main-routes
+ * STORM-1115: Stale leader-lock key effectively bans all nodes from becoming leaders
+ * STORM-1119: Create access logging for all daemons
+ * STORM-1117: Adds visualization-init route previously missing
+ * STORM-1118: Added test to compare latency vs. throughput in storm.
+ * STORM-1110: Fix Component Page for system components
+ * STORM-1093: Launching Workers with resources specified in resource-aware schedulers
+ * STORM-1102: Add a default flush interval for HiveBolt
+ * STORM-1112: Add executor id to the thread name of the executor thread for debug
+ * STORM-1079: Batch Puts to HBase
+ * STORM-1084: Improve Storm config validation process to use java annotations instead of *_SCHEMA format
+ * STORM-1106: Netty should not limit attempts to reconnect
+ * STORM-1103: Changes log message to DEBUG from INFO
+ * STORM-1104: Nimbus HA fails to find newly downloaded code files
+ * STORM-1087: Avoid issues with transfer-queue backpressure.
+ * 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`
+ * STORM-820: Aggregate topo stats on nimbus, not ui
+ * STORM-412: Allow users to modify logging levels of running topologies
+ * STORM-1078: Updated RateTracker to be thread safe
+ * STORM-1082: fix nits for properties in kafka tests
+ * STORM-993: include uptimeSeconds as JSON integer field
+ * STORM-1053: Update storm-kafka README for new producer API confs.
+ * STORM-1058: create CLI kill_workers to kill workers on a supervisor node
+ * STORM-1063: support relative log4j conf dir for both daemons and workers
+ * STORM-1059: Upgrade Storm to use Clojure 1.7.0
+ * STORM-1069: add check case for external change of 'now' value.
+ * STORM-969: HDFS Bolt can end up in an unrecoverable state.
+ * STORM-1068: Configure request.required.acks to be 1 in KafkaUtilsTest for sync
+ * STORM-1017: If ignoreZkOffsets set true,KafkaSpout will reset zk offset when recover from failure.
  * STORM-1054: Excessive logging ShellBasedGroupsMapping if the user doesn't have any groups.
  * STORM-954: Toplogy Event Inspector
  * STORM-862: Pluggable System Metrics
@@ -59,10 +154,21 @@
  * STORM-975: Storm-Kafka trident topology example
  * STORM-958: Add config for init params of group mapping service
  * STORM-949: On the topology summary UI page, last shown error should have the time and date
+ * STORM-1142: Some config validators for positive ints need to allow 0
+ * STORM-901: Worker Artifacts Directory
+ * STORM-1144: Display requested and assigned cpu/mem resources for schedulers in UI
+ * STORM-1217: making small fixes in RAS
 
 ## 0.10.0-beta2
+ * STORM-1108: Fix NPE in simulated time
+ * STORM-1099: Fix worker childopts as arraylist of strings
+ * STORM-1094: advance kafka offset when deserializer yields no object
+ * STORM-1066: Specify current directory when supervisor launches a worker
+ * STORM-1012: Shaded everything that was not already shaded
+ * STORM-967: Shaded everything that was not already shaded
+ * STORM-922: Shaded everything that was not already shaded
+ * STORM-1042: Shaded everything that was not already shaded
  * STORM-1026: Adding external classpath elements does not work
- * STORM-1056: allow supervisor log filename to be configurable via ENV variable
  * STORM-1055: storm-jdbc README needs fixes and context
  * STORM-1044: Setting dop to zero does not raise an error
  * STORM-1050: Topologies with same name run on one cluster
@@ -70,7 +176,6 @@
  * STORM-803: Better CI logs
  * STORM-1027: Use overflow buffer for emitting metrics
  * STORM-1024: log4j changes leaving ${sys:storm.log.dir} under STORM_HOME dir
- * STORM-996: netty-unit-tests/test-batch demonstrates out-of-order delivery
  * STORM-944: storm-hive pom.xml has a dependency conflict with calcite
  * STORM-994: Connection leak between nimbus and supervisors
  * STORM-1001: Undefined STORM_EXT_CLASSPATH adds '::' to classpath of workers
@@ -88,6 +193,9 @@
  * STORM-742: Let ShellBolt treat all messages to update heartbeat
  * STORM-992: A bug in the timer.clj might cause unexpected delay to schedule new event
 
+## 0.10.0
+ * STORM-1096: Fix some issues with impersonation on the UI
+
 ## 0.10.0-beta1
  * STORM-873: Flux does not handle diamond topologies
 
@@ -254,6 +362,8 @@
  * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist.
 
 ## 0.9.6
+ * STORM-996: netty-unit-tests/test-batch demonstrates out-of-order delivery
+ * STORM-1056: allow supervisor log filename to be configurable via ENV variable
  * STORM-1051: Netty Client.java's flushMessages produces a NullPointerException
  * STORM-763: nimbus reassigned worker A to another machine, but other worker's netty client can't connect to the new worker A
  * STORM-935: Update Disruptor queue version to 2.10.4
@@ -815,3 +925,4 @@
 ## 0.5.0
  
  * Initial release!
+
diff --git a/DEVELOPER.md b/DEVELOPER.md
index 02fe135..18b2ef6 100644
--- a/DEVELOPER.md
+++ b/DEVELOPER.md
@@ -192,6 +192,7 @@
 3. Merge the pull request into your local test branch.
 
         $ git pull <remote_repo_url> <remote_branch>
+    You can use `./dev-tools/storm-merge.py <pull-number>` to produce the above command most of the time.
 
 4.  Assuming that the pull request merges without any conflicts:
     Update the top-level `CHANGELOG.md`, and add in the JIRA ticket number (example: `STORM-1234`) and ticket
@@ -223,6 +224,9 @@
 # Build the code and run the tests
 
 ## Prerequisites
+Firt of all you need to make sure you are using maven 3.2.5 or below.  There is a bug in later versions of maven as linked to from https://issues.apache.org/jira/browse/MSHADE-206 that
+cause shaded dependencies to not be packaged correctly.  Also please be aware that because we are shading dependencies mvn dependency:tree will not always show the dependencies correctly. 
+
 In order to build `storm` you need `python`, `ruby` and `nodejs`. In order to avoid an overful page we don't provide platform/OS specific installation instructions for those here. Please refer to you platform's/OS' documentation for support.
 
 The `ruby` package manager `rvm` and `nodejs` package manager `nvm` are for convenience and are used in the tests which run on [travis](https://travis-ci.org/apache/storm). They can be installed using `curl -L https://get.rvm.io | bash -s stable --autolibs=enabled && source ~/.profile` (see the [rvm installation instructions](https://github.com/rvm/rvm) for details) and `wget -qO- https://raw.githubusercontent.com/creationix/nvm/v0.26.1/install.sh | bash && source ~/.bashrc` (see the [nvm installation instructions](https://github.com/creationix/nvm) for details).
@@ -241,17 +245,9 @@
 
 The following commands must be run from the top-level directory.
 
-    # Build the code and run the tests (requires nodejs, python and ruby installed) 
-    # `mvn clean package` will fail because storm-core requires storm-maven-plugin.
-    # This plugin should be installed before compiling storm-core.
-    $ mvn clean install
+`mvn clean install`
 
-    # Build the code and run the tests, with specifying default test timeout (in millisecond)
-    $ export STORM_TEST_TIMEOUT_MS=10000
-    $ mvn clean install
-
-    # Build the code but skip the tests
-    $ mvn clean install -DskipTests=true
+If you wish to skip the unit tests you can do this by adding `-DskipTests` to the command line. 
 
 In case you modified `storm.thrift`, you have to regenerate thrift code as java and python code before compiling whole project.
 
@@ -266,19 +262,7 @@
 [auth_test.clj](storm-core/test/clj/backtype/storm/security/auth/auth_test.clj), which has the namespace
 `backtype.storm.security.auth.auth-test`.
 
-First, start the REPL from within the relevant sub-project (here: `storm-core`):
-
-    $ cd storm-core/
-    $ mvn clojure:repl
-
-Now we run the tests in `auth_test.clj` in the REPL:
-
-```clojure
-;; You can use both absolute as well as relative paths to the .clj file.
-(load-file "test/clj/backtype/storm/security/auth/auth_test.clj")
-(ns backtype.storm.security.auth.auth-test)
-(run-tests)
-```
+You can also run tests selectively with `-Dtest=<test_name>`.  This works for both clojure and junit tests.
 
 > Tip: IDEs such as IntelliJ IDEA support a built-in Clojure REPL, which you can also use to run tests selectively.
 > Sometimes you may find that tests pass/fail depending on which REPL you use, which -- although frustrating --
@@ -286,7 +270,6 @@
 
 Unfortunately you might experience failures in clojure tests which are wrapped in the `maven-clojure-plugin` and thus doesn't provide too much useful output at first sight - you might end up with a maven test failure with an error message as unhelpful as `Clojure failed.`. In this case it's recommended to look into `target/test-reports` of the failed project to see what actual tests have failed or scroll through the maven output looking for obvious issues like missing binaries.
 
-
 <a name="packaging"></a>
 
 ## Create a Storm distribution (packaging)
@@ -295,7 +278,7 @@
 do not use the Maven release plugin because creating an official release is the task of our release manager.
 
     # First, build the code.
-    $ mvn clean install  # you may skip tests with `-DskipTests=true` to save time
+    $ mvn clean install # you may skip tests with `-DskipTests=true` to save time
 
     # Create the binary distribution.
     $ cd storm-dist/binary && mvn package
@@ -383,6 +366,8 @@
 
 If you do not have a JIRA account yet, then you can create one via the link above (registration is free).
 
+<a name="code-search"></a>
+The storm codebase is available at [Codota](https://www.codota.com/xref/#/github_apache_storm_560da9ada8cb8703008bbfdc/findUsages) where you can semantically search it like in an IDE (e.g. find usages for a method). 
 
 <a name="questions"></a>
 
diff --git a/DISCLAIMER b/DISCLAIMER
deleted file mode 100644
index 8638904..0000000
--- a/DISCLAIMER
+++ /dev/null
@@ -1,10 +0,0 @@
-Apache Storm is an effort undergoing incubation at the Apache Software
-Foundation (ASF), sponsored by the Apache Incubator PMC.
-
-Incubation is required of all newly accepted projects until a further review
-indicates that the infrastructure, communications, and decision making process
-have stabilized in a manner consistent with other successful ASF projects.
-
-While incubation status is not necessarily a reflection of the completeness
-or stability of the code, it does indicate that the project has yet to be
-fully endorsed by the ASF.
diff --git a/LICENSE b/LICENSE
index 8755d1b..a76af64 100644
--- a/LICENSE
+++ b/LICENSE
@@ -258,6 +258,18 @@
 
 -----------------------------------------------------------------------
 
+For typeahead jquery 0.10.5 (storm-core/src/ui/public/js/typeahead.jquery.min.js)
+
+Copyright (c) 2013-2014 Twitter, Inc
+
+    * Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution.
+    * Neither the name of Allan Jardine nor SpryMedia UK may be used to endorse or promote products derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDERS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 
+
+-----------------------------------------------------------------------
+
 Copyright (c) 2008-2010, Allan Jardine
 All rights reserved.
 
@@ -542,3 +554,32 @@
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
 SOFTWARE.
 
+-----------------------------------------------------------------------
+
+For statistic image
+
+storm-core/src/ui/public/images/statistic.png
+
+Copyright (c) 2015 Github, Inc.
+
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.  IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
diff --git a/README.markdown b/README.markdown
index 90a79c5..19ad520 100644
--- a/README.markdown
+++ b/README.markdown
@@ -232,6 +232,17 @@
 * Matthias J. Sax ([@mjsax](https://github.com/mjsax))
 * Erik Weathers ([@erikdw](https://github.com/erikdw))
 * Bryan Shell ([@shellbj](https://github.com/shellbj))
+* Dmytro Dragan ([@3Dragan](https://github.com/3Dragan))
+* Ningyu An ([@jetan9](https://github.com/jetan9))
+* Pete Prokopowicz ([@prokopowicz](https://github.com/prokopowicz))
+* Priyank Shah ([@priyank5485](https://github.com/priyank5485))
+* Joshua Martell ([@jmartell7](https://github.com/jmartell7))
+* Matthew Tieman ([@mjtieman](https://github.com/mjtieman))
+* Chuck Burgess ([@ashnazg](https://github.com/ashnazg))
+* Tom Graves ([@tgravescs](https://github.com/tgravescs))
+* Dror Weiss ([@drorweiss](https://github.com/drorweiss))
+* Victor Wong ([@victor-wong](https://github.com/victor-wong))
+* David Wimsey ([@dwimsey](https://github.com/dwimsey))
 
 ## Acknowledgements
 
diff --git a/bin/flight.bash b/bin/flight.bash
new file mode 100755
index 0000000..957c9ac
--- /dev/null
+++ b/bin/flight.bash
@@ -0,0 +1,154 @@
+#!/bin/bash
+# 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.
+
+JDKPATH="/home/y/share/yjava_jdk/java"
+BINPATH="/usr/bin"
+USER=`whoami`
+
+#SETTINGS=/Library/Java/JavaVirtualMachines/jdk1.8.0_51.jdk/Contents/Home/jre/lib/jfr/profile.jfc
+SETTINGS=profile
+
+platform='unknown'
+unamestr=`uname`
+if [[ "$unamestr" == 'Linux' ]]; then
+    platform='linux'
+elif [[ "$unamestr" == 'Darwin' ]]; then
+    platform='darwin'
+elif [[ "$unamestr" == 'FreeBSD' ]]; then
+    platform='freebsd'
+fi
+
+if [[ $platform == 'linux' ]]; then
+    BINPATH="$JDKPATH/bin"
+elif [[ $platform == 'darwin' ]]; then
+    BINPATH="/usr/bin"
+fi
+
+function start_record {
+    # start_record pid
+    already_recording=false
+    for rid in `get_recording_ids $1`; do
+        already_recording=true
+        break;
+    done
+    if [ "$already_recording" = false ]; then
+        $BINPATH/jcmd $1 JFR.start settings=${SETTINGS}
+    fi
+}
+
+function dump_record {
+    for rid in `get_recording_ids $1`; do
+        FILENAME=recording-$1-${rid}-${NOW}.jfr
+        $BINPATH/jcmd $1 JFR.dump recording=$rid filename="$2/${FILENAME}"
+    done
+}
+
+function jstack_record {
+    FILENAME=jstack-$1-${NOW}.txt
+    $BINPATH/jstack $1 > "$2/${FILENAME}"
+}
+
+function jmap_record {
+    FILENAME=recording-$1-${NOW}.bin
+    $BINPATH/jmap -dump:format=b,file="$2/${FILENAME}" $1
+}
+
+function stop_record {
+    for rid in `get_recording_ids $1`; do
+        FILENAME=recording-$1-${rid}-${NOW}.jfr
+        $BINPATH/jcmd $1 JFR.dump recording=$rid filename="$2/${FILENAME}"
+        $BINPATH/jcmd $1 JFR.stop recording=$rid
+    done
+}
+
+function get_recording_ids {
+    $BINPATH/jcmd $1 JFR.check | perl -n -e '/recording=([0-9]+)/ && print "$1 "'
+}
+
+function usage_and_quit {
+    echo "Usage: $0 pid start [profile_settings]"
+    echo "       $0 pid dump target_dir"
+    echo "       $0 pid stop target_dir"
+    echo "       $0 pid jstack target_dir"
+    echo "       $0 pid jmap target_dir"
+    echo "       $0 pid kill"
+    exit -1
+}
+
+# Before using this script: make sure FlightRecorder is enabled
+
+if [ "$#" -le 1 ]; then
+    echo "Wrong number of arguments.."
+    usage_and_quit
+
+fi
+# call this script with the process pid, example: "./flight PID start" or "./flight PID stop"
+PID="$1"
+CMD="$2"
+
+if /bin/ps -p $PID > /dev/null
+then
+    if [[ $platform == 'linux' ]]; then
+        USER=`/bin/ps -ouser --noheader $PID`
+    elif [[ $platform == 'darwin' ]]; then
+        USER=`/bin/ps -ouser $PID`
+    fi
+else
+    echo "No such pid running: $PID"
+    usage_and_quit
+fi
+
+if [ "$CMD" != "start" ] && [ "$CMD" != "kill" ]; then
+    if [[ $3 ]] && [[ -d $3 ]]
+    then
+        TARGETDIR="$3"
+        mkdir -p ${TARGETDIR}
+    else
+        echo "Missing target directory"
+        usage_and_quit
+    fi
+fi
+
+NOW=`date +'%Y%m%d-%H%M%S'`
+if [ "$CMD" = "" ]; then
+    usage_and_quit
+elif [ "$CMD" = "kill" ]; then
+    echo "Killing process with pid: $PID"
+    kill -9 ${PID}
+elif [ "$CMD" = "start" ]; then
+    if [[ $3 ]]
+    then
+        SETTINGS=$3
+    fi
+    start_record ${PID}
+elif [ "$CMD" = "stop" ]; then
+    echo "Capturing dump before stopping in dir $TARGETDIR"
+    stop_record ${PID} ${TARGETDIR}
+elif [ "$CMD" = "jstack" ]; then
+    echo "Capturing dump in dir $TARGETDIR"
+    jstack_record ${PID} ${TARGETDIR}
+elif [ "$CMD" = "jmap" ]; then
+    echo "Capturing dump in dir $TARGETDIR"
+    jmap_record ${PID} ${TARGETDIR}
+elif [ "$CMD" = "dump" ]; then
+    echo "Capturing dump in dir $TARGETDIR"
+    dump_record ${PID} ${TARGETDIR}
+else
+    usage_and_quit
+fi
+
+
diff --git a/bin/storm b/bin/storm
index 809a83a..0963065 100755
--- a/bin/storm
+++ b/bin/storm
@@ -1,4 +1,4 @@
-#!/bin/bash
+#!/usr/bin/env bash
 #
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
@@ -52,18 +52,16 @@
 export STORM_BASE_DIR=`cd ${STORM_BIN_DIR}/..;pwd`
 
 #check to see if the conf dir is given as an optional argument
-if [ $# -gt 1 ]
-then
-    if [ "--config" = "$1" ]
-          then
-              conf_file=$2
-              if [ ! -f "$conf_file" ]; then
-                echo "Error: Cannot find configuration directory: $conf_file"
-                exit 1
-             fi
-              STORM_CONF_FILE=$conf_file
-              STORM_CONF_DIR=`dirname $conf_file`
+if [ $# -gt 1 ]; then
+  if [ "--config" = "$1" ]; then
+    conf_file=$2
+    if [ ! -f "$conf_file" ]; then
+      echo "Error: Cannot find configuration directory: $conf_file"
+      exit 1
     fi
+    STORM_CONF_FILE=$conf_file
+    STORM_CONF_DIR=`dirname $conf_file`
+  fi
 fi
 
 export STORM_CONF_DIR="${STORM_CONF_DIR:-$STORM_BASE_DIR/conf}"
diff --git a/bin/storm-config.cmd b/bin/storm-config.cmd
index b839160..d259e30 100644
--- a/bin/storm-config.cmd
+++ b/bin/storm-config.cmd
@@ -87,7 +87,7 @@
 @rem
 
 "%JAVA%" -client -Dstorm.options= -Dstorm.conf.file= -cp "%CLASSPATH%" backtype.storm.command.config_value storm.log4j2.conf.dir > %CMD_TEMP_FILE%
-  
+
 FOR /F "delims=" %%i in (%CMD_TEMP_FILE%) do (
 	FOR /F "tokens=1,* delims= " %%a in ("%%i") do (
 		if %%a == VALUE: (
@@ -95,22 +95,22 @@
 			del /F %CMD_TEMP_FILE%)
 		)
 	)
-)		
+)
 
 @rem
 @rem if STORM_LOG4J2_CONFIGURATION_DIR was defined, also set STORM_LOG4J2_CONFIGURATION_FILE
 @rem
 
 if not %STORM_LOG4J2_CONFIGURATION_DIR% == nil (
-	set STORM_LOG4J2_CONFIGURATION_FILE=%STORM_LOG4J2_CONFIGURATION_DIR%\cluster.xml
-) 
+	set STORM_LOG4J2_CONFIGURATION_FILE="file://%STORM_LOG4J2_CONFIGURATION_DIR%\cluster.xml"
+)
 
 @rem
 @rem otherwise, fall back to default
 @rem
 
 if not defined STORM_LOG4J2_CONFIGURATION_FILE (
-  set STORM_LOG4J2_CONFIGURATION_FILE=%STORM_HOME%\log4j2\cluster.xml
+  set STORM_LOG4J2_CONFIGURATION_FILE="file://%STORM_HOME%\log4j2\cluster.xml"
 )
 
 "%JAVA%" -client -Dstorm.options= -Dstorm.conf.file= -cp "%CLASSPATH%" backtype.storm.command.config_value java.library.path > %CMD_TEMP_FILE%
diff --git a/bin/storm.py b/bin/storm.py
index 024912f..44d54d5 100755
--- a/bin/storm.py
+++ b/bin/storm.py
@@ -218,7 +218,7 @@
     Runs the main method of class with the specified arguments.
     The storm jars and configs in ~/.storm are put on the classpath.
     The process is configured so that StormSubmitter
-    (http://storm.incubator.apache.org/apidocs/backtype/storm/StormSubmitter.html)
+    (http://storm.apache.org/apidocs/backtype/storm/StormSubmitter.html)
     will upload the jar at topology-jar-path when the topology is submitted.
     """
     exec_storm_class(
@@ -263,6 +263,18 @@
         jvmtype="-client",
         extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
 
+def heartbeats(*args):
+    """Syntax: [storm heartbeats [cmd]]
+
+    list PATH - lists heartbeats nodes under PATH currently in the ClusterState.
+    get  PATH - Get the heartbeat data at PATH
+    """
+    exec_storm_class(
+        "backtype.storm.command.heartbeats",
+        args=args,
+        jvmtype="-client",
+        extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
+    
 def activate(*args):
     """Syntax: [storm activate topology-name]
 
@@ -277,6 +289,39 @@
         jvmtype="-client",
         extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
 
+def set_log_level(*args):
+    """
+    Dynamically change topology log levels
+
+    Syntax: [storm set_log_level -l [logger name]=[log level][:optional timeout] -r [logger name]
+    where log level is one of:
+        ALL, TRACE, DEBUG, INFO, WARN, ERROR, FATAL, OFF
+    and timeout is integer seconds.
+
+    e.g.
+        ./bin/storm set_log_level -l ROOT=DEBUG:30
+
+        Set the root logger's level to DEBUG for 30 seconds
+
+        ./bin/storm set_log_level -l com.myapp=WARN
+
+        Set the com.myapp logger's level to WARN for 30 seconds
+
+        ./bin/storm set_log_level -l com.myapp=WARN -l com.myOtherLogger=ERROR:123
+
+        Set the com.myapp logger's level to WARN indifinitely, and com.myOtherLogger
+        to ERROR for 123 seconds
+
+        ./bin/storm set_log_level -r com.myOtherLogger
+
+        Clears settings, resetting back to the original level
+    """
+    exec_storm_class(
+        "backtype.storm.command.set_log_level",
+        args=args,
+        jvmtype="-client",
+        extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
+
 def listtopos(*args):
     """Syntax: [storm list]
 
@@ -348,6 +393,30 @@
         jvmtype="-client",
         extrajars=[USER_CONF_DIR, os.path.join(STORM_DIR, "bin")])
 
+def healthcheck(*args):
+    """Syntax: [storm node-health-check]
+
+    Run health checks on the local supervisor.
+    """
+    exec_storm_class(
+        "backtype.storm.command.healthcheck",
+        args=args,
+        jvmtype="-client",
+        extrajars=[USER_CONF_DIR, os.path.join(STORM_DIR, "bin")])
+
+def kill_workers(*args):
+    """Syntax: [storm kill_workers]
+
+    Kill the workers running on this supervisor. This command should be run
+    on a supervisor node. If the cluster is running in secure mode, then user needs
+    to have admin rights on the node to be able to successfully kill all workers.
+    """
+    exec_storm_class(
+        "backtype.storm.command.kill_workers",
+        args=args,
+        jvmtype="-client",
+        extrajars=[USER_CONF_DIR, os.path.join(STORM_DIR, "bin")])
+
 def shell(resourcesdir, command, *args):
     tmpjarpath = "stormshell" + str(random.randint(0, 10000000)) + ".jar"
     os.system("jar cf %s %s" % (tmpjarpath, resourcesdir))
@@ -375,6 +444,8 @@
     storm_log4j2_conf_dir = confvalue("storm.log4j2.conf.dir", cppaths)
     if(storm_log4j2_conf_dir == None or storm_log4j2_conf_dir == "nil"):
         storm_log4j2_conf_dir = STORM_LOG4J2_CONF_DIR
+    elif(not os.path.isabs(storm_log4j2_conf_dir)):
+        storm_log4j2_conf_dir = os.path.join(STORM_DIR, storm_log4j2_conf_dir)
     return storm_log4j2_conf_dir
 
 def nimbus(klass="backtype.storm.daemon.nimbus"):
@@ -384,11 +455,12 @@
     supervision with a tool like daemontools or monit.
 
     See Setting up a Storm cluster for more information.
-    (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster)
+    (http://storm.apache.org/documentation/Setting-up-a-Storm-cluster)
     """
     cppaths = [CLUSTER_CONF_DIR]
     jvmopts = parse_args(confvalue("nimbus.childopts", cppaths)) + [
         "-Dlogfile.name=nimbus.log",
+        "-DLog4jContextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector",
         "-Dlog4j.configurationFile=" + os.path.join(get_log4j2_conf_dir(), "cluster.xml"),
     ]
     exec_storm_class(
@@ -398,6 +470,27 @@
         extrajars=cppaths,
         jvmopts=jvmopts)
 
+def pacemaker(klass="org.apache.storm.pacemaker.pacemaker"):
+    """Syntax: [storm pacemaker]
+
+    Launches the Pacemaker daemon. This command should be run under
+    supervision with a tool like daemontools or monit.
+
+    See Setting up a Storm cluster for more information.
+    (http://storm.apache.org/documentation/Setting-up-a-Storm-cluster)
+    """
+    cppaths = [CLUSTER_CONF_DIR]
+    jvmopts = parse_args(confvalue("pacemaker.childopts", cppaths)) + [
+        "-Dlogfile.name=pacemaker.log",
+        "-Dlog4j.configurationFile=" + os.path.join(get_log4j2_conf_dir(), "cluster.xml"),
+    ]
+    exec_storm_class(
+        klass,
+        jvmtype="-server",
+        daemonName="pacemaker",
+        extrajars=cppaths,
+        jvmopts=jvmopts)
+
 def supervisor(klass="backtype.storm.daemon.supervisor"):
     """Syntax: [storm supervisor]
 
@@ -405,11 +498,12 @@
     under supervision with a tool like daemontools or monit.
 
     See Setting up a Storm cluster for more information.
-    (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster)
+    (http://storm.apache.org/documentation/Setting-up-a-Storm-cluster)
     """
     cppaths = [CLUSTER_CONF_DIR]
     jvmopts = parse_args(confvalue("supervisor.childopts", cppaths)) + [
         "-Dlogfile.name=" + STORM_SUPERVISOR_LOG_FILE,
+        "-DLog4jContextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector",
         "-Dlog4j.configurationFile=" + os.path.join(get_log4j2_conf_dir(), "cluster.xml"),
     ]
     exec_storm_class(
@@ -427,11 +521,12 @@
     should be run under supervision with a tool like daemontools or monit.
 
     See Setting up a Storm cluster for more information.
-    (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster)
+    (http://storm.apache.org/documentation/Setting-up-a-Storm-cluster)
     """
     cppaths = [CLUSTER_CONF_DIR]
     jvmopts = parse_args(confvalue("ui.childopts", cppaths)) + [
         "-Dlogfile.name=ui.log",
+        "-DLog4jContextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector",
         "-Dlog4j.configurationFile=" + os.path.join(get_log4j2_conf_dir(), "cluster.xml")
     ]
     exec_storm_class(
@@ -449,11 +544,12 @@
     tool like daemontools or monit.
 
     See Setting up a Storm cluster for more information.
-    (http://storm.incubator.apache.org/documentation/Setting-up-a-Storm-cluster)
+    (http://storm.apache.org/documentation/Setting-up-a-Storm-cluster)
     """
     cppaths = [CLUSTER_CONF_DIR]
     jvmopts = parse_args(confvalue("logviewer.childopts", cppaths)) + [
         "-Dlogfile.name=logviewer.log",
+        "-DLog4jContextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector",
         "-Dlog4j.configurationFile=" + os.path.join(get_log4j2_conf_dir(), "cluster.xml")
     ]
     exec_storm_class(
@@ -470,11 +566,12 @@
     with a tool like daemontools or monit.
 
     See Distributed RPC for more information.
-    (http://storm.incubator.apache.org/documentation/Distributed-RPC)
+    (http://storm.apache.org/documentation/Distributed-RPC)
     """
     cppaths = [CLUSTER_CONF_DIR]
     jvmopts = parse_args(confvalue("drpc.childopts", cppaths)) + [
         "-Dlogfile.name=drpc.log",
+        "-DLog4jContextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector",
         "-Dlog4j.configurationFile=" + os.path.join(get_log4j2_conf_dir(), "cluster.xml")
     ]
     exec_storm_class(
@@ -537,7 +634,7 @@
     """Print all client commands and link to documentation"""
     print("Commands:\n\t" +  "\n\t".join(sorted(COMMANDS.keys())))
     print("\nHelp: \n\thelp \n\thelp <command>")
-    print("\nDocumentation for the storm client can be found at http://storm.incubator.apache.org/documentation/Command-line-client.html\n")
+    print("\nDocumentation for the storm client can be found at http://storm.apache.org/documentation/Command-line-client.html\n")
     print("Configs can be overridden using one or more -c flags, e.g. \"storm list -c nimbus.host=nimbus.mycompany.com\"\n")
 
 def print_usage(command=None):
@@ -561,7 +658,9 @@
             "remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath,
             "activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage,
             "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version, "monitor": monitor,
-            "upload-credentials": upload_credentials, "get-errors": get_errors }
+            "upload-credentials": upload_credentials, "pacemaker": pacemaker, "heartbeats": heartbeats,
+            "get-errors": get_errors, "set_log_level": set_log_level, "kill_workers": kill_workers,
+            "node-health-check": healthcheck}
 
 def parse_config(config_list):
     global CONFIG_OPTS
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 43ef8f4..295ac7c 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -49,8 +49,11 @@
 storm.auth.simple-acl.users: []
 storm.auth.simple-acl.users.commands: []
 storm.auth.simple-acl.admins: []
+storm.cluster.state.store: "backtype.storm.cluster_state.zookeeper_state_factory"
 storm.meta.serialization.delegate: "backtype.storm.serialization.GzipThriftSerializationDelegate"
 storm.codedistributor.class: "backtype.storm.codedistributor.LocalFileSystemCodeDistributor"
+storm.health.check.dir: "healthchecks"
+storm.health.check.timeout.ms: 5000
 
 ### nimbus.* configs are for the master
 nimbus.seeds : ["localhost"]
@@ -65,12 +68,14 @@
 nimbus.inbox.jar.expiration.secs: 3600
 nimbus.code.sync.freq.secs: 300
 nimbus.task.launch.secs: 120
-nimbus.reassign: true
 nimbus.file.copy.expiration.secs: 600
 nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator"
 topology.min.replication.count: 1
 topology.max.replication.wait.time.sec: 60
 nimbus.credential.renewers.freq.secs: 600
+nimbus.impersonation.authorizer: "backtype.storm.security.auth.authorizer.ImpersonationAuthorizer"
+
+scheduler.display.resource: false
 
 ### ui.* configs are for the master
 ui.host: 0.0.0.0
@@ -87,6 +92,8 @@
 logviewer.childopts: "-Xmx128m"
 logviewer.cleanup.age.mins: 10080
 logviewer.appender.name: "A1"
+logviewer.max.sum.worker.logs.size.mb: 4096
+logviewer.max.per.worker.logs.size.mb: 2048
 
 logs.users: null
 
@@ -132,12 +139,24 @@
 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"
+worker.heap.memory.mb: 768
+worker.childopts: "-Xmx%HEAP-MEM%m -XX:+PrintGCDetails -Xloggc:artifacts/gc.log -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=1M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=artifacts/heapdump"
 worker.gc.childopts: ""
+worker.profiler.childopts: "-XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
+worker.profiler.enabled: true
+worker.profiler.command: "flight.bash"
 worker.heartbeat.frequency.secs: 1
 
+# check whether dynamic log levels can be reset from DEBUG to INFO in workers
+worker.log.level.reset.poll.secs: 30
+
 # control how many worker receiver threads we need per worker
 topology.worker.receiver.thread.count: 1
 
@@ -171,6 +190,9 @@
 # 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
 
@@ -198,7 +220,6 @@
 topology.transfer.buffer.size: 1024 # batched
 topology.tick.tuple.freq.secs: null
 topology.worker.shared.thread.pool.size: 4
-topology.disruptor.wait.strategy: "com.lmax.disruptor.BlockingWaitStrategy"
 topology.spout.wait.strategy: "backtype.storm.spout.SleepSpoutWaitStrategy"
 topology.sleep.spout.wait.strategy.time.ms: 1
 topology.error.throttle.interval.secs: 10
@@ -211,5 +232,23 @@
 topology.environment: null
 topology.bolts.outgoing.overflow.buffer.enable: false
 topology.disruptor.wait.timeout.millis: 1000
+topology.disruptor.batch.size: 100
+topology.disruptor.batch.timeout.millis: 1
+topology.disable.loadaware: false
+
+# 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"
+
+pacemaker.host: "localhost"
+pacemaker.port: 6699
+pacemaker.base.threads: 10
+pacemaker.max.threads: 50
+pacemaker.thread.timeout: 10
+pacemaker.childopts: "-Xmx1024m"
+pacemaker.auth.method: "NONE"
+pacemaker.kerberos.users: []
diff --git a/conf/storm.yaml.example b/conf/storm.yaml.example
index 5fd35f8..13c2f8e 100644
--- a/conf/storm.yaml.example
+++ b/conf/storm.yaml.example
@@ -19,7 +19,7 @@
 #     - "server1"
 #     - "server2"
 # 
-# nimbus.host: "nimbus"
+# nimbus.seeds: ["host1", "host2", "host3"]
 # 
 # 
 # ##### These may optionally be filled in:
diff --git a/dev-tools/storm-merge.py b/dev-tools/storm-merge.py
index ed06216..33690c2 100755
--- a/dev-tools/storm-merge.py
+++ b/dev-tools/storm-merge.py
@@ -24,7 +24,7 @@
 
         for pullNumber in args:
 		pull = github.pull("apache", "storm", pullNumber)
-		print "git pull "+pull.from_repo()+" "+pull.from_branch()
+		print "git pull --no-ff "+pull.from_repo()+" "+pull.from_branch()
 
 if __name__ == "__main__":
 	main()
diff --git a/dev-tools/travis/ratprint.py b/dev-tools/travis/ratprint.py
new file mode 100755
index 0000000..5031045
--- /dev/null
+++ b/dev-tools/travis/ratprint.py
@@ -0,0 +1,26 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+#  Licensed 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.
+import sys
+import re
+
+p = re.compile('Unapproved licenses:\s*([^\s\*]*).*\*\*\*')
+
+with open (sys.argv[1]) as ratfile:
+  rat = ratfile.read().replace('\n','')
+
+matches = p.search(rat)
+failed =  matches.group(1)
+
+if re.search('\S', failed):
+  print failed
diff --git a/dev-tools/travis/travis-install.sh b/dev-tools/travis/travis-install.sh
index 6af2f90..4cbebb1 100755
--- a/dev-tools/travis/travis-install.sh
+++ b/dev-tools/travis/travis-install.sh
@@ -15,10 +15,12 @@
 PYTHON_VERSION=`cat /tmp/python_version`
 RUBY_VERSION=`ruby -v`
 NODEJS_VERSION=`node -v`
+MVN_VERSION=`mvn -v`
 
 echo "Python version : $PYTHON_VERSION"
 echo "Ruby version : $RUBY_VERSION"
 echo "NodeJs version : $NODEJS_VERSION"
+echo "mvn version : $MVN_VERSION"
 
 STORM_SRC_ROOT_DIR=$1
 
@@ -26,12 +28,17 @@
 
 cd ${STORM_SRC_ROOT_DIR}
 
-python ${TRAVIS_SCRIPT_DIR}/save-logs.py "install.txt" mvn clean install --batch-mode -DskipTests -Pnative
+python ${TRAVIS_SCRIPT_DIR}/save-logs.py "install.txt" mvn clean install -DskipTests -Pnative --batch-mode
 BUILD_RET_VAL=$?
 
 if [[ "$BUILD_RET_VAL" != "0" ]];
 then
   cat "install.txt"
+  echo "Looking for unapproved licenses"
+  for rat in `find . -name rat.txt`;
+  do
+    python ${TRAVIS_SCRIPT_DIR}/ratprint.py "${rat}"
+  done
 fi
 
 exit ${BUILD_RET_VAL}
diff --git a/dev-tools/travis/travis-script.sh b/dev-tools/travis/travis-script.sh
index 4caff17..a302e23 100755
--- a/dev-tools/travis/travis-script.sh
+++ b/dev-tools/travis/travis-script.sh
@@ -15,11 +15,12 @@
 PYTHON_VERSION=`cat /tmp/python_version`
 RUBY_VERSION=`ruby -v`
 NODEJS_VERSION=`node -v`
+MVN_VERSION=`mvn -v`
 
 echo "Python version : $PYTHON_VERSION"
 echo "Ruby version : $RUBY_VERSION"
 echo "NodeJs version : $NODEJS_VERSION"
-
+echo "mvn version : $MVN_VERSION"
 
 STORM_SRC_ROOT_DIR=$1
 
@@ -27,11 +28,11 @@
 
 cd ${STORM_SRC_ROOT_DIR}
 
-# We should concern that Travis CI could be very slow cause it uses VM
-export STORM_TEST_TIMEOUT_MS=100000
+# We should be concerned that Travis CI could be very slow because it uses VM
+export STORM_TEST_TIMEOUT_MS=150000
 
 # We now lean on Travis CI's implicit behavior, ```mvn clean install -DskipTests``` before running script
-mvn --batch-mode test -fae -Pnative
+mvn --batch-mode test -fae -Pnative -pl $2
 BUILD_RET_VAL=$?
 
 for dir in `find . -type d -and -wholename \*/target/\*-reports`;
@@ -40,4 +41,10 @@
   python ${TRAVIS_SCRIPT_DIR}/print-errors-from-test-reports.py "${dir}"
 done
 
+echo "Looking for unapproved licenses"
+for rat in `find . -name rat.txt`;
+do
+  python ${TRAVIS_SCRIPT_DIR}/ratprint.py "${rat}"
+done
+
 exit ${BUILD_RET_VAL}
diff --git a/docs/documentation/Documentation.md b/docs/documentation/Documentation.md
index f32268c..7044664 100644
--- a/docs/documentation/Documentation.md
+++ b/docs/documentation/Documentation.md
@@ -44,6 +44,10 @@
 * [Hooks](Hooks.html)
 * [Metrics](Metrics.html)
 * [Lifecycle of a trident tuple]()
+* [UI REST API](ui-rest-api.html)
+* [Log Search](Log-Search.html)
+* [Dynamic Log Level Settings](dynamic-log-level-settings.html)
+* [Dynamic Worker Profiling](dynamic-worker-profiling.html)
 
 ### Advanced
 
diff --git a/docs/documentation/FAQ.md b/docs/documentation/FAQ.md
index a69862e..90222d1 100644
--- a/docs/documentation/FAQ.md
+++ b/docs/documentation/FAQ.md
@@ -112,7 +112,7 @@
 
 ### How do I aggregate events by time?
 
-If have records with an immutable timestamp, and you would like to count, average or otherwise aggregate them into discrete time buckets, Trident is an excellent and scalable solution.
+If you have records with an immutable timestamp, and you would like to count, average or otherwise aggregate them into discrete time buckets, Trident is an excellent and scalable solution.
 
 Write an `Each` function that turns the timestamp into a time bucket: if the bucket size was "by hour", then the timestamp `2013-08-08 12:34:56` would be mapped to the `2013-08-08 12:00:00` time bucket, and so would everything else in the twelve o'clock hour. Then group on that timebucket and use a grouped persistentAggregate. The persistentAggregate uses a local cacheMap backed by a data store. Groups with many records require very few reads from the data store, and use efficient bulk reads and writes; as long as your data feed is relatively prompt Trident will make very efficient use of memory and network. Even if a server drops off line for a day, then delivers that full day's worth of data in a rush, the old results will be calmly retrieved and updated -- and without interfering with calculating the current results.
 
diff --git a/docs/documentation/Log-Search.md b/docs/documentation/Log-Search.md
new file mode 100644
index 0000000..03297b7
--- /dev/null
+++ b/docs/documentation/Log-Search.md
@@ -0,0 +1,14 @@
+Log Search
+==========================
+
+This feature is aimed for improving the debugging of Storm. Log Search supports searching in a certain log file or in a topology's all log files:
+
+Using the Storm UI
+-------------
+String search in a log file: In the log page for a worker, user can search a certain string, e.g., "Exception", in for a certain worker log. This search can happen for both normal text log or rolled ziplog files. In the results, the offset and matched lines will be displayed.
+
+![Search in a log](images/search-for-a-single-worker-log.png "Seach in a log")
+
+Search in a topology: User can also search a string for a certain topology by clicking the icon of magnifying lens at the top right corner of the UI page. This means the UI will try to search on all the supervisor nodes in a distributed way to find the matched string in all logs for this topology. The search can happen for both normal text log or rolled zip log files by checking/unchecking the "Search archived logs:" box. Then the matched results can be shown on the UI with url links, directing user to the certain logs on each supervisor node. This powerful feature is very helpful for users to find certain problematic supervisor nodes running this topology.
+
+![Seach in a topology](images/search-a-topology.png "Search in a topology")
diff --git a/docs/documentation/Message-passing-implementation.md b/docs/documentation/Message-passing-implementation.md
index e17fd3f..a17f66a 100644
--- a/docs/documentation/Message-passing-implementation.md
+++ b/docs/documentation/Message-passing-implementation.md
@@ -8,23 +8,23 @@
 This page walks through how emitting and transferring tuples works in Storm.
 
 - Worker is responsible for message transfer
-   - `refresh-connections` is called every "task.refresh.poll.secs" or whenever assignment in ZK changes. It manages connections to other workers and maintains a mapping from task -> worker [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123)
-   - Provides a "transfer function" that is used by tasks to send tuples to other tasks. The transfer function takes in a task id and a tuple, and it serializes the tuple and puts it onto a "transfer queue". There is a single transfer queue for each worker. [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L56)
-   - The serializer is thread-safe [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/jvm/backtype/storm/serialization/KryoTupleSerializer.java#L26)
-   - The worker has a single thread which drains the transfer queue and sends the messages to other workers [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L185)
-   - Message sending happens through this protocol: [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/protocol.clj)
-   - The implementation for distributed mode uses ZeroMQ [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/zmq.clj)
-   - The implementation for local mode uses in memory Java queues (so that it's easy to use Storm locally without needing to get ZeroMQ installed) [code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/local.clj)
+   - `refresh-connections` is called every "task.refresh.poll.secs" or whenever assignment in ZK changes. It manages connections to other workers and maintains a mapping from task -> worker [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123)
+   - Provides a "transfer function" that is used by tasks to send tuples to other tasks. The transfer function takes in a task id and a tuple, and it serializes the tuple and puts it onto a "transfer queue". There is a single transfer queue for each worker. [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L56)
+   - The serializer is thread-safe [code](https://github.com/apache/storm/blob/0.7.1/src/jvm/backtype/storm/serialization/KryoTupleSerializer.java#L26)
+   - The worker has a single thread which drains the transfer queue and sends the messages to other workers [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L185)
+   - Message sending happens through this protocol: [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/protocol.clj)
+   - The implementation for distributed mode uses ZeroMQ [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/zmq.clj)
+   - The implementation for local mode uses in memory Java queues (so that it's easy to use Storm locally without needing to get ZeroMQ installed) [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/local.clj)
 - Receiving messages in tasks works differently in local mode and distributed mode
-   - In local mode, the tuple is sent directly to an in-memory queue for the receiving task [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/messaging/local.clj#L21)
-   - In distributed mode, each worker listens on a single TCP port for incoming messages and then routes those messages in-memory to tasks. The TCP port is called a "virtual port", because it receives [task id, message] and then routes it to the actual task. [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/worker.clj#L204)
-      - The virtual port implementation is here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/zilch/virtual_port.clj)
-      - Tasks listen on an in-memory ZeroMQ port for messages from the virtual port [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L201)
-        - Bolts listen here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L489)
-        - Spouts listen here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L382)
+   - In local mode, the tuple is sent directly to an in-memory queue for the receiving task [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/local.clj#L21)
+   - In distributed mode, each worker listens on a single TCP port for incoming messages and then routes those messages in-memory to tasks. The TCP port is called a "virtual port", because it receives [task id, message] and then routes it to the actual task. [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L204)
+      - The virtual port implementation is here: [code](https://github.com/apache/storm/blob/0.7.1/src/clj/zilch/virtual_port.clj)
+      - Tasks listen on an in-memory ZeroMQ port for messages from the virtual port [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L201)
+        - Bolts listen here: [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L489)
+        - Spouts listen here: [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L382)
 - Tasks are responsible for message routing. A tuple is emitted either to a direct stream (where the task id is specified) or a regular stream. In direct streams, the message is only sent if that bolt subscribes to that direct stream. In regular streams, the stream grouping functions are used to determine the task ids to send the tuple to.
-  - Tasks have a routing map from {stream id} -> {component id} -> {stream grouping function} [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L198)
-  - The "tasks-fn" returns the task ids to send the tuples to for either regular stream emit or direct stream emit [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L207)
+  - Tasks have a routing map from {stream id} -> {component id} -> {stream grouping function} [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L198)
+  - The "tasks-fn" returns the task ids to send the tuples to for either regular stream emit or direct stream emit [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L207)
   - After getting the output task ids, bolts and spouts use the transfer-fn provided by the worker to actually transfer the tuples
-      - Bolt transfer code here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L429)
-      - Spout transfer code here: [code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L329)
+      - Bolt transfer code here: [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L429)
+      - Spout transfer code here: [code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L329)
diff --git a/docs/documentation/Pacemaker.md b/docs/documentation/Pacemaker.md
new file mode 100644
index 0000000..24e3e7f
--- /dev/null
+++ b/docs/documentation/Pacemaker.md
@@ -0,0 +1,108 @@
+# Pacemaker
+
+### Introduction
+Pacemaker is a storm daemon designed to process heartbeats from workers. As Storm is scaled up, ZooKeeper begins to become a bottleneck due to high volumes of writes from workers doing heartbeats. Lots of writes to disk and too much traffic across the network is generated as ZooKeeper tries to maintain consistency.
+
+Because heartbeats are of an ephemeral nature, they do not need to be persisted to disk or synced across nodes; an in-memory store will do. This is the role of Pacemaker. Pacemaker functions as a simple in-memory key/value store with ZooKeeper-like, directory-style keys and byte array values.
+
+The corresponding Pacemaker client is a plugin for the `ClusterState` interface, `org.apache.storm.pacemaker.pacemaker_state_factory`. Heartbeat calls are funneled by the `ClusterState` produced by `pacemaker_state_factory` into the Pacemaker daemon, while other set/get operations are forwarded to ZooKeeper.
+
+------
+
+### Configuration
+
+ - `pacemaker.host` : The host that the Pacemaker daemon is running on
+ - `pacemaker.port` : The port that Pacemaker will listen on
+ - `pacemaker.max.threads` : Maximum number of threads Pacemaker daemon will use to handle requests.
+ - `pacemaker.childopts` : Any JVM parameters that need to go to the Pacemaker. (used by storm-deploy project)
+ - `pacemaker.auth.method` : The authentication method that is used (more info below)
+
+#### Example
+
+To get Pacemaker up and running, set the following option in the cluster config on all nodes:
+```
+storm.cluster.state.store: "org.apache.storm.pacemaker.pacemaker_state_factory"
+```
+
+The Pacemaker host also needs to be set on all nodes:
+```
+pacemaker.host: somehost.mycompany.com
+```
+
+And then start all of your daemons
+
+(including Pacemaker):
+```
+$ storm pacemaker
+```
+
+The Storm cluster should now be pushing all worker heartbeats through Pacemaker.
+
+### Security
+
+Currently digest (password-based) and Kerberos security are supported. Security is currently only around reads, not writes. Writes may be performed by anyone, whereas reads may only be performed by authorized and authenticated users. This is an area for future development, as it leaves the cluster open to DoS attacks, but it prevents any sensitive information from reaching unauthorized eyes, which was the main goal.
+
+#### Digest
+To configure digest authentication, set `pacemaker.auth.method: DIGEST` in the cluster config on the nodes hosting Nimbus and Pacemaker.
+The nodes must also have `java.security.auth.login.config` set to point to a JAAS config file containing the following structure:
+```
+PacemakerDigest {
+    username="some username"
+    password="some password";
+};
+```
+
+Any node with these settings configured will be able to read from Pacemaker.
+Worker nodes need not have these configs set, and may keep `pacemaker.auth.method: NONE` set, since they do not need to read from the Pacemaker daemon.
+
+#### Kerberos
+To configure Kerberos authentication, set `pacemaker.auth.method: KERBEROS` in the cluster config on the nodes hosting Nimbus and Pacemaker.
+The nodes must also have `java.security.auth.login.config` set to point to a JAAS config.
+
+The JAAS config on Nimbus must look something like this:
+```
+PacemakerClient {
+    com.sun.security.auth.module.Krb5LoginModule required
+    useKeyTab=true
+    keyTab="/etc/keytabs/nimbus.keytab"
+    storeKey=true
+    useTicketCache=false
+    serviceName="pacemaker"
+    principal="nimbus@MY.COMPANY.COM";
+};
+                         
+```
+
+The JAAS config on Pacemaker must look something like this:
+```
+PacemakerServer {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/etc/keytabs/pacemaker.keytab"
+   storeKey=true
+   useTicketCache=false
+   principal="pacemaker@MY.COMPANY.COM";
+};
+```
+
+ - The client's user principal in the `PacemakerClient` section on the Nimbus host must match the `nimbus.daemon.user` storm cluster config value.
+ - The client's `serviceName` value must match the server's user principal in the `PacemakerServer` section on the Pacemaker host.
+
+
+### Fault Tolerance
+
+Pacemaker runs as a single daemon instance, making it a potential Single Point of Failure.
+
+If Pacemaker becomes unreachable by Nimbus, through crash or network partition, the workers will continue to run, and Nimbus will repeatedly attempt to reconnect. Nimbus functionality will be disrupted, but the topologies themselves will continue to run.
+In case of partition of the cluster where Nimbus and Pacemaker are on the same side of the partition, the workers that are on the other side of the partition will not be able to heartbeat, and Nimbus will reschedule the tasks elsewhere. This is probably what we want to happen anyway.
+
+
+### ZooKeeper Comparison
+Compared to ZooKeeper, Pacemaker uses less CPU, less memory, and of course no disk for the same load, thanks to lack of overhead from maintaining consistency between nodes.
+On Gigabit networking, there is a theoretical limit of about 6000 nodes. However, the real limit is likely around 2000-3000 nodes. These limits have not yet been tested.
+On a 270 supervisor cluster, fully scheduled with topologies, Pacemaker resource utilization was 70% of one core and nearly 1GiB of RAM on a machine with 4 `Intel(R) Xeon(R) CPU E5530 @ 2.40GHz` and 24GiB of RAM.
+
+
+There is an easy route to HA for Pacemaker. Unlike ZooKeeper, Pacemaker should be able to scale horizontally without overhead. By contrast, with ZooKeeper, there are diminishing returns when adding ZK nodes.
+
+In short, a single Pacemaker node should be able to handle many times the load that a ZooKeeper cluster can, and future HA work allowing horizontal scaling will increase that even farther.
diff --git a/docs/documentation/Setting-up-a-Storm-cluster.md b/docs/documentation/Setting-up-a-Storm-cluster.md
index 9fabb27..ee4ad15 100644
--- a/docs/documentation/Setting-up-a-Storm-cluster.md
+++ b/docs/documentation/Setting-up-a-Storm-cluster.md
@@ -80,6 +80,25 @@
     - 6703
 ```
 
+### Monitoring Health of Supervisors
+
+Storm provides a mechanism by which administrators can configure the supervisor to run administrator supplied scripts periodically to determine if a node is healthy or not. Administrators can have the supervisor determine if the node is in a healthy state by performing any checks of their choice in scripts located in storm.health.check.dir. If a script detects the node to be in an unhealthy state, it must print a line to standard output beginning with the string ERROR. The supervisor will periodically run the scripts in the health check dir and check the output. If the script’s output contains the string ERROR, as described above, the supervisor will shut down any workers and exit. 
+
+If the supervisor is running with supervision "/bin/storm node-health-check" can be called to determine if the supervisor should be launched or if the node is unhealthy.
+
+The health check directory location can be configured with:
+
+```yaml
+storm.health.check.dir: "healthchecks"
+
+```
+The scripts must have execute permissions.
+The time to allow any given healthcheck script to run before it is marked failed due to timeout can be configured with:
+
+```yaml
+storm.health.check.timeout.ms: 5000
+```
+
 ### Configure external libraries and environmental variables (optional)
 
 If you need support from external libraries or custom plugins, you can place such jars into the extlib/ and extlib-daemon/ directories. Note that the extlib-daemon/ directory stores jars used only by daemons (Nimbus, Supervisor, DRPC, UI, Logviewer), e.g., HDFS and customized scheduling libraries. Accordingly, two environmental variables STORM_EXT_CLASSPATH and STORM_EXT_CLASSPATH_DAEMON can be configured by users for including the external classpath and daemon-only external classpath.
diff --git a/docs/documentation/Windowing.md b/docs/documentation/Windowing.md
new file mode 100644
index 0000000..8f9d758
--- /dev/null
+++ b/docs/documentation/Windowing.md
@@ -0,0 +1,144 @@
+# Windowing support in core storm
+
+Storm core has support for processing a group of tuples that falls within a window. Windows are specified with the 
+following two parameters,
+
+1. Window length - the length or duration of the window
+2. Sliding interval - the interval at which the windowing slides
+
+## Sliding Window
+
+Tuples are grouped in windows and window slides every sliding interval. A tuple can belong to more than one window.
+
+For example a time duration based sliding window with length 10 secs and sliding interval of 5 seconds.
+
+```
+| e1 e2 | e3 e4 e5 e6 | e7 e8 e9 |...
+0       5             10         15    -> time
+
+|<------- w1 -------->|
+        |------------ w2 ------->|
+```
+
+The window is evaluated every 5 seconds and some of the tuples in the first window overlaps with the second one.
+		
+
+## Tumbling Window
+
+Tuples are grouped in a single window based on time or count. Any tuple belongs to only one of the windows.
+
+For example a time duration based tumbling window with length 5 secs.
+
+```
+| e1 e2 | e3 e4 e5 e6 | e7 e8 e9 |...
+0       5             10         15    -> time
+   w1         w2            w3
+```
+
+The window is evaluated every five seconds and none of the windows overlap.
+
+Storm supports specifying the window length and sliding intervals as a count of the number of tuples or as a time duration.
+
+The bolt interface `IWindowedBolt` is implemented by bolts that needs windowing support.
+
+```java
+public interface IWindowedBolt extends IComponent {
+    void prepare(Map stormConf, TopologyContext context, OutputCollector collector);
+    /**
+     * Process tuples falling within the window and optionally emit 
+     * new tuples based on the tuples in the input window.
+     */
+    void execute(TupleWindow inputWindow);
+    void cleanup();
+}
+```
+
+Every time the window activates, the `execute` method is invoked. The TupleWindow parameter gives access to the current tuples
+in the window, the tuples that expired and the new tuples that are added since last window was computed which will be useful 
+for efficient windowing computations.
+
+Bolts that needs windowing support typically would extend `BaseWindowedBolt` which has the apis for specifying the
+window length and sliding intervals.
+
+E.g. 
+
+```java
+public class SlidingWindowBolt extends BaseWindowedBolt {
+	private OutputCollector collector;
+	
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+    	this.collector = collector;
+    }
+	
+    @Override
+    public void execute(TupleWindow inputWindow) {
+	  for(Tuple tuple: inputWindow.get()) {
+	    // do the windowing computation
+		...
+	  }
+	  // emit the results
+	  collector.emit(new Values(computedValue));
+    }
+}
+
+public static void main(String[] args) {
+    TopologyBuilder builder = new TopologyBuilder();
+     builder.setSpout("spout", new RandomSentenceSpout(), 1);
+     builder.setBolt("slidingwindowbolt", 
+                     new SlidingWindowBolt().withWindow(new Count(30), new Count(10)),
+                     1).shuffleGrouping("spout");
+    Config conf = new Config();
+    conf.setDebug(true);
+    conf.setNumWorkers(1);
+
+    StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+	
+}
+```
+
+The following window configurations are supported.
+
+```java
+withWindow(Count windowLength, Count slidingInterval)
+Tuple count based sliding window that slides after `slidingInterval` number of tuples.
+
+withWindow(Count windowLength)
+Tuple count based window that slides with every incoming tuple.
+
+withWindow(Count windowLength, Duration slidingInterval)
+Tuple count based sliding window that slides after `slidingInterval` time duration.
+
+withWindow(Duration windowLength, Duration slidingInterval)
+Time duration based sliding window that slides after `slidingInterval` time duration.
+
+withWindow(Duration windowLength)
+Time duration based window that slides with every incoming tuple.
+
+withWindow(Duration windowLength, Count slidingInterval)
+Time duration based sliding window configuration that slides after `slidingInterval` number of tuples.
+
+withTumblingWindow(BaseWindowedBolt.Count count)
+Count based tumbling window that tumbles after the specified count of tuples.
+
+withTumblingWindow(BaseWindowedBolt.Duration duration)
+Time duration based tumbling window that tumbles after the specified time duration.
+
+```
+
+## Guarentees
+The windowing functionality in storm core currently provides at-least once guarentee. The values emitted from the bolts
+`execute(TupleWindow inputWindow)` method are automatically anchored to all the tuples in the inputWindow. The downstream
+bolts are expected to ack the received tuple (i.e the tuple emitted from the windowed bolt) to complete the tuple tree. 
+If not the tuples will be replayed and the windowing computation will be re-evaluated. 
+
+The tuples in the window are automatically acked when the expire, i.e. when they fall out of the window after 
+`windowLength + slidingInterval`. Note that the configuration `topology.message.timeout.secs` should be sufficiently more 
+than `windowLength + slidingInterval` for time based windows; otherwise the tuples will timeout and get replayed and can result
+in duplicate evaluations. For count based windows, the configuration should be adjusted such that `windowLength + slidingInterval`
+tuples can be received within the timeout period.
+
+## Example topology
+An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
+average.
+
diff --git a/docs/documentation/dynamic-log-level-settings.md b/docs/documentation/dynamic-log-level-settings.md
new file mode 100644
index 0000000..f38b708
--- /dev/null
+++ b/docs/documentation/dynamic-log-level-settings.md
@@ -0,0 +1,41 @@
+Dynamic Log Level Settings
+==========================
+
+We have added the ability to set log level settings for a running topology using the Storm UI and the Storm CLI. 
+
+The log level settings apply the same way as you'd expect from log4j, as all we are doing is telling log4j to set the level of the logger you provide. If you set the log level of a parent logger, the children loggers start using that level (unless the children have a more restrictive level already). A timeout can optionally be provided (except for DEBUG mode, where it’s required in the UI), if workers should reset log levels automatically.
+
+This revert action is triggered using a polling mechanism (every 30 seconds, but this is configurable), so you should expect your timeouts to be the value you provided plus anywhere between 0 and the setting's value.
+
+Using the Storm UI
+-------------
+
+In order to set a level, click on a running topology, and then click on “Change Log Level” in the Topology Actions section.
+
+![Change Log Level dialog](images/dynamic_log_level_settings_1.png "Change Log Level dialog")
+
+Next, provide the logger name, select the level you expect (e.g. WARN), and a timeout in seconds (or 0 if not needed). Then click on “Add”.
+
+![After adding a log level setting](images/dynamic_log_level_settings_2.png "After adding a log level setting")
+
+To clear the log level click on the “Clear” button. This reverts the log level back to what it was before you added the setting. The log level line will disappear from the UI.
+
+While there is a delay resetting log levels back, setting the log level in the first place is immediate (or as quickly as the message can travel from the UI/CLI to the workers by way of nimbus and zookeeper).
+
+Using the CLI
+-------------
+
+Using the CLI, issue the command:
+
+`./bin/storm set_log_level [topology name] -l [logger name]=[LEVEL]:[TIMEOUT]`
+
+For example:
+
+`./bin/storm set_log_level my_topology -l ROOT=DEBUG:30`
+
+Sets the ROOT logger to DEBUG for 30 seconds.
+
+`./bin/storm set_log_level my_topology -r ROOT`
+
+Clears the ROOT logger dynamic log level, resetting it to its original value.
+
diff --git a/docs/documentation/dynamic-worker-profiling.md b/docs/documentation/dynamic-worker-profiling.md
new file mode 100644
index 0000000..9bc8da3
--- /dev/null
+++ b/docs/documentation/dynamic-worker-profiling.md
@@ -0,0 +1,29 @@
+Dynamic Worker Profiling
+==========================
+
+In multi-tenant mode, storm launches long-running JVMs across cluster without sudo access to user. Self-serving of Java heap-dumps, jstacks and java profiling of these JVMs would improve users' ability to analyze and debug issues when monitoring it actively.
+
+The storm dynamic profiler lets you dynamically take heap-dumps, jprofile or jstack for a worker jvm running on stock cluster. It let user download these dumps from the browser and use your favorite tools to analyze it  The UI component page provides list workers for the component and action buttons. The logviewer lets you download the dumps generated by these logs. Please see the screenshots for more information.
+
+Using the Storm UI
+-------------
+
+In order to request for heap-dump, jstack, start/stop/dump jprofile or restart a worker, click on a running topology, then click on specific component, then you can select worker from the dropdown for that particular component and then click on “Start","Heap", "Jstack" or "Restart Worker" in the "Profiing and Debugging" section.
+
+![Profiling and Debugging](images/dynamic_profiling_debugging_1.png "Profiling and Debugging")
+
+For start jprofile, provide a timeout in minutes (or 10 if not needed). Then click on “Start”.
+
+![After starting jprofile for worker](images/dynamic_profiling_debugging_2.png "After jprofile for worker ")
+
+To stop the jprofile logging click on the “Stop” button. This dumps the jprofile stats and stops the profiling. Refresh the page for the line to disappear from the UI.
+
+Click on "My Dump Files" to go the logviewer UI for list of worker specific dump files.
+
+![Dump Files Links for worker](images/dynamic_profiling_debugging_3.png "Dump Files Links for worker")
+
+Configuration
+-------------
+
+The "worker.profiler.command" can be configured to point to specific pluggable profiler, heapdump commands. The "worker.profiler.enabled" can be disabled if plugin is not available or jdk does not support Jprofile flight recording so that worker JVM options will not have "worker.profiler.childopts". To use different profiler plugin, you can change these configuration.
+
diff --git a/docs/documentation/images/dynamic_log_level_settings_1.png b/docs/documentation/images/dynamic_log_level_settings_1.png
new file mode 100644
index 0000000..71d42e7
--- /dev/null
+++ b/docs/documentation/images/dynamic_log_level_settings_1.png
Binary files differ
diff --git a/docs/documentation/images/dynamic_log_level_settings_2.png b/docs/documentation/images/dynamic_log_level_settings_2.png
new file mode 100644
index 0000000..d0e61a7
--- /dev/null
+++ b/docs/documentation/images/dynamic_log_level_settings_2.png
Binary files differ
diff --git a/docs/documentation/images/dynamic_profiling_debugging_1.png b/docs/documentation/images/dynamic_profiling_debugging_1.png
new file mode 100644
index 0000000..3913e86
--- /dev/null
+++ b/docs/documentation/images/dynamic_profiling_debugging_1.png
Binary files differ
diff --git a/docs/documentation/images/dynamic_profiling_debugging_2.png b/docs/documentation/images/dynamic_profiling_debugging_2.png
new file mode 100644
index 0000000..66c0236
--- /dev/null
+++ b/docs/documentation/images/dynamic_profiling_debugging_2.png
Binary files differ
diff --git a/docs/documentation/images/dynamic_profiling_debugging_3.png b/docs/documentation/images/dynamic_profiling_debugging_3.png
new file mode 100644
index 0000000..5706d7e
--- /dev/null
+++ b/docs/documentation/images/dynamic_profiling_debugging_3.png
Binary files differ
diff --git a/docs/documentation/images/search-a-topology.png b/docs/documentation/images/search-a-topology.png
new file mode 100644
index 0000000..8d6153c
--- /dev/null
+++ b/docs/documentation/images/search-a-topology.png
Binary files differ
diff --git a/docs/documentation/images/search-for-a-single-worker-log.png b/docs/documentation/images/search-for-a-single-worker-log.png
new file mode 100644
index 0000000..8c6f423
--- /dev/null
+++ b/docs/documentation/images/search-for-a-single-worker-log.png
Binary files differ
diff --git a/docs/documentation/storm-metrics-profiling-internal-actions.md b/docs/documentation/storm-metrics-profiling-internal-actions.md
new file mode 100644
index 0000000..e549c0c
--- /dev/null
+++ b/docs/documentation/storm-metrics-profiling-internal-actions.md
@@ -0,0 +1,70 @@
+# Storm Metrics for Profiling Various Storm Internal Actions
+
+With the addition of these metrics, Storm users can collect, view, and analyze the performance of various internal actions.  The actions that are profiled include thrift rpc calls and http quests within the storm daemons. For instance, in the Storm Nimbus daemon, the following thrift calls defined in the Nimbus$Iface are profiled:
+
+- submitTopology
+- submitTopologyWithOpts
+- killTopology
+- killTopologyWithOpts
+- activate
+- deactivate
+- rebalance
+- setLogConfig
+- getLogConfig
+
+Various HTTP GET and POST requests are marked for profiling as well such as the GET and POST requests for the Storm UI daemon (ui/core.cj)
+To implement these metrics the following packages are used: 
+- io.dropwizard.metrics
+- metrics-clojure
+
+## How it works
+
+By using packages io.dropwizard.metrics and metrics-clojure (clojure wrapper for the metrics Java API), we can mark functions to profile by declaring (defmeter num-some-func-calls) and then adding the (mark! num-some-func-calls) to where the function is invoked. For example:
+
+    (defmeter num-some-func-calls)
+    (defn some-func [args]
+        (mark! num-some-func-calls)
+        (body))
+        
+What essentially the mark! API call does is increment a counter that represents how many times a certain action occured.  For instantanous measurements user can use gauges.  For example: 
+
+    (defgauge nimbus:num-supervisors
+         (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))
+         
+The above example will get the number of supervisors in the cluster.  This metric is not accumlative like one previously discussed.
+
+A metrics reporting server needs to also be activated to collect the metrics. You can do this by calling the following function:
+
+    (defn start-metrics-reporters []
+        (jmx/start (jmx/reporter {})))
+
+## How to collect the metrics
+
+Metrics can be reported via JMX or HTTP.  A user can use JConsole or VisualVM to connect to the jvm process and view the stats.
+
+To view the metrics in a GUI use VisualVM or JConsole.  Screenshot of using VisualVm for metrics: 
+
+![Viewing metrics with VisualVM](images/viewing_metrics_with_VisualVM.png)
+
+For detailed information regarding how to collect the metrics please reference: 
+
+https://dropwizard.github.io/metrics/3.1.0/getting-started/
+
+If you want use JMX and view metrics through JConsole or VisualVM, remember launch JVM processes your want to profile with the correct JMX configurations.  For example in Storm you would add the following to conf/storm.yaml
+
+    nimbus.childopts: "-Xmx1024m -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=3333  -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+    
+    ui.childopts: "-Xmx768m -Dcom.sun.management.jmxremote.port=3334 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+    
+    logviewer.childopts: "-Xmx128m -Dcom.sun.management.jmxremote.port=3335 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+    
+    drpc.childopts: "-Xmx768m -Dcom.sun.management.jmxremote.port=3336 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+   
+    supervisor.childopts: "-Xmx256m -Dcom.sun.management.jmxremote.port=3337 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+
+### Please Note:
+Since we shade all of the packages we use, additional plugins for collecting metrics might not work at this time.  Currently collecting the metrics via JMX is supported.
+   
+For more information about io.dropwizard.metrics and metrics-clojure packages please reference their original documentation:
+- https://dropwizard.github.io/metrics/3.1.0/
+- http://metrics-clojure.readthedocs.org/en/latest/
\ No newline at end of file
diff --git a/STORM-UI-REST-API.md b/docs/documentation/ui-rest-api.md
similarity index 76%
rename from STORM-UI-REST-API.md
rename to docs/documentation/ui-rest-api.md
index a8f2bd1..7de2bcb 100644
--- a/STORM-UI-REST-API.md
+++ b/docs/documentation/ui-rest-api.md
@@ -84,7 +84,6 @@
 |Field  |Value|Description
 |---	|---	|---
 |stormVersion|String| Storm version|
-|nimbusUptime|String| Shows how long the cluster is running|
 |supervisors|Integer| Number of supervisors running|
 |topologies| Integer| Number of topologies running| 
 |slotsTotal| Integer|Total number of available worker slots|
@@ -98,7 +97,6 @@
 ```json
    {
     "stormVersion": "0.9.2-incubating-SNAPSHOT",
-    "nimbusUptime": "3m 53s",
     "supervisors": 1,
     "slotsTotal": 4,
     "slotsUsed": 3,
@@ -119,8 +117,13 @@
 |id| String | Supervisor's id|
 |host| String| Supervisor's host name|
 |uptime| String| Shows how long the supervisor is running|
+|uptimeSeconds| Integer| Shows how long the supervisor is running in seconds|
 |slotsTotal| Integer| Total number of available worker slots for this supervisor|
 |slotsUsed| Integer| Number of worker slots used on this supervisor|
+|totalMem| Double| Total memory capacity on this supervisor|
+|totalCpu| Double| Total CPU capacity on this supervisor|
+|usedMem| Double| Used memory capacity on this supervisor|
+|usedCpu| Double| Used CPU capacity on this supervisor|
 
 Sample response:
 
@@ -131,10 +134,16 @@
             "id": "0b879808-2a26-442b-8f7d-23101e0c3696",
             "host": "10.11.1.7",
             "uptime": "5m 58s",
+            "uptimeSeconds": 358,
             "slotsTotal": 4,
-            "slotsUsed": 3
+            "slotsUsed": 3,
+            "totalMem": 3000,
+            "totalCpu": 400,
+            "usedMem": 1280,
+            "usedCPU": 160
         }
-    ]
+    ],
+    "schedulerDisplayResource": true
 }
 ```
 
@@ -150,6 +159,7 @@
 |port| int| Nimbus' port number|
 |status| String| Possible values are Leader, Not a Leader, Dead|
 |nimbusUpTime| String| Shows since how long the nimbus has been running|
+|nimbusUpTimeSeconds| String| Shows since how long the nimbus has been running in seconds|
 |nimbusLogLink| String| Logviewer url to view the nimbus.log|
 |version| String| Version of storm this nimbus host is running|
 
@@ -164,12 +174,34 @@
             "nimbusLogLink":"http:\/\/192.168.202.1:8000\/log?file=nimbus.log",
             "status":Leader,
             "version":"0.10.0-SNAPSHOT",
-            "nimbusUpTime":"3m 33s"
+            "nimbusUpTime":"3m 33s",
+            "nimbusUpTimeSeconds":"213"
         }
     ]
 }
 ```
 
+### /api/v1/history/summary (GET)
+
+Returns a list of all running topologies' IDs submitted by the current user.
+
+Response fields:
+
+|Field  |Value | Description|
+|---	|---	|---
+|topo-history| List| List of Topologies' IDs|
+
+Sample response:
+
+```json
+{
+    "topo-history":[
+        "wc6-1-1446571009",
+        "wc8-2-1446587178"
+     ]
+}
+```
+
 ### /api/v1/topology/summary (GET)
 
 Returns summary information for all topologies.
@@ -182,10 +214,20 @@
 |name| String| Topology Name|
 |status| String| Topology Status|
 |uptime| String|  Shows how long the topology is running|
+|uptimeSeconds| Integer|  Shows how long the topology is running in seconds|
 |tasksTotal| Integer |Total number of tasks for this topology|
 |workersTotal| Integer |Number of workers used for this topology|
 |executorsTotal| Integer |Number of executors used for this topology|
 |replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
+|requestedMemOnHeap| Double|Requested On-Heap Memory by User (MB)
+|requestedMemOffHeap| Double|Requested Off-Heap Memory by User (MB)|
+|requestedTotalMem| Double|Requested Total Memory by User (MB)|
+|requestedCpu| Double|Requested CPU by User (%)|
+|assignedMemOnHeap| Double|Assigned On-Heap Memory by Scheduler (MB)|
+|assignedMemOffHeap| Double|Assigned Off-Heap Memory by Scheduler (MB)|
+|assignedTotalMem| Double|Assigned Total Memory by Scheduler (MB)|
+|assignedCpu| Double|Assigned CPU by Scheduler (%)|
+
 Sample response:
 
 ```json
@@ -196,12 +238,55 @@
             "name": "WordCount3",
             "status": "ACTIVE",
             "uptime": "6m 5s",
+            "uptimeSeconds": 365,
             "tasksTotal": 28,
             "workersTotal": 3,
             "executorsTotal": 28,
-            "replicationCount": 1
+            "replicationCount": 1,
+            "requestedMemOnHeap": 640,
+            "requestedMemOffHeap": 128,
+            "requestedTotalMem": 768,
+            "requestedCpu": 80,
+            "assignedMemOnHeap": 640,
+            "assignedMemOffHeap": 128,
+            "assignedTotalMem": 768,
+            "assignedCpu": 80
         }
     ]
+    "schedulerDisplayResource": true
+}
+```
+
+### /api/v1/topology-workers/:id (GET)
+
+Returns the worker' information (host and port) for a topology.
+
+Response fields:
+
+|Field  |Value | Description|
+|---	|---	|---
+|hostPortList| List| Workers' information for a topology|
+|name| Integer| Logviewer Port|
+
+Sample response:
+
+```json
+{
+    "hostPortList":[
+            {
+                "host":"192.168.202.2",
+                "port":6701
+            },
+            {
+                "host":"192.168.202.2",
+                "port":6702
+            },
+            {
+                "host":"192.168.202.3",
+                "port":6700
+            }
+        ],
+    "logviewerPort":8000
 }
 ```
 
@@ -225,12 +310,14 @@
 |id| String| Topology Id|
 |name| String |Topology Name|
 |uptime| String |How long the topology has been running|
+|uptimeSeconds| Integer |How long the topology has been running in seconds|
 |status| String |Current status of the topology, e.g. "ACTIVE"|
 |tasksTotal| Integer |Total number of tasks for this topology|
 |workersTotal| Integer |Number of workers used for this topology|
 |executorsTotal| Integer |Number of executors used for this topology|
 |msgTimeout| Integer | Number of seconds a tuple has before the spout considers it failed |
 |windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"|
+|schedulerDisplayResource| Boolean | Whether to display scheduler resource information|
 |topologyStats| Array | Array of all the topology related stats per time window|
 |topologyStats.windowPretty| String |Duration passed in HH:MM:SS format|
 |topologyStats.window| String |User requested time window for metrics|
@@ -286,8 +373,10 @@
     "tasksTotal": 28,
     "executorsTotal": 28,
     "uptime": "29m 19s",
+    "uptimeSeconds": 1759,
     "msgTimeout": 30,
     "windowHint": "10m 0s",
+    "schedulerDisplayResource": true,
     "topologyStats": [
         {
             "windowPretty": "10m 0s",
@@ -436,7 +525,7 @@
 |windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"|
 |executors| Integer |Number of executor tasks in the component|
 |componentErrors| Array of Errors | List of component errors|
-|componentErrors.time| Long | Timestamp when the exception occurred |
+|componentErrors.errorTime| Long | Timestamp when the exception occurred (Prior to 0.11.0, this field was named 'time'.)|
 |componentErrors.errorHost| String | host name for the error|
 |componentErrors.errorPort| String | port for the error|
 |componentErrors.error| String |Shows the error happened in a component|
@@ -477,7 +566,7 @@
     "componentType": "spout",
     "windowHint": "10m 0s",
     "executors": 5,
-    "componentErrors":[{"time": 1406006074000,
+    "componentErrors":[{"errorTime": 1406006074000,
                         "errorHost": "10.11.1.70",
                         "errorPort": 6701,
                         "errorWorkerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log",
@@ -553,6 +642,7 @@
             "host": "10.11.1.7",
             "acked": 0,
             "uptime": "43m 4s",
+            "uptimeSeconds": 2584,
             "id": "[24-24]",
             "failed": 0
         },
@@ -565,6 +655,7 @@
             "host": "10.11.1.7",
             "acked": 0,
             "uptime": "42m 57s",
+            "uptimeSeconds": 2577,
             "id": "[25-25]",
             "failed": 0
         },
@@ -577,6 +668,7 @@
             "host": "10.11.1.7",
             "acked": 0,
             "uptime": "42m 57s",
+            "uptimeSeconds": 2577,
             "id": "[26-26]",
             "failed": 0
         },
@@ -589,6 +681,7 @@
             "host": "10.11.1.7",
             "acked": 0,
             "uptime": "43m 4s",
+            "uptimeSeconds": 2584,
             "id": "[27-27]",
             "failed": 0
         },
@@ -601,6 +694,7 @@
             "host": "10.11.1.7",
             "acked": 0,
             "uptime": "42m 57s",
+            "uptimeSeconds": 2577,
             "id": "[28-28]",
             "failed": 0
         }
@@ -608,6 +702,201 @@
 }
 ```
 
+## Profiling and Debugging GET Operations
+
+###  /api/v1/topology/:id/profiling/start/:host-port/:timeout (GET)
+
+Request to start profiler on worker with timeout. Returns status and link to profiler artifacts for worker.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+|timeout |String (required)| Time out for profiler to stop in minutes |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+|timeout | String | Requested timeout
+|dumplink | String | Link to logviewer URL for worker profiler documents.|
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/10
+2. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/5
+3. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/20
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+   "timeout": "10",
+   "dumplink": "http:\/\/10.11.1.7:8000\/dumps\/wordcount-1-1446614150\/10.11.1.7%3A6701"
+}
+```
+
+###  /api/v1/topology/:id/profiling/dumpprofile/:host-port (GET)
+
+Request to dump profiler recording on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpprofile/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/stop/:host-port (GET)
+
+Request to stop profiler on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/stop/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/dumpjstack/:host-port (GET)
+
+Request to dump jstack on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpjstack/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/dumpheap/:host-port (GET)
+
+Request to dump heap (jmap) on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpheap/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/restartworker/:host-port (GET)
+
+Request to request the worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/restartworker/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
 ## POST Operations
 
 ### /api/v1/topology/:id/activate (POST)
diff --git a/docs/images/viewing_metrics_with_VisualVM.png b/docs/images/viewing_metrics_with_VisualVM.png
new file mode 100644
index 0000000..5ccd81f
--- /dev/null
+++ b/docs/images/viewing_metrics_with_VisualVM.png
Binary files differ
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index b24c436..ac6472e 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -33,6 +33,10 @@
 
   <dependencies>
     <dependency>
+      <groupId>org.hdrhistogram</groupId>
+      <artifactId>HdrHistogram</artifactId>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -74,6 +78,13 @@
     </dependency>
       <dependency>
           <groupId>org.apache.storm</groupId>
+          <artifactId>storm-core</artifactId>
+          <version>${project.version}</version>
+          <type>test-jar</type>
+          <scope>test</scope>
+      </dependency>
+      <dependency>
+          <groupId>org.apache.storm</groupId>
           <artifactId>multilang-javascript</artifactId>
           <version>${project.version}</version>
       </dependency>
@@ -98,6 +109,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.storm</groupId>
+      <artifactId>storm-metrics</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.storm</groupId>
       <artifactId>storm-kafka</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
@@ -142,7 +158,6 @@
             <artifactId>maven-shade-plugin</artifactId>
             <configuration>
                 <createDependencyReducedPom>true</createDependencyReducedPom>
-                <minimizeJar>true</minimizeJar>
             </configuration>
             <executions>
                 <execution>
diff --git a/examples/storm-starter/src/jvm/storm/starter/FastWordCountTopology.java b/examples/storm-starter/src/jvm/storm/starter/FastWordCountTopology.java
new file mode 100644
index 0000000..8f78abd
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/FastWordCountTopology.java
@@ -0,0 +1,198 @@
+/**
+ * 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 storm.starter;
+
+import backtype.storm.Config;
+import backtype.storm.StormSubmitter;
+import backtype.storm.generated.*;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.topology.base.BaseBasicBolt;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.NimbusClient;
+import backtype.storm.utils.Utils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * WordCount but teh spout does not stop, and the bolts are implemented in
+ * java.  This can show how fast the word count can run.
+ */
+public class FastWordCountTopology {
+  public static class FastRandomSentenceSpout extends BaseRichSpout {
+    SpoutOutputCollector _collector;
+    Random _rand;
+    private static final String[] CHOICES = {
+        "marry had a little lamb whos fleese was white as snow",
+        "and every where that marry went the lamb was sure to go",
+        "one two three four five six seven eight nine ten",
+        "this is a test of the emergency broadcast system this is only a test",
+        "peter piper picked a peck of pickeled peppers"
+    };
+
+    @Override
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+      _collector = collector;
+      _rand = ThreadLocalRandom.current();
+    }
+
+    @Override
+    public void nextTuple() {
+      String sentence = CHOICES[_rand.nextInt(CHOICES.length)];
+      _collector.emit(new Values(sentence), sentence);
+    }
+
+    @Override
+    public void ack(Object id) {
+        //Ignored
+    }
+
+    @Override
+    public void fail(Object id) {
+      _collector.emit(new Values(id), id);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("sentence"));
+    }
+  }
+
+  public static class SplitSentence extends BaseBasicBolt {
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+      String sentence = tuple.getString(0);
+      for (String word: sentence.split("\\s+")) {
+          collector.emit(new Values(word, 1));
+      }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("word", "count"));
+    }
+  }
+
+  public static class WordCount extends BaseBasicBolt {
+    Map<String, Integer> counts = new HashMap<String, Integer>();
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+      String word = tuple.getString(0);
+      Integer count = counts.get(word);
+      if (count == null)
+        count = 0;
+      count++;
+      counts.put(word, count);
+      collector.emit(new Values(word, count));
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("word", "count"));
+    }
+  }
+
+  public static void printMetrics(Nimbus.Client client, String name) throws Exception {
+    ClusterSummary summary = client.getClusterInfo();
+    String id = null;
+    for (TopologySummary ts: summary.get_topologies()) {
+      if (name.equals(ts.get_name())) {
+        id = ts.get_id();
+      }
+    }
+    if (id == null) {
+      throw new Exception("Could not find a topology named "+name);
+    }
+    TopologyInfo info = client.getTopologyInfo(id);
+    int uptime = info.get_uptime_secs();
+    long acked = 0;
+    long failed = 0;
+    double weightedAvgTotal = 0.0;
+    for (ExecutorSummary exec: info.get_executors()) {
+      if ("spout".equals(exec.get_component_id())) {
+        SpoutStats stats = exec.get_stats().get_specific().get_spout();
+        Map<String, Long> failedMap = stats.get_failed().get(":all-time");
+        Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
+        Map<String, Double> avgLatMap = stats.get_complete_ms_avg().get(":all-time");
+        for (String key: ackedMap.keySet()) {
+          if (failedMap != null) {
+              Long tmp = failedMap.get(key);
+              if (tmp != null) {
+                  failed += tmp;
+              }
+          }
+          long ackVal = ackedMap.get(key);
+          double latVal = avgLatMap.get(key) * ackVal;
+          acked += ackVal;
+          weightedAvgTotal += latVal;
+        }
+      }
+    }
+    double avgLatency = weightedAvgTotal/acked;
+    System.out.println("uptime: "+uptime+" acked: "+acked+" avgLatency: "+avgLatency+" acked/sec: "+(((double)acked)/uptime+" failed: "+failed));
+  } 
+
+  public static void kill(Nimbus.Client client, String name) throws Exception {
+    KillOptions opts = new KillOptions();
+    opts.set_wait_secs(0);
+    client.killTopologyWithOpts(name, opts);
+  } 
+
+  public static void main(String[] args) throws Exception {
+
+    TopologyBuilder builder = new TopologyBuilder();
+
+    builder.setSpout("spout", new FastRandomSentenceSpout(), 4);
+
+    builder.setBolt("split", new SplitSentence(), 4).shuffleGrouping("spout");
+    builder.setBolt("count", new WordCount(), 4).fieldsGrouping("split", new Fields("word"));
+
+    Config conf = new Config();
+    conf.registerMetricsConsumer(backtype.storm.metric.LoggingMetricsConsumer.class);
+
+    String name = "wc-test";
+    if (args != null && args.length > 0) {
+        name = args[0];
+    }
+
+    conf.setNumWorkers(1);
+    StormSubmitter.submitTopologyWithProgressBar(name, conf, builder.createTopology());
+
+    Map clusterConf = Utils.readStormConfig();
+    clusterConf.putAll(Utils.readCommandLineOpts());
+    Nimbus.Client client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+
+    //Sleep for 5 mins
+    for (int i = 0; i < 10; i++) {
+        Thread.sleep(30 * 1000);
+        printMetrics(client, name);
+    }
+    kill(client, name);
+  }
+}
diff --git a/examples/storm-starter/src/jvm/storm/starter/InOrderDeliveryTest.java b/examples/storm-starter/src/jvm/storm/starter/InOrderDeliveryTest.java
new file mode 100644
index 0000000..5df0688
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/InOrderDeliveryTest.java
@@ -0,0 +1,175 @@
+/**
+ * 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 storm.starter;
+
+import backtype.storm.Config;
+import backtype.storm.StormSubmitter;
+import backtype.storm.generated.*;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.FailedException;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.topology.base.BaseBasicBolt;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.NimbusClient;
+import backtype.storm.utils.Utils;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+public class InOrderDeliveryTest {
+  public static class InOrderSpout extends BaseRichSpout {
+    SpoutOutputCollector _collector;
+    int _base = 0;
+    int _i = 0;
+
+    @Override
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+      _collector = collector;
+      _base = context.getThisTaskIndex();
+    }
+
+    @Override
+    public void nextTuple() {
+      Values v = new Values(_base, _i);
+      _collector.emit(v, "ACK");
+      _i++;
+    }
+
+    @Override
+    public void ack(Object id) {
+      //Ignored
+    }
+
+    @Override
+    public void fail(Object id) {
+      //Ignored
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("c1", "c2"));
+    }
+  }
+
+  public static class Check extends BaseBasicBolt {
+    Map<Integer, Integer> expected = new HashMap<Integer, Integer>();
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+      Integer c1 = tuple.getInteger(0);
+      Integer c2 = tuple.getInteger(1);
+      Integer exp = expected.get(c1);
+      if (exp == null) exp = 0;
+      if (c2.intValue() != exp.intValue()) {
+          System.out.println(c1+" "+c2+" != "+exp);
+          throw new FailedException(c1+" "+c2+" != "+exp);
+      }
+      exp = c2 + 1;
+      expected.put(c1, exp);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      //Empty
+    }
+  }
+
+  public static void printMetrics(Nimbus.Client client, String name) throws Exception {
+    ClusterSummary summary = client.getClusterInfo();
+    String id = null;
+    for (TopologySummary ts: summary.get_topologies()) {
+      if (name.equals(ts.get_name())) {
+        id = ts.get_id();
+      }
+    }
+    if (id == null) {
+      throw new Exception("Could not find a topology named "+name);
+    }
+    TopologyInfo info = client.getTopologyInfo(id);
+    int uptime = info.get_uptime_secs();
+    long acked = 0;
+    long failed = 0;
+    double weightedAvgTotal = 0.0;
+    for (ExecutorSummary exec: info.get_executors()) {
+      if ("spout".equals(exec.get_component_id())) {
+        SpoutStats stats = exec.get_stats().get_specific().get_spout();
+        Map<String, Long> failedMap = stats.get_failed().get(":all-time");
+        Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
+        Map<String, Double> avgLatMap = stats.get_complete_ms_avg().get(":all-time");
+        for (String key: ackedMap.keySet()) {
+          if (failedMap != null) {
+              Long tmp = failedMap.get(key);
+              if (tmp != null) {
+                  failed += tmp;
+              }
+          }
+          long ackVal = ackedMap.get(key);
+          double latVal = avgLatMap.get(key) * ackVal;
+          acked += ackVal;
+          weightedAvgTotal += latVal;
+        }
+      }
+    }
+    double avgLatency = weightedAvgTotal/acked;
+    System.out.println("uptime: "+uptime+" acked: "+acked+" avgLatency: "+avgLatency+" acked/sec: "+(((double)acked)/uptime+" failed: "+failed));
+  } 
+
+  public static void kill(Nimbus.Client client, String name) throws Exception {
+    KillOptions opts = new KillOptions();
+    opts.set_wait_secs(0);
+    client.killTopologyWithOpts(name, opts);
+  } 
+
+  public static void main(String[] args) throws Exception {
+
+    TopologyBuilder builder = new TopologyBuilder();
+
+    builder.setSpout("spout", new InOrderSpout(), 8);
+    builder.setBolt("count", new Check(), 8).fieldsGrouping("spout", new Fields("c1"));
+
+    Config conf = new Config();
+    conf.registerMetricsConsumer(backtype.storm.metric.LoggingMetricsConsumer.class);
+
+    String name = "in-order-test";
+    if (args != null && args.length > 0) {
+        name = args[0];
+    }
+
+    conf.setNumWorkers(1);
+    StormSubmitter.submitTopologyWithProgressBar(name, conf, builder.createTopology());
+
+    Map clusterConf = Utils.readStormConfig();
+    clusterConf.putAll(Utils.readCommandLineOpts());
+    Nimbus.Client client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+
+    //Sleep for 50 mins
+    for (int i = 0; i < 50; i++) {
+        Thread.sleep(30 * 1000);
+        printMetrics(client, name);
+    }
+    kill(client, name);
+  }
+}
diff --git a/examples/storm-starter/src/jvm/storm/starter/MultipleLoggerTopology.java b/examples/storm-starter/src/jvm/storm/starter/MultipleLoggerTopology.java
new file mode 100644
index 0000000..4285ff9
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/MultipleLoggerTopology.java
@@ -0,0 +1,105 @@
+/**
+ * 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 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.OutputFieldsDeclarer;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This is a basic example of a Storm topology.
+ */
+public class MultipleLoggerTopology {
+  public static class ExclamationLoggingBolt extends BaseRichBolt {
+    OutputCollector _collector;
+    Logger _rootLogger = LoggerFactory.getLogger (Logger.ROOT_LOGGER_NAME);
+    // ensure the loggers are configured in the worker.xml before
+    // trying to use them here
+    Logger _logger = LoggerFactory.getLogger ("com.myapp");
+    Logger _subLogger = LoggerFactory.getLogger ("com.myapp.sub");
+
+    @Override
+    public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+      _collector = collector;
+    }
+
+    @Override
+    public void execute(Tuple tuple) {
+      _rootLogger.debug ("root: This is a DEBUG message");
+      _rootLogger.info ("root: This is an INFO message");
+      _rootLogger.warn ("root: This is a WARN message");
+      _rootLogger.error ("root: This is an ERROR message");
+
+      _logger.debug ("myapp: This is a DEBUG message");
+      _logger.info ("myapp: This is an INFO message");
+      _logger.warn ("myapp: This is a WARN message");
+      _logger.error ("myapp: This is an ERROR message");
+
+      _subLogger.debug ("myapp.sub: This is a DEBUG message");
+      _subLogger.info ("myapp.sub: This is an INFO message");
+      _subLogger.warn ("myapp.sub: This is a WARN message");
+      _subLogger.error ("myapp.sub: This is an ERROR message");
+
+      _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();
+
+    builder.setSpout("word", new TestWordSpout(), 10);
+    builder.setBolt("exclaim1", new ExclamationLoggingBolt(), 3).shuffleGrouping("word");
+    builder.setBolt("exclaim2", new ExclamationLoggingBolt(), 2).shuffleGrouping("exclaim1");
+
+    Config conf = new Config();
+    conf.setDebug(true);
+
+    if (args != null && args.length > 0) {
+      conf.setNumWorkers(2);
+      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();
+    }
+  }
+}
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..e8225d4
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/ResourceAwareExampleTopology.java
@@ -0,0 +1,101 @@
+/**
+ * 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 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;
+
+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(1024.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();
+    }
+  }
+}
diff --git a/examples/storm-starter/src/jvm/storm/starter/SlidingWindowTopology.java b/examples/storm-starter/src/jvm/storm/starter/SlidingWindowTopology.java
new file mode 100644
index 0000000..beee8de
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/SlidingWindowTopology.java
@@ -0,0 +1,185 @@
+/**
+ * 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 storm.starter;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.topology.base.BaseWindowedBolt;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
+import backtype.storm.windowing.TupleWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import storm.starter.bolt.PrinterBolt;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import static backtype.storm.topology.base.BaseWindowedBolt.Count;
+
+/**
+ * A sample topology that demonstrates the usage of {@link backtype.storm.topology.IWindowedBolt}
+ * to calculate sliding window sum.
+ */
+public class SlidingWindowTopology {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SlidingWindowTopology.class);
+
+    /*
+     * emits a random integer every 100 ms
+     */
+
+    private static class RandomIntegerSpout extends BaseRichSpout {
+        SpoutOutputCollector collector;
+        Random rand;
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("value"));
+        }
+
+        @Override
+        public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+            this.collector = collector;
+            this.rand = new Random();
+        }
+
+        @Override
+        public void nextTuple() {
+            Utils.sleep(100);
+            collector.emit(new Values(rand.nextInt(1000)));
+        }
+    }
+
+    /*
+     * Computes sliding window sum
+     */
+    private static class SlidingWindowSumBolt extends BaseWindowedBolt {
+        private int sum = 0;
+        private OutputCollector collector;
+
+        @Override
+        public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+            this.collector = collector;
+        }
+
+        @Override
+        public void execute(TupleWindow inputWindow) {
+            /*
+             * The inputWindow gives a view of
+             * (a) all the events in the window
+             * (b) events that expired since last activation of the window
+             * (c) events that newly arrived since last activation of the window
+             */
+            List<Tuple> tuplesInWindow = inputWindow.get();
+            List<Tuple> newTuples = inputWindow.getNew();
+            List<Tuple> expiredTuples = inputWindow.getExpired();
+
+            LOG.debug("Events in current window: " + tuplesInWindow.size());
+            /*
+             * Instead of iterating over all the tuples in the window to compute
+             * the sum, the values for the new events are added and old events are
+             * subtracted. Similar optimizations might be possible in other
+             * windowing computations.
+             */
+            for (Tuple tuple : newTuples) {
+                sum += (int) tuple.getValue(0);
+            }
+            for (Tuple tuple : expiredTuples) {
+                sum -= (int) tuple.getValue(0);
+            }
+            collector.emit(new Values(sum));
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("sum"));
+        }
+    }
+
+
+    /*
+     * Computes tumbling window average
+     */
+    private static class TumblingWindowAvgBolt extends BaseWindowedBolt {
+        private OutputCollector collector;
+
+        @Override
+        public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+            this.collector = collector;
+        }
+
+        @Override
+        public void execute(TupleWindow inputWindow) {
+            int sum = 0;
+            List<Tuple> tuplesInWindow = inputWindow.get();
+            LOG.debug("Events in current window: " + tuplesInWindow.size());
+            if (tuplesInWindow.size() > 0) {
+                /*
+                * Since this is a tumbling window calculation,
+                * we use all the tuples in the window to compute the avg.
+                */
+                for (Tuple tuple : tuplesInWindow) {
+                    sum += (int) tuple.getValue(0);
+                }
+                collector.emit(new Values(sum / tuplesInWindow.size()));
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("avg"));
+        }
+    }
+
+
+    public static void main(String[] args) throws Exception {
+        TopologyBuilder builder = new TopologyBuilder();
+        builder.setSpout("integer", new RandomIntegerSpout(), 1);
+        builder.setBolt("slidingsum", new SlidingWindowSumBolt().withWindow(new Count(30), new Count(10)), 1)
+                .shuffleGrouping("integer");
+        builder.setBolt("tumblingavg", new TumblingWindowAvgBolt().withTumblingWindow(new Count(3)), 1)
+                .shuffleGrouping("slidingsum");
+        builder.setBolt("printer", new PrinterBolt(), 1).shuffleGrouping("tumblingavg");
+        Config conf = new Config();
+        conf.setDebug(true);
+
+        if (args != null && args.length > 0) {
+            conf.setNumWorkers(1);
+
+            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+        } else {
+
+            LocalCluster cluster = new LocalCluster();
+            cluster.submitTopology("test", conf, builder.createTopology());
+            Utils.sleep(40000);
+            cluster.killTopology("test");
+            cluster.shutdown();
+        }
+    }
+}
diff --git a/examples/storm-starter/src/jvm/storm/starter/ThroughputVsLatency.java b/examples/storm-starter/src/jvm/storm/starter/ThroughputVsLatency.java
new file mode 100644
index 0000000..4c6680e
--- /dev/null
+++ b/examples/storm-starter/src/jvm/storm/starter/ThroughputVsLatency.java
@@ -0,0 +1,432 @@
+/**
+ * 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 storm.starter;
+
+import backtype.storm.Config;
+import backtype.storm.LocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.metric.HttpForwardingMetricsServer;
+import backtype.storm.metric.HttpForwardingMetricsConsumer;
+import backtype.storm.metric.api.IMetric;
+import backtype.storm.metric.api.IMetricsConsumer.TaskInfo;
+import backtype.storm.metric.api.IMetricsConsumer.DataPoint;
+import backtype.storm.generated.*;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.topology.base.BaseBasicBolt;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.NimbusClient;
+import backtype.storm.utils.Utils;
+import backtype.storm.StormSubmitter;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.storm.metrics.hdrhistogram.HistogramMetric;
+import org.HdrHistogram.Histogram;
+
+/**
+ * WordCount but the spout goes at a predefined rate and we collect
+ * proper latency statistics.
+ */
+public class ThroughputVsLatency {
+  private static class SentWithTime {
+    public final String sentence;
+    public final long time;
+
+    SentWithTime(String sentence, long time) {
+        this.sentence = sentence;
+        this.time = time;
+    }
+  }
+
+  public static class C {
+    LocalCluster _local = null;
+    Nimbus.Client _client = null;
+
+    public C(Map conf) {
+      Map clusterConf = Utils.readStormConfig();
+      if (conf != null) {
+        clusterConf.putAll(conf);
+      }
+      Boolean isLocal = (Boolean)clusterConf.get("run.local");
+      if (isLocal != null && isLocal) {
+        _local = new LocalCluster();
+      } else {
+        _client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+      }
+    }
+
+    public ClusterSummary getClusterInfo() throws Exception {
+      if (_local != null) {
+        return _local.getClusterInfo();
+      } else {
+        return _client.getClusterInfo();
+      }
+    }
+
+    public TopologyInfo getTopologyInfo(String id) throws Exception {
+      if (_local != null) {
+        return _local.getTopologyInfo(id);
+      } else {
+        return _client.getTopologyInfo(id);
+      }
+    }
+
+    public void killTopologyWithOpts(String name, KillOptions opts) throws Exception {
+      if (_local != null) {
+        _local.killTopologyWithOpts(name, opts);
+      } else {
+        _client.killTopologyWithOpts(name, opts);
+      }
+    }
+
+    public void submitTopology(String name, Map stormConf, StormTopology topology) throws Exception {
+      if (_local != null) {
+        _local.submitTopology(name, stormConf, topology);
+      } else {
+        StormSubmitter.submitTopology(name, stormConf, topology);
+      }
+    }
+
+    public boolean isLocal() {
+      return _local != null;
+    }
+  }
+
+  public static class FastRandomSentenceSpout extends BaseRichSpout {
+    static final String[] SENTENCES = new String[]{ "the cow jumped over the moon", "an apple a day keeps the doctor away",
+          "four score and seven years ago", "snow white and the seven dwarfs", "i am at two with nature" };
+
+    SpoutOutputCollector _collector;
+    long _periodNano;
+    long _emitAmount;
+    Random _rand;
+    long _nextEmitTime;
+    long _emitsLeft;
+    HistogramMetric _histo;
+
+    public FastRandomSentenceSpout(long ratePerSecond) {
+        if (ratePerSecond > 0) {
+            _periodNano = Math.max(1, 1000000000/ratePerSecond);
+            _emitAmount = Math.max(1, (long)((ratePerSecond / 1000000000.0) * _periodNano));
+        } else {
+            _periodNano = Long.MAX_VALUE - 1;
+            _emitAmount = 1;
+        }
+    }
+
+    @Override
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+      _collector = collector;
+      _rand = ThreadLocalRandom.current();
+      _nextEmitTime = System.nanoTime();
+      _emitsLeft = _emitAmount;
+      _histo = new HistogramMetric(3600000000000L, 3);
+      context.registerMetric("comp-lat-histo", _histo, 10); //Update every 10 seconds, so we are not too far behind
+    }
+
+    @Override
+    public void nextTuple() {
+      if (_emitsLeft <= 0 && _nextEmitTime <= System.nanoTime()) {
+          _emitsLeft = _emitAmount;
+          _nextEmitTime = _nextEmitTime + _periodNano;
+      }
+
+      if (_emitsLeft > 0) {
+          String sentence = SENTENCES[_rand.nextInt(SENTENCES.length)];
+          _collector.emit(new Values(sentence), new SentWithTime(sentence, _nextEmitTime - _periodNano));
+          _emitsLeft--;
+      }
+    }
+
+    @Override
+    public void ack(Object id) {
+      long end = System.nanoTime();
+      SentWithTime st = (SentWithTime)id;
+      _histo.recordValue(end-st.time);
+    }
+
+    @Override
+    public void fail(Object id) {
+      SentWithTime st = (SentWithTime)id;
+      _collector.emit(new Values(st.sentence), id);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("sentence"));
+    }
+  }
+
+  public static class SplitSentence extends BaseBasicBolt {
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+      String sentence = tuple.getString(0);
+      for (String word: sentence.split("\\s+")) {
+          collector.emit(new Values(word, 1));
+      }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("word", "count"));
+    }
+  }
+
+  public static class WordCount extends BaseBasicBolt {
+    Map<String, Integer> counts = new HashMap<String, Integer>();
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+      String word = tuple.getString(0);
+      Integer count = counts.get(word);
+      if (count == null)
+        count = 0;
+      count++;
+      counts.put(word, count);
+      collector.emit(new Values(word, count));
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("word", "count"));
+    }
+  }
+
+  private static class MemMeasure {
+    private long _mem = 0;
+    private long _time = 0;
+
+    public synchronized void update(long mem) {
+        _mem = mem;
+        _time = System.currentTimeMillis();
+    }
+
+    public synchronized long get() {
+        return isExpired() ? 0l : _mem;
+    }
+
+    public synchronized boolean isExpired() {
+        return (System.currentTimeMillis() - _time) >= 20000;
+    }
+  }
+
+  private static final Histogram _histo = new Histogram(3600000000000L, 3);
+  private static final AtomicLong _systemCPU = new AtomicLong(0);
+  private static final AtomicLong _userCPU = new AtomicLong(0);
+  private static final AtomicLong _gcCount = new AtomicLong(0);
+  private static final AtomicLong _gcMs = new AtomicLong(0);
+  private static final ConcurrentHashMap<String, MemMeasure> _memoryBytes = new ConcurrentHashMap<String, MemMeasure>();
+
+  private static long readMemory() {
+    long total = 0;
+    for (MemMeasure mem: _memoryBytes.values()) {
+      total += mem.get();
+    }
+    return total;
+  }
+
+  private static long _prev_acked = 0;
+  private static long _prev_uptime = 0;
+
+  public static void printMetrics(C client, String name) throws Exception {
+    ClusterSummary summary = client.getClusterInfo();
+    String id = null;
+    for (TopologySummary ts: summary.get_topologies()) {
+      if (name.equals(ts.get_name())) {
+        id = ts.get_id();
+      }
+    }
+    if (id == null) {
+      throw new Exception("Could not find a topology named "+name);
+    }
+    TopologyInfo info = client.getTopologyInfo(id);
+    int uptime = info.get_uptime_secs();
+    long acked = 0;
+    long failed = 0;
+    for (ExecutorSummary exec: info.get_executors()) {
+      if ("spout".equals(exec.get_component_id())) {
+        SpoutStats stats = exec.get_stats().get_specific().get_spout();
+        Map<String, Long> failedMap = stats.get_failed().get(":all-time");
+        Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
+        if (ackedMap != null) {
+          for (String key: ackedMap.keySet()) {
+            if (failedMap != null) {
+              Long tmp = failedMap.get(key);
+              if (tmp != null) {
+                  failed += tmp;
+              }
+            }
+            long ackVal = ackedMap.get(key);
+            acked += ackVal;
+          }
+        }
+      }
+    }
+    long ackedThisTime = acked - _prev_acked;
+    long thisTime = uptime - _prev_uptime;
+    long nnpct, nnnpct, min, max;
+    double mean, stddev;
+    synchronized(_histo) {
+      nnpct = _histo.getValueAtPercentile(99.0);
+      nnnpct = _histo.getValueAtPercentile(99.9);
+      min = _histo.getMinValue();
+      max = _histo.getMaxValue();
+      mean = _histo.getMean();
+      stddev = _histo.getStdDeviation();
+      _histo.reset();
+    }
+    long user = _userCPU.getAndSet(0);
+    long sys = _systemCPU.getAndSet(0);
+    long gc = _gcMs.getAndSet(0);
+    double memMB = readMemory() / (1024.0 * 1024.0);
+    System.out.printf("uptime: %,4d acked: %,9d acked/sec: %,10.2f failed: %,8d " +
+                      "99%%: %,15d 99.9%%: %,15d min: %,15d max: %,15d mean: %,15.2f " +
+                      "stddev: %,15.2f user: %,10d sys: %,10d gc: %,10d mem: %,10.2f\n",
+                       uptime, ackedThisTime, (((double)ackedThisTime)/thisTime), failed, nnpct, nnnpct,
+                       min, max, mean, stddev, user, sys, gc, memMB);
+    _prev_uptime = uptime;
+    _prev_acked = acked;
+  }
+
+  public static void kill(C client, String name) throws Exception {
+    KillOptions opts = new KillOptions();
+    opts.set_wait_secs(0);
+    client.killTopologyWithOpts(name, opts);
+  }
+
+  public static void main(String[] args) throws Exception {
+    long ratePerSecond = 500;
+    if (args != null && args.length > 0) {
+        ratePerSecond = Long.valueOf(args[0]);
+    }
+
+    int parallelism = 4;
+    if (args != null && args.length > 1) {
+        parallelism = Integer.valueOf(args[1]);
+    }
+
+    int numMins = 5;
+    if (args != null && args.length > 2) {
+        numMins = Integer.valueOf(args[2]);
+    }
+
+    String name = "wc-test";
+    if (args != null && args.length > 3) {
+        name = args[3];
+    }
+
+    Config conf = new Config();
+    HttpForwardingMetricsServer metricServer = new HttpForwardingMetricsServer(conf) {
+        @Override
+        public void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
+            String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
+            for (DataPoint dp: dataPoints) {
+                if ("comp-lat-histo".equals(dp.name) && dp.value instanceof Histogram) {
+                    synchronized(_histo) {
+                        _histo.add((Histogram)dp.value);
+                    }
+                } else if ("CPU".equals(dp.name) && dp.value instanceof Map) {
+                   Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                   Object sys = m.get("sys-ms");
+                   if (sys instanceof Number) {
+                       _systemCPU.getAndAdd(((Number)sys).longValue());
+                   }
+                   Object user = m.get("user-ms");
+                   if (user instanceof Number) {
+                       _userCPU.getAndAdd(((Number)user).longValue());
+                   }
+                } else if (dp.name.startsWith("GC/") && dp.value instanceof Map) {
+                   Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                   Object count = m.get("count");
+                   if (count instanceof Number) {
+                       _gcCount.getAndAdd(((Number)count).longValue());
+                   }
+                   Object time = m.get("timeMs");
+                   if (time instanceof Number) {
+                       _gcMs.getAndAdd(((Number)time).longValue());
+                   }
+                } else if (dp.name.startsWith("memory/") && dp.value instanceof Map) {
+                   Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                   Object val = m.get("usedBytes");
+                   if (val instanceof Number) {
+                       MemMeasure mm = _memoryBytes.get(worker);
+                       if (mm == null) {
+                         mm = new MemMeasure();
+                         MemMeasure tmp = _memoryBytes.putIfAbsent(worker, mm);
+                         mm = tmp == null ? mm : tmp; 
+                       }
+                       mm.update(((Number)val).longValue());
+                   }
+                }
+            }
+        }
+    };
+
+    metricServer.serve();
+    String url = metricServer.getUrl();
+
+    C cluster = new C(conf);
+    conf.setNumWorkers(parallelism);
+    conf.registerMetricsConsumer(backtype.storm.metric.LoggingMetricsConsumer.class);
+    conf.registerMetricsConsumer(backtype.storm.metric.HttpForwardingMetricsConsumer.class, url, 1);
+    Map<String, String> workerMetrics = new HashMap<String, String>();
+    if (!cluster.isLocal()) {
+      //sigar uses JNI and does not work in local mode
+      workerMetrics.put("CPU", "org.apache.storm.metrics.sigar.CPUMetric");
+    }
+    conf.put(Config.TOPOLOGY_WORKER_METRICS, workerMetrics);
+    conf.put(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS, 10);
+    conf.put(Config.TOPOLOGY_WORKER_GC_CHILDOPTS,
+      "-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:NewSize=128m -XX:CMSInitiatingOccupancyFraction=70 -XX:-CMSConcurrentMTEnabled");
+    conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, "-Xmx2g");
+
+    TopologyBuilder builder = new TopologyBuilder();
+
+    int numEach = 4 * parallelism;
+    builder.setSpout("spout", new FastRandomSentenceSpout(ratePerSecond/numEach), numEach);
+
+    builder.setBolt("split", new SplitSentence(), numEach).shuffleGrouping("spout");
+    builder.setBolt("count", new WordCount(), numEach).fieldsGrouping("split", new Fields("word"));
+
+    try {
+        cluster.submitTopology(name, conf, builder.createTopology());
+
+        for (int i = 0; i < numMins * 2; i++) {
+            Thread.sleep(30 * 1000);
+            printMetrics(cluster, name);
+        }
+    } finally {
+        kill(cluster, name);
+    }
+    System.exit(0);
+  }
+}
diff --git a/examples/storm-starter/src/jvm/storm/starter/trident/TridentKafkaWordCount.java b/examples/storm-starter/src/jvm/storm/starter/trident/TridentKafkaWordCount.java
index 813841a..bd8ecba 100644
--- a/examples/storm-starter/src/jvm/storm/starter/trident/TridentKafkaWordCount.java
+++ b/examples/storm-starter/src/jvm/storm/starter/trident/TridentKafkaWordCount.java
@@ -149,14 +149,14 @@
      *
      * @return the storm topology
      */
-    public StormTopology buildProducerTopology() {
+    public StormTopology buildProducerTopology(Properties prop) {
         TopologyBuilder builder = new TopologyBuilder();
         builder.setSpout("spout", new RandomSentenceSpout(), 2);
         /**
          * The output field of the RandomSentenceSpout ("word") is provided as the boltMessageField
          * so that this gets written out as the message in the kafka topic.
          */
-        KafkaBolt bolt = new KafkaBolt()
+        KafkaBolt bolt = new KafkaBolt().withProducerProperties(prop)
                 .withTopicSelector(new DefaultTopicSelector("test"))
                 .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("key", "word"));
         builder.setBolt("forwardToKafka", bolt, 1).shuffleGrouping("spout");
@@ -169,16 +169,13 @@
      *
      * @return the topology config
      */
-    public Config getProducerConfig() {
-        Config conf = new Config();
-        conf.setMaxSpoutPending(20);
+    public Properties getProducerConfig() {
         Properties props = new Properties();
         props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl);
         props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
         props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer");
         props.put(ProducerConfig.CLIENT_ID_CONFIG, "storm-kafka-producer");
-        conf.put(KafkaBolt.KAFKA_BROKER_PROPERTIES, props);
-        return conf;
+        return props;
     }
 
     /**
@@ -214,8 +211,10 @@
         // submit the consumer topology.
         cluster.submitTopology("wordCounter", wordCount.getConsumerConfig(), wordCount.buildConsumerTopology(drpc));
 
+        Config conf = new Config();
+        conf.setMaxSpoutPending(20);
         // submit the producer topology.
-        cluster.submitTopology("kafkaBolt", wordCount.getProducerConfig(), wordCount.buildProducerTopology());
+        cluster.submitTopology("kafkaBolt", conf, wordCount.buildProducerTopology(wordCount.getProducerConfig()));
 
         // keep querying the word counts for a minute.
         for (int i = 0; i < 60; i++) {
diff --git a/examples/storm-starter/test/jvm/storm/starter/bolt/IntermediateRankingsBoltTest.java b/examples/storm-starter/test/jvm/storm/starter/bolt/IntermediateRankingsBoltTest.java
index c296a89..278a513 100644
--- a/examples/storm-starter/test/jvm/storm/starter/bolt/IntermediateRankingsBoltTest.java
+++ b/examples/storm-starter/test/jvm/storm/starter/bolt/IntermediateRankingsBoltTest.java
@@ -23,10 +23,10 @@
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
+import backtype.storm.utils.MockTupleHelpers;
 import com.google.common.collect.Lists;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
-import storm.starter.tools.MockTupleHelpers;
 
 import java.util.Map;
 
diff --git a/examples/storm-starter/test/jvm/storm/starter/bolt/RollingCountBoltTest.java b/examples/storm-starter/test/jvm/storm/starter/bolt/RollingCountBoltTest.java
index bc31ba0..ecb1216 100644
--- a/examples/storm-starter/test/jvm/storm/starter/bolt/RollingCountBoltTest.java
+++ b/examples/storm-starter/test/jvm/storm/starter/bolt/RollingCountBoltTest.java
@@ -24,8 +24,8 @@
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
+import backtype.storm.utils.MockTupleHelpers;
 import org.testng.annotations.Test;
-import storm.starter.tools.MockTupleHelpers;
 
 import java.util.Map;
 
diff --git a/examples/storm-starter/test/jvm/storm/starter/bolt/TotalRankingsBoltTest.java b/examples/storm-starter/test/jvm/storm/starter/bolt/TotalRankingsBoltTest.java
index 49e3d67..a6af931 100644
--- a/examples/storm-starter/test/jvm/storm/starter/bolt/TotalRankingsBoltTest.java
+++ b/examples/storm-starter/test/jvm/storm/starter/bolt/TotalRankingsBoltTest.java
@@ -23,9 +23,9 @@
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
+import backtype.storm.utils.MockTupleHelpers;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
-import storm.starter.tools.MockTupleHelpers;
 import storm.starter.tools.Rankings;
 
 import java.util.Map;
diff --git a/external/flux/README.md b/external/flux/README.md
index 2c5127e..c4ef145 100644
--- a/external/flux/README.md
+++ b/external/flux/README.md
@@ -365,6 +365,7 @@
     className: "storm.kafka.ZkHosts"
     constructorArgs:
       - "localhost:2181"
+      - true
 ```
 
 ####References
@@ -432,6 +433,8 @@
       - name: "withFoo"
         args:
           - "foo"
+      - name: "withNone"
+      # no args needed, so no "args" line
       - name: "withBar"
         args:
           - "bar"
@@ -445,6 +448,7 @@
 
 ```java
     public void withFoo(String foo);
+    public void withNone(); // method with zero arguments
     public void withBar(String bar);
     public void withFooBar(String foo, String bar);
 ```
diff --git a/external/flux/flux-core/pom.xml b/external/flux/flux-core/pom.xml
index 8d0c1f2..bd5a18f 100644
--- a/external/flux/flux-core/pom.xml
+++ b/external/flux/flux-core/pom.xml
@@ -72,7 +72,6 @@
             <artifactId>maven-shade-plugin</artifactId>
             <configuration>
                 <createDependencyReducedPom>true</createDependencyReducedPom>
-                <minimizeJar>true</minimizeJar>
             </configuration>
             <executions>
                 <execution>
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
index 6300631..71c20a7 100644
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
+++ b/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
@@ -253,7 +253,8 @@
         // banner
         InputStream is = Flux.class.getResourceAsStream("/splash.txt");
         if(is != null){
-            BufferedReader br = new BufferedReader(new InputStreamReader(is));
+            InputStreamReader isr = new InputStreamReader(is, "UTF-8");
+            BufferedReader br = new BufferedReader(isr);
             String line = null;
             while((line = br.readLine()) != null){
                 System.out.println(line);
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
index 9f15e6b..014116d 100644
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
+++ b/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
@@ -167,6 +167,14 @@
                             topologyDef.parallelismForBolt(stream.getTo()));
                     declarers.put(stream.getTo(), declarer);
                 }
+            } else if (boltObj instanceof IWindowedBolt) {
+                if(declarer == null) {
+                    declarer = builder.setBolt(
+                            stream.getTo(),
+                            (IWindowedBolt) boltObj,
+                            topologyDef.parallelismForBolt(stream.getTo()));
+                    declarers.put(stream.getTo(), declarer);
+                }
             } else {
                 throw new IllegalArgumentException("Class does not appear to be a bolt: " +
                         boltObj.getClass().getName());
@@ -417,6 +425,9 @@
         Class clazz = instance.getClass();
         for(ConfigMethodDef methodDef : methodDefs){
             List<Object> args = methodDef.getArgs();
+            if (args == null){
+                args = new ArrayList();
+            }
             if(methodDef.hasReferences()){
                 args = resolveReferences(args, context);
             }
@@ -444,7 +455,13 @@
             Class[] paramClasses = method.getParameterTypes();
             if (paramClasses.length == args.size() && method.getName().equals(methodName)) {
                 LOG.debug("found constructor with same number of args..");
-                boolean invokable = canInvokeWithArgs(args, method.getParameterTypes());
+                boolean invokable = false;
+                if (args.size() == 0){
+                    // it's a method with zero args
+                    invokable = true;
+                } else {
+                    invokable = canInvokeWithArgs(args, method.getParameterTypes());
+                }
                 if (invokable) {
                     retval = method;
                     eligibleCount++;
@@ -492,6 +509,12 @@
                 constructorParams[i] = args.get(i);
                 continue;
             }
+            if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
+                LOG.debug("Its a primitive boolean.");
+                Boolean bool = (Boolean)args.get(i);
+                constructorParams[i] = bool.booleanValue();
+                continue;
+            }
             if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
                 LOG.debug("Its a primitive number.");
                 Number num = (Number)args.get(i);
@@ -563,33 +586,31 @@
                     paramType, objectType);
             if (paramType.equals(objectType)) {
                 LOG.debug("Yes, they are the same class.");
-                return true;
-            }
-            if (paramType.isAssignableFrom(objectType)) {
+            } else if (paramType.isAssignableFrom(objectType)) {
                 LOG.debug("Yes, assignment is possible.");
-                return true;
-            }
-            if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
-                return true;
-            }
-            if(paramType.isEnum() && objectType.equals(String.class)){
+            } else if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
+                LOG.debug("Yes, assignment is possible.");
+            } else if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
+                LOG.debug("Yes, assignment is possible.");
+            } else if(paramType.isEnum() && objectType.equals(String.class)){
                 LOG.debug("Yes, will convert a String to enum");
-                return true;
-            }
-            if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
+            } else if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
                 // TODO more collection content type checking
                 LOG.debug("Assignment is possible if we convert a List to an array.");
                 LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), ((List) obj).get(0).getClass());
-
-                return true;
+            } else {
+                return false;
             }
-            return false;
         }
-        return false;
+        return true;
     }
 
     public static boolean isPrimitiveNumber(Class clazz){
         return clazz.isPrimitive() && !clazz.equals(boolean.class);
     }
+
+    public static boolean isPrimitiveBoolean(Class clazz){
+        return clazz.isPrimitive() && clazz.equals(boolean.class);
+    }
 }
 
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
index 7386900..dc9e6cb 100644
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
+++ b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
@@ -56,6 +56,8 @@
                 if(map.containsKey("ref") && map.size() == 1){
                     newVal.add(new BeanReference((String)map.get("ref")));
                     this.hasReferences = true;
+                } else {
+                    newVal.add(obj);
                 }
             } else {
                 newVal.add(obj);
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
index 7f11460..f9f28c5 100644
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
+++ b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
@@ -31,6 +31,7 @@
     private String foo;
     private String bar;
     private String fooBar;
+    private String none;
 
     public static enum TestEnum {
         FOO,
@@ -45,6 +46,10 @@
 
     }
 
+    public TestBolt(TestEnum te, float f, boolean b){
+
+    }
+
     @Override
     public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
         LOG.info("{}", tuple);
@@ -59,6 +64,9 @@
     public void withFoo(String foo){
         this.foo = foo;
     }
+    public void withNone(){
+        this.none = "hit";
+    }
     public void withBar(String bar){
         this.bar = bar;
     }
diff --git a/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml b/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
index 65211ff..cda151e 100644
--- a/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
+++ b/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
@@ -36,10 +36,12 @@
     constructorArgs:
       - FOO # enum class
       - 1.0
+      - true
     configMethods:
       - name: "withFoo"
         args:
           - "foo"
+      - name: "withNone"
       - name: "withBar"
         args:
           - "bar"
diff --git a/external/flux/flux-examples/README.md b/external/flux/flux-examples/README.md
index 0a7085e..a6afec2 100644
--- a/external/flux/flux-examples/README.md
+++ b/external/flux/flux-examples/README.md
@@ -64,3 +64,12 @@
 ```bash
 storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hbase.yaml --filter my_hbase_bolt.properties
 ```
+### [simple_windowing.yaml](src/main/resources/simple_windowing.yaml)
+
+This example illustrates how to use Flux to set up a storm topology that contains windowing operations.
+
+To run,
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_windowing.yaml
+```
diff --git a/external/flux/flux-examples/pom.xml b/external/flux/flux-examples/pom.xml
index fe43663..537b4e0 100644
--- a/external/flux/flux-examples/pom.xml
+++ b/external/flux/flux-examples/pom.xml
@@ -117,7 +117,6 @@
                 <artifactId>maven-shade-plugin</artifactId>
                 <configuration>
                     <createDependencyReducedPom>true</createDependencyReducedPom>
-                    <minimizeJar>true</minimizeJar>
                 </configuration>
                 <executions>
                     <execution>
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
new file mode 100644
index 0000000..7e84441
--- /dev/null
+++ b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
@@ -0,0 +1,39 @@
+/**
+ * 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 org.apache.storm.flux.examples;
+
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseBasicBolt;
+import backtype.storm.tuple.Tuple;
+
+/**
+ * Prints the tuples to stdout
+ */
+public class TestPrintBolt extends BaseBasicBolt {
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+        System.out.println(tuple);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer ofd) {
+    }
+
+}
\ No newline at end of file
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
new file mode 100644
index 0000000..3aab9b6
--- /dev/null
+++ b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
@@ -0,0 +1,47 @@
+/*
+ * 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 org.apache.storm.flux.examples;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseWindowedBolt;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import backtype.storm.windowing.TupleWindow;
+
+import java.util.Map;
+
+public class TestWindowBolt extends BaseWindowedBolt {
+    private OutputCollector collector;
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.collector = collector;
+    }
+
+    @Override
+    public void execute(TupleWindow inputWindow) {
+        collector.emit(new Values(inputWindow.get().size()));
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("count"));
+    }
+}
diff --git a/external/flux/flux-examples/src/main/resources/simple_windowing.yaml b/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
new file mode 100755
index 0000000..31be109
--- /dev/null
+++ b/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
@@ -0,0 +1,69 @@
+# 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.
+
+---
+
+name: "sliding-window-topology"
+
+components:
+  - id: "windowLength"
+    className: "backtype.storm.topology.base.BaseWindowedBolt$Count"
+    constructorArgs:
+      - 5
+  - id: "slidingInterval"
+    className: "backtype.storm.topology.base.BaseWindowedBolt$Count"
+    constructorArgs:
+      - 3
+
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "backtype.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.flux.examples.TestWindowBolt"
+    configMethods:
+      - name: "withWindow"
+        args: [ref: "windowLength", ref: "slidingInterval"]
+    parallelism: 1
+  - id: "bolt-2"
+    className: "org.apache.storm.flux.examples.TestPrintBolt"
+    parallelism: 1
+
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+  - name: "bolt-1 --> bolt-2" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "bolt-1"
+    to: "bolt-2"
+    grouping:
+      type: SHUFFLE
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
index 0e5fa0b..092230f 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
@@ -1,3 +1,20 @@
+/**
+ * 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 org.apache.storm.sql.compiler;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
diff --git a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
index 80037c6..1b45b30 100644
--- a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
+++ b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
@@ -19,9 +19,11 @@
 
 import backtype.storm.spout.Scheme;
 import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -34,11 +36,11 @@
   }
 
   @Override
-  public List<Object> deserialize(byte[] ser) {
+  public List<Object> deserialize(ByteBuffer ser) {
     ObjectMapper mapper = new ObjectMapper();
     try {
       @SuppressWarnings("unchecked")
-      HashMap<String, Object> map = mapper.readValue(ser, HashMap.class);
+      HashMap<String, Object> map = mapper.readValue(Utils.toByteArray(ser), HashMap.class);
       ArrayList<Object> list = new ArrayList<>();
       for (String f : fields) {
         list.add(map.get(f));
diff --git a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
index d2898e8..5973672 100644
--- a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
+++ b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
@@ -34,7 +34,7 @@
     final List<String> fields = Lists.newArrayList("ID", "val");
     final String s = "{\"ID\": 1, \"val\": \"2\"}";
     JsonScheme scheme = new JsonScheme(fields);
-    List<Object> o = scheme.deserialize(s.getBytes(Charset.defaultCharset()));
+    List<Object> o = scheme.deserialize(ByteBuffer.wrap(s.getBytes(Charset.defaultCharset())));
     assertArrayEquals(new Object[] {1, "2"}, o.toArray());
   }
 
diff --git a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
index 531f764..418bc68 100644
--- a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
+++ b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
@@ -21,6 +21,8 @@
 import com.google.common.collect.Lists;
 import kafka.javaapi.producer.Producer;
 import kafka.producer.KeyedMessage;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.storm.sql.kafka.KafkaDataSourcesProvider.KafkaTridentSink;
 import org.apache.storm.sql.runtime.DataSourcesRegistry;
 import org.apache.storm.sql.runtime.FieldInfo;
@@ -37,6 +39,7 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.Future;
 
 import static org.mockito.Mockito.*;
 
@@ -56,7 +59,8 @@
     KafkaTridentSink sink = (KafkaTridentSink) ds.getConsumer();
     sink.prepare(null, null);
     TridentKafkaState state = (TridentKafkaState) Whitebox.getInternalState(sink, "state");
-    Producer producer = mock(Producer.class);
+    KafkaProducer producer = mock(KafkaProducer.class);
+    doReturn(mock(Future.class)).when(producer).send(any(ProducerRecord.class));
     Whitebox.setInternalState(state, "producer", producer);
     List<TridentTuple> tupleList = mockTupleList();
     for (TridentTuple t : tupleList) {
@@ -79,7 +83,7 @@
     return tupleList;
   }
 
-  private static class KafkaMessageMatcher extends ArgumentMatcher<KeyedMessage> {
+  private static class KafkaMessageMatcher extends ArgumentMatcher<ProducerRecord> {
     private static final int PRIMARY_INDEX = 0;
     private final TridentTuple tuple;
 
@@ -90,11 +94,11 @@
     @SuppressWarnings("unchecked")
     @Override
     public boolean matches(Object o) {
-      KeyedMessage<Object, ByteBuffer> m = (KeyedMessage<Object,ByteBuffer>)o;
+      ProducerRecord<Object, ByteBuffer> m = (ProducerRecord<Object,ByteBuffer>)o;
       if (m.key() != tuple.get(PRIMARY_INDEX)) {
         return false;
       }
-      ByteBuffer buf = m.message();
+      ByteBuffer buf = m.value();
       ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
       return b.equals(buf);
     }
diff --git a/external/storm-elasticsearch/pom.xml b/external/storm-elasticsearch/pom.xml
index 7777549..fd263e6 100644
--- a/external/storm-elasticsearch/pom.xml
+++ b/external/storm-elasticsearch/pom.xml
@@ -41,6 +41,11 @@
 
     <dependencies>
         <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.storm</groupId>
             <artifactId>storm-core</artifactId>
             <version>${project.version}</version>
diff --git a/external/storm-eventhubs/pom.xml b/external/storm-eventhubs/pom.xml
index 0338e1e..b227f24 100755
--- a/external/storm-eventhubs/pom.xml
+++ b/external/storm-eventhubs/pom.xml
@@ -49,7 +49,6 @@
                     </execution>

                 </executions>

                 <configuration>

-                    <minimizeJar>true</minimizeJar>

                     <transformers>

                         <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer">

                         </transformer>

diff --git a/external/storm-hbase/README.md b/external/storm-hbase/README.md
index 81d351a..fd4d0ad 100644
--- a/external/storm-hbase/README.md
+++ b/external/storm-hbase/README.md
@@ -107,6 +107,16 @@
  ```java
 HBaseBolt hbase = new HBaseBolt("WordCount", mapper);
  ```
+ 
+ HBaseBolt params
+
+|Arg  |Description | Type | Default |
+|---	|--- |---
+|writeToWAL | To turn Durability SYNC_WAL or SKIP_WAL | Boolean (Optional) | True |
+|configKey | Any Hbase related configs | Map (Optional) | |
+|batchSize | Max no.of Tuples batched together to write to HBase | Int (Optional) | 15000 |
+|flushIntervalSecs| (In seconds)  If > 0 HBase Bolt will periodically flush transaction batches. Enabling this is recommended to avoid tuple timeouts while waiting for a batch to fill up. | Int (Optional) | 0 |
+
 
 The `HBaseBolt` will delegate to the `mapper` instance to figure out how to persist tuple data to HBase.
 
diff --git a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java
index d814117..404aa7a 100644
--- a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java
+++ b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/AbstractHBaseBolt.java
@@ -42,6 +42,8 @@
     protected String tableName;
     protected HBaseMapper mapper;
     protected String configKey;
+    protected int batchSize = 15000;
+    protected int flushIntervalSecs = 0;
 
     public AbstractHBaseBolt(String tableName, HBaseMapper mapper) {
         Validate.notEmpty(tableName, "Table name can not be blank or null");
diff --git a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java
index cf29aa5..ea17b2c 100644
--- a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java
+++ b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseBolt.java
@@ -19,6 +19,8 @@
 
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.TupleUtils;
+import backtype.storm.Config;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.storm.hbase.bolt.mapper.HBaseMapper;
@@ -26,7 +28,9 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Map;
 import java.util.List;
+import java.util.LinkedList;
 
 /**
  * Basic bolt for writing to HBase.
@@ -38,9 +42,13 @@
     private static final Logger LOG = LoggerFactory.getLogger(HBaseBolt.class);
 
     boolean writeToWAL = true;
+    List<Mutation> batchMutations;
+    List<Tuple> tupleBatch;
 
     public HBaseBolt(String tableName, HBaseMapper mapper) {
         super(tableName, mapper);
+        this.batchMutations = new LinkedList<>();
+        this.tupleBatch = new LinkedList<>();
     }
 
     public HBaseBolt writeToWAL(boolean writeToWAL) {
@@ -53,21 +61,70 @@
         return this;
     }
 
-    @Override
-    public void execute(Tuple tuple) {
-        byte[] rowKey = this.mapper.rowKey(tuple);
-        ColumnList cols = this.mapper.columns(tuple);
-        List<Mutation> mutations = hBaseClient.constructMutationReq(rowKey, cols, writeToWAL? Durability.SYNC_WAL : Durability.SKIP_WAL);
+    public HBaseBolt withBatchSize(int batchSize) {
+        this.batchSize = batchSize;
+        return this;
+    }
 
-        try {
-            this.hBaseClient.batchMutate(mutations);
-        } catch(Exception e){
-            this.collector.reportError(e);
-            this.collector.fail(tuple);
-            return;
+    public HBaseBolt withFlushIntervalSecs(int flushIntervalSecs) {
+        this.flushIntervalSecs = flushIntervalSecs;
+        return this;
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        Map<String, Object> conf = super.getComponentConfiguration();
+        if (conf == null) {
+            conf = new Config();
         }
 
-        this.collector.ack(tuple);
+        if (conf.containsKey("topology.message.timeout.secs") && flushIntervalSecs == 0) {
+            Integer topologyTimeout = Integer.parseInt(conf.get("topology.message.timeout.secs").toString());
+            flushIntervalSecs = (int)(Math.floor(topologyTimeout / 2));
+            LOG.debug("Setting flush interval to [{}] based on topology.message.timeout.secs", flushIntervalSecs);
+        }
+
+        LOG.info("Enabling tick tuple with interval [{}]", flushIntervalSecs);
+        conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, flushIntervalSecs);
+        return conf;
+    }
+
+
+    @Override
+    public void execute(Tuple tuple) {
+        boolean flush = false;
+        try {
+            if (TupleUtils.isTick(tuple)) {
+                LOG.debug("TICK received! current batch status [" + tupleBatch.size() + "/" + batchSize + "]");
+                flush = true;
+            } else {
+                byte[] rowKey = this.mapper.rowKey(tuple);
+                ColumnList cols = this.mapper.columns(tuple);
+                List<Mutation> mutations = hBaseClient.constructMutationReq(rowKey, cols, writeToWAL? Durability.SYNC_WAL : Durability.SKIP_WAL);
+                batchMutations.addAll(mutations);
+                tupleBatch.add(tuple);
+                if (tupleBatch.size() >= batchSize) {
+                    flush = true;
+                }
+            }
+
+            if (flush && !tupleBatch.isEmpty()) {
+                this.hBaseClient.batchMutate(batchMutations);
+                LOG.debug("acknowledging tuples after batchMutate");
+                for(Tuple t : tupleBatch) {
+                    collector.ack(t);
+                }
+                tupleBatch.clear();
+                batchMutations.clear();
+            }
+        } catch(Exception e){
+            this.collector.reportError(e);
+            for (Tuple t : tupleBatch) {
+                collector.fail(t);
+            }
+            tupleBatch.clear();
+            batchMutations.clear();
+        }
     }
 
     @Override
diff --git a/external/storm-hdfs/README.md b/external/storm-hdfs/README.md
index e1f2b28..7819f81 100644
--- a/external/storm-hdfs/README.md
+++ b/external/storm-hdfs/README.md
@@ -277,6 +277,39 @@
 }
 ```
 
+## Support for Avro Files
+
+The `org.apache.storm.hdfs.bolt.AvroGenericRecordBolt` class allows you to write Avro objects directly to HDFS:
+ 
+```java
+        // sync the filesystem after every 1k tuples
+        SyncPolicy syncPolicy = new CountSyncPolicy(1000);
+
+        // rotate files when they reach 5MB
+        FileRotationPolicy rotationPolicy = new FileSizeRotationPolicy(5.0f, Units.MB);
+
+        FileNameFormat fileNameFormat = new DefaultFileNameFormat()
+                .withExtension(".avro")
+                .withPath("/data/");
+
+        // create sequence format instance.
+        DefaultSequenceFormat format = new DefaultSequenceFormat("timestamp", "sentence");
+
+        SequenceFileBolt bolt = new SequenceFileBolt()
+                .withFsUrl("hdfs://localhost:54310")
+                .withFileNameFormat(fileNameFormat)
+                .withSchemaAsString(schema)
+                .withRotationPolicy(rotationPolicy)
+                .withSyncPolicy(syncPolicy);
+```
+
+The setup is very similar to the `SequenceFileBolt` example above.  The key difference is that instead of specifying a
+`SequenceFormat` you must provide a string representation of an Avro schema through the `withSchemaAsString()` method.
+An `org.apache.avro.Schema` object cannot be directly provided since it does not implement `Serializable`.
+
+The AvroGenericRecordBolt expects to receive tuples containing an Avro GenericRecord that conforms to the provided
+schema.
+
 ## Trident API
 storm-hdfs also includes a Trident `state` implementation for writing data to HDFS, with an API that closely mirrors
 that of the bolts.
diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml
index 09b9f65..1765be9 100644
--- a/external/storm-hdfs/pom.xml
+++ b/external/storm-hdfs/pom.xml
@@ -41,6 +41,29 @@
             <artifactId>storm-core</artifactId>
             <version>${project.version}</version>
             <scope>provided</scope>
+            <exclusions>
+                <!--log4j-over-slf4j must be excluded for hadoop-minicluster
+                    see: http://stackoverflow.com/q/20469026/3542091 -->
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <exclusions>
+                <!--log4j-over-slf4j must be excluded for hadoop-minicluster
+                    see: http://stackoverflow.com/q/20469026/3542091 -->
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
             <groupId>org.apache.hadoop</groupId>
@@ -69,6 +92,17 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+            <version>${hadoop.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
             <scope>test</scope>
@@ -81,19 +115,28 @@
         </dependency>
     </dependencies>
     <build>
-      <plugins>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-jar-plugin</artifactId>
-          <version>2.2</version>
-          <executions>
-            <execution>
-              <goals>
-                <goal>test-jar</goal>
-              </goals>
-            </execution>
-          </executions>
-        </plugin>
-      </plugins>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.2</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>2.5.1</version>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                </configuration>
+            </plugin>
+        </plugins>
     </build>
 </project>
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java
index f260598..1108fbb 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AbstractHdfsBolt.java
@@ -17,10 +17,14 @@
  */
 package org.apache.storm.hdfs.bolt;
 
+import backtype.storm.Config;
 import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.TupleUtils;
+import backtype.storm.utils.Utils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -35,12 +39,15 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.LinkedList;
 import java.util.Map;
+import java.util.List;
 import java.util.Timer;
 import java.util.TimerTask;
 
 public abstract class AbstractHdfsBolt extends BaseRichBolt {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractHdfsBolt.class);
+    private static final Integer DEFAULT_RETRY_COUNT = 3;
 
     protected ArrayList<RotationAction> rotationActions = new ArrayList<RotationAction>();
     private Path currentFile;
@@ -54,6 +61,10 @@
     protected String configKey;
     protected transient Object writeLock;
     protected transient Timer rotationTimer; // only used for TimedRotationPolicy
+    private List<Tuple> tupleBatch = new LinkedList<>();
+    protected long offset = 0;
+    protected Integer fileRetryCount = DEFAULT_RETRY_COUNT;
+    protected Integer tickTupleInterval = 0;
 
     protected transient Configuration hdfsConfig;
 
@@ -99,6 +110,13 @@
             }
         }
 
+        // If interval is non-zero then it has already been explicitly set and we should not default it
+        if (conf.containsKey("topology.message.timeout.secs") && tickTupleInterval == 0)
+        {
+            Integer topologyTimeout = Utils.getInt(conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS));
+            tickTupleInterval = (int)(Math.floor(topologyTimeout / 2));
+            LOG.debug("Setting tick tuple interval to [{}] based on topology timeout", tickTupleInterval);
+        }
 
         try{
             HdfsSecurityUtil.login(conf, hdfsConfig);
@@ -127,9 +145,115 @@
     }
 
     @Override
+    public final void execute(Tuple tuple) {
+
+        synchronized (this.writeLock) {
+            boolean forceSync = false;
+            if (TupleUtils.isTick(tuple)) {
+                LOG.debug("TICK! forcing a file system flush");
+                forceSync = true;
+            } else {
+                try {
+                    writeTuple(tuple);
+                    tupleBatch.add(tuple);
+                } catch (IOException e) {
+                    //If the write failed, try to sync anything already written
+                    LOG.info("Tuple failed to write, forcing a flush of existing data.");
+                    this.collector.reportError(e);
+                    forceSync = true;
+                    this.collector.fail(tuple);
+                }
+            }
+
+            if (this.syncPolicy.mark(tuple, this.offset) || (forceSync && tupleBatch.size() > 0)) {
+                int attempts = 0;
+                boolean success = false;
+                IOException lastException = null;
+                // Make every attempt to sync the data we have.  If it can't be done then kill the bolt with
+                // a runtime exception.  The filesystem is presumably in a very bad state.
+                while (success == false && attempts < fileRetryCount) {
+                    attempts += 1;
+                    try {
+                        syncTuples();
+                        LOG.debug("Data synced to filesystem. Ack'ing [{}] tuples", tupleBatch.size());
+                        for (Tuple t : tupleBatch) {
+                            this.collector.ack(t);
+                        }
+                        tupleBatch.clear();
+                        syncPolicy.reset();
+                        success = true;
+                    } catch (IOException e) {
+                        LOG.warn("Data could not be synced to filesystem on attempt [{}]", attempts);
+                        this.collector.reportError(e);
+                        lastException = e;
+                    }
+                }
+
+                // If unsuccesful fail the pending tuples
+                if (success == false) {
+                    LOG.warn("Data could not be synced to filesystem, failing this batch of tuples");
+                    for (Tuple t : tupleBatch) {
+                        this.collector.fail(t);
+                    }
+                    tupleBatch.clear();
+
+                    throw new RuntimeException("Sync failed [" + attempts + "] times.", lastException);
+                }
+            }
+
+            if(this.rotationPolicy.mark(tuple, this.offset)) {
+                try {
+                    rotateOutputFile();
+                    this.rotationPolicy.reset();
+                    this.offset = 0;
+                } catch (IOException e) {
+                    this.collector.reportError(e);
+                    LOG.warn("File could not be rotated");
+                    //At this point there is nothing to do.  In all likelihood any filesystem operations will fail.
+                    //The next tuple will almost certainly fail to write and/or sync, which force a rotation.  That
+                    //will give rotateAndReset() a chance to work which includes creating a fresh file handle.
+                }
+            }
+        }
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        Map<String, Object> conf = super.getComponentConfiguration();
+        if (conf == null)
+            conf = new Config();
+
+        if (tickTupleInterval > 0) {
+            LOG.info("Enabling tick tuple with interval [{}]", tickTupleInterval);
+            conf.put(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS, tickTupleInterval);
+        }
+
+        return conf;
+    }
+
+    @Override
     public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
     }
 
+    /**
+     * writes a tuple to the underlying filesystem but makes no guarantees about syncing data.
+     *
+     * this.offset is also updated to reflect additional data written
+     *
+     * @param tuple
+     * @throws IOException
+     */
+    abstract void writeTuple(Tuple tuple) throws IOException;
+
+    /**
+     * Make the best effort to sync written data to the underlying file system.  Concrete classes should very clearly
+     * state the file state that sync guarantees.  For example, HdfsBolt can make a much stronger guarantee than
+     * SequenceFileBolt.
+     *
+     * @throws IOException
+     */
+    abstract void syncTuples() throws IOException;
+
     abstract void closeOutputFile() throws IOException;
 
     abstract Path createOutputFile() throws IOException;
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java
new file mode 100644
index 0000000..d2bfba8
--- /dev/null
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBolt.java
@@ -0,0 +1,145 @@
+/**
+ * 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 org.apache.storm.hdfs.bolt;
+
+import backtype.storm.Config;
+import backtype.storm.Constants;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+
+import org.apache.storm.hdfs.bolt.format.FileNameFormat;
+import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
+import org.apache.storm.hdfs.common.rotation.RotationAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.net.URI;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.LinkedList;
+import java.util.Map;
+
+public class AvroGenericRecordBolt extends AbstractHdfsBolt{
+
+    private static final Logger LOG = LoggerFactory.getLogger(AvroGenericRecordBolt.class);
+
+    private transient FSDataOutputStream out;
+    private Schema schema;
+    private String schemaAsString;
+    private DataFileWriter<GenericRecord> avroWriter;
+
+    public AvroGenericRecordBolt withSchemaAsString(String schemaAsString)
+    {
+        this.schemaAsString = schemaAsString;
+        return this;
+    }
+
+    public AvroGenericRecordBolt withFsUrl(String fsUrl){
+        this.fsUrl = fsUrl;
+        return this;
+    }
+
+    public AvroGenericRecordBolt withConfigKey(String configKey){
+        this.configKey = configKey;
+        return this;
+    }
+
+    public AvroGenericRecordBolt withFileNameFormat(FileNameFormat fileNameFormat){
+        this.fileNameFormat = fileNameFormat;
+        return this;
+    }
+
+    public AvroGenericRecordBolt withSyncPolicy(SyncPolicy syncPolicy){
+        this.syncPolicy = syncPolicy;
+        return this;
+    }
+
+    public AvroGenericRecordBolt withRotationPolicy(FileRotationPolicy rotationPolicy){
+        this.rotationPolicy = rotationPolicy;
+        return this;
+    }
+
+    public AvroGenericRecordBolt addRotationAction(RotationAction action){
+        this.rotationActions.add(action);
+        return this;
+    }
+
+    public AvroGenericRecordBolt withTickTupleIntervalSeconds(int interval) {
+        this.tickTupleInterval = interval;
+        return this;
+    }
+
+    @Override
+    void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collector) throws IOException {
+        LOG.info("Preparing AvroGenericRecord Bolt...");
+        this.fs = FileSystem.get(URI.create(this.fsUrl), hdfsConfig);
+        Schema.Parser parser = new Schema.Parser();
+        this.schema = parser.parse(this.schemaAsString);
+    }
+
+    @Override
+    void writeTuple(Tuple tuple) throws IOException {
+        GenericRecord avroRecord = (GenericRecord) tuple.getValue(0);
+        avroWriter.append(avroRecord);
+        offset = this.out.getPos();
+    }
+
+    @Override
+    void syncTuples() throws IOException {
+        avroWriter.flush();
+
+        LOG.debug("Attempting to sync all data to filesystem");
+        if (this.out instanceof HdfsDataOutputStream) {
+            ((HdfsDataOutputStream) this.out).hsync(EnumSet.of(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH));
+        } else {
+            this.out.hsync();
+        }
+        this.syncPolicy.reset();
+    }
+
+    @Override
+    protected void closeOutputFile() throws IOException
+    {
+        avroWriter.close();
+        this.out.close();
+    }
+
+    @Override
+    Path createOutputFile() throws IOException {
+        Path path = new Path(this.fileNameFormat.getPath(), this.fileNameFormat.getName(this.rotation, System.currentTimeMillis()));
+        this.out = this.fs.create(path);
+
+        //Initialize writer
+        DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        avroWriter = new DataFileWriter<>(datumWriter);
+        avroWriter.create(this.schema, this.out);
+
+        return path;
+    }
+}
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java
index dcb09e7..101aa57 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java
@@ -43,7 +43,6 @@
 
     private transient FSDataOutputStream out;
     private RecordFormat format;
-    private long offset = 0;
 
     public HdfsBolt withFsUrl(String fsUrl){
         this.fsUrl = fsUrl;
@@ -80,6 +79,16 @@
         return this;
     }
 
+    public HdfsBolt withTickTupleIntervalSeconds(int interval) {
+        this.tickTupleInterval = interval;
+        return this;
+    }
+
+    public HdfsBolt withRetryCount(int fileRetryCount) {
+        this.fileRetryCount = fileRetryCount;
+        return this;
+    }
+
     @Override
     public void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collector) throws IOException {
         LOG.info("Preparing HDFS Bolt...");
@@ -87,37 +96,23 @@
     }
 
     @Override
-    public void execute(Tuple tuple) {
-        try {
-            byte[] bytes = this.format.format(tuple);
-            synchronized (this.writeLock) {
-                out.write(bytes);
-                this.offset += bytes.length;
-
-                if (this.syncPolicy.mark(tuple, this.offset)) {
-                    if (this.out instanceof HdfsDataOutputStream) {
-                        ((HdfsDataOutputStream) this.out).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
-                    } else {
-                        this.out.hsync();
-                    }
-                    this.syncPolicy.reset();
-                }
-            }
-
-            this.collector.ack(tuple);
-
-            if(this.rotationPolicy.mark(tuple, this.offset)){
-                rotateOutputFile(); // synchronized
-                this.offset = 0;
-                this.rotationPolicy.reset();
-            }
-        } catch (IOException e) {
-            this.collector.reportError(e);
-            this.collector.fail(tuple);
+    void syncTuples() throws IOException {
+        LOG.debug("Attempting to sync all data to filesystem");
+        if (this.out instanceof HdfsDataOutputStream) {
+            ((HdfsDataOutputStream) this.out).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
+        } else {
+            this.out.hsync();
         }
     }
 
     @Override
+    void writeTuple(Tuple tuple) throws IOException {
+        byte[] bytes = this.format.format(tuple);
+        out.write(bytes);
+        this.offset += bytes.length;
+    }
+
+    @Override
     void closeOutputFile() throws IOException {
         this.out.close();
     }
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java
index baf4df0..fcd0d29 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileBolt.java
@@ -89,11 +89,21 @@
         return this;
     }
 
+    public SequenceFileBolt withTickTupleIntervalSeconds(int interval) {
+        this.tickTupleInterval = interval;
+        return this;
+    }
+
     public SequenceFileBolt addRotationAction(RotationAction action){
         this.rotationActions.add(action);
         return this;
     }
 
+    public SequenceFileBolt withRetryCount(int fileRetryCount) {
+        this.fileRetryCount = fileRetryCount;
+        return this;
+    }
+
     @Override
     public void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector collector) throws IOException {
         LOG.info("Preparing Sequence File Bolt...");
@@ -104,29 +114,15 @@
     }
 
     @Override
-    public void execute(Tuple tuple) {
-        try {
-            long offset;
-            synchronized (this.writeLock) {
-                this.writer.append(this.format.key(tuple), this.format.value(tuple));
-                offset = this.writer.getLength();
+    void syncTuples() throws IOException {
+        LOG.debug("Attempting to sync all data to filesystem");
+        this.writer.hsync();
+    }
 
-                if (this.syncPolicy.mark(tuple, offset)) {
-                    this.writer.hsync();
-                    this.syncPolicy.reset();
-                }
-            }
-
-            this.collector.ack(tuple);
-            if (this.rotationPolicy.mark(tuple, offset)) {
-                rotateOutputFile(); // synchronized
-                this.rotationPolicy.reset();
-            }
-        } catch (IOException e) {
-            this.collector.reportError(e);
-            this.collector.fail(tuple);
-        }
-
+    @Override
+    void writeTuple(Tuple tuple) throws IOException {
+        this.writer.append(this.format.key(tuple), this.format.value(tuple));
+        this.offset = this.writer.getLength();
     }
 
     Path createOutputFile() throws IOException {
@@ -144,6 +140,4 @@
     void closeOutputFile() throws IOException {
         this.writer.close();
     }
-
-
 }
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java
index f6736bd..1e38051 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/ha/codedistributor/HDFSCodeDistributor.java
@@ -1,3 +1,20 @@
+/**
+ * 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 org.apache.storm.hdfs.ha.codedistributor;
 
 import backtype.storm.codedistributor.ICodeDistributor;
diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBoltTest.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBoltTest.java
new file mode 100644
index 0000000..37ba27a
--- /dev/null
+++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/AvroGenericRecordBoltTest.java
@@ -0,0 +1,220 @@
+/**
+ * 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 org.apache.storm.hdfs.bolt;
+
+import backtype.storm.Config;
+import backtype.storm.task.GeneralTopologyContext;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.TupleImpl;
+import backtype.storm.tuple.Values;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.file.DataFileReader;
+import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.bolt.format.FileNameFormat;
+import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy;
+import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
+import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.Assert;
+
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+
+public class AvroGenericRecordBoltTest {
+
+    private String hdfsURI;
+    private DistributedFileSystem fs;
+    private MiniDFSCluster hdfsCluster;
+    private static final String testRoot = "/unittest";
+    private static final Schema schema;
+    private static final Tuple tuple1;
+    private static final Tuple tuple2;
+    private static final String userSchema = "{\"type\":\"record\"," +
+            "\"name\":\"myrecord\"," +
+            "\"fields\":[{\"name\":\"foo1\",\"type\":\"string\"}," +
+            "{ \"name\":\"int1\", \"type\":\"int\" }]}";
+
+    static {
+
+        Schema.Parser parser = new Schema.Parser();
+        schema = parser.parse(userSchema);
+
+        GenericRecord record1 = new GenericData.Record(schema);
+        record1.put("foo1", "bar1");
+        record1.put("int1", 1);
+        tuple1 = generateTestTuple(record1);
+
+        GenericRecord record2 = new GenericData.Record(schema);
+        record2.put("foo1", "bar2");
+        record2.put("int1", 2);
+        tuple2 = generateTestTuple(record2);
+    }
+
+    @Mock private OutputCollector collector;
+    @Mock private TopologyContext topologyContext;
+
+    @Before
+    public void setup() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        Configuration conf = new Configuration();
+        conf.set("fs.trash.interval", "10");
+        conf.setBoolean("dfs.permissions", true);
+        File baseDir = new File("./target/hdfs/").getAbsoluteFile();
+        FileUtil.fullyDelete(baseDir);
+        conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+
+        MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+        hdfsCluster = builder.build();
+        fs = hdfsCluster.getFileSystem();
+        hdfsURI = fs.getUri() + "/";
+    }
+
+    @After
+    public void shutDown() throws IOException {
+        fs.close();
+        hdfsCluster.shutdown();
+    }
+
+    @Test public void multipleTuplesOneFile() throws IOException
+    {
+        AvroGenericRecordBolt bolt = makeAvroBolt(hdfsURI, 1, 1f, userSchema);
+
+        bolt.prepare(new Config(), topologyContext, collector);
+        bolt.execute(tuple1);
+        bolt.execute(tuple2);
+        bolt.execute(tuple1);
+        bolt.execute(tuple2);
+
+        Assert.assertEquals(1, countNonZeroLengthFiles(testRoot));
+        verifyAllAvroFiles(testRoot, schema);
+    }
+
+    @Test public void multipleTuplesMutliplesFiles() throws IOException
+    {
+        AvroGenericRecordBolt bolt = makeAvroBolt(hdfsURI, 1, .0001f, userSchema);
+
+        bolt.prepare(new Config(), topologyContext, collector);
+        bolt.execute(tuple1);
+        bolt.execute(tuple2);
+        bolt.execute(tuple1);
+        bolt.execute(tuple2);
+
+        Assert.assertEquals(4, countNonZeroLengthFiles(testRoot));
+        verifyAllAvroFiles(testRoot, schema);
+    }
+
+    private AvroGenericRecordBolt makeAvroBolt(String nameNodeAddr, int countSync, float rotationSizeMB, String schemaAsString) {
+
+        SyncPolicy fieldsSyncPolicy = new CountSyncPolicy(countSync);
+
+        FileNameFormat fieldsFileNameFormat = new DefaultFileNameFormat().withPath(testRoot);
+
+        FileRotationPolicy rotationPolicy =
+                new FileSizeRotationPolicy(rotationSizeMB, FileSizeRotationPolicy.Units.MB);
+
+        return new AvroGenericRecordBolt()
+                .withFsUrl(nameNodeAddr)
+                .withFileNameFormat(fieldsFileNameFormat)
+                .withSchemaAsString(schemaAsString)
+                .withRotationPolicy(rotationPolicy)
+                .withSyncPolicy(fieldsSyncPolicy);
+    }
+
+    private static Tuple generateTestTuple(GenericRecord record) {
+        TopologyBuilder builder = new TopologyBuilder();
+        GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(),
+                new Config(), new HashMap(), new HashMap(), new HashMap(), "") {
+            @Override
+            public Fields getComponentOutputFields(String componentId, String streamId) {
+                return new Fields("record");
+            }
+        };
+        return new TupleImpl(topologyContext, new Values(record), 1, "");
+    }
+
+    private void verifyAllAvroFiles(String path, Schema schema) throws IOException {
+        Path p = new Path(path);
+
+        for (FileStatus file : fs.listStatus(p)) {
+            if (file.getLen() > 0) {
+                fileIsGoodAvro(file.getPath(), schema);
+            }
+        }
+    }
+
+    private int countNonZeroLengthFiles(String path) throws IOException {
+        Path p = new Path(path);
+        int nonZero = 0;
+
+        for (FileStatus file : fs.listStatus(p)) {
+            if (file.getLen() > 0) {
+                nonZero++;
+            }
+        }
+
+        return nonZero;
+    }
+
+    private void fileIsGoodAvro (Path path, Schema schema) throws IOException {
+        DatumReader<GenericRecord> datumReader = new GenericDatumReader<>(schema);
+        FSDataInputStream in = fs.open(path, 0);
+        FileOutputStream out = new FileOutputStream("target/FOO.avro");
+
+        byte[] buffer = new byte[100];
+        int bytesRead;
+        while ((bytesRead = in.read(buffer)) > 0) {
+            out.write(buffer, 0, bytesRead);
+        }
+        out.close();
+
+        java.io.File file = new File("target/FOO.avro");
+
+        DataFileReader<GenericRecord> dataFileReader = new DataFileReader<>(file, datumReader);
+        GenericRecord user = null;
+        while (dataFileReader.hasNext()) {
+            user = dataFileReader.next(user);
+            System.out.println(user);
+        }
+
+        file.delete();
+    }
+}
diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/TestHdfsBolt.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/TestHdfsBolt.java
new file mode 100644
index 0000000..2f2014c
--- /dev/null
+++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/TestHdfsBolt.java
@@ -0,0 +1,258 @@
+/**
+ * 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 org.apache.storm.hdfs.bolt;
+
+import backtype.storm.Config;
+import backtype.storm.task.GeneralTopologyContext;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.TupleImpl;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.MockTupleHelpers;
+import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat;
+import org.apache.storm.hdfs.bolt.format.FileNameFormat;
+import org.apache.storm.hdfs.bolt.format.RecordFormat;
+import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy;
+import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
+import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.Assert;
+
+import org.junit.rules.ExpectedException;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import static org.mockito.Mockito.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+
+public class TestHdfsBolt {
+
+    private String hdfsURI;
+    private DistributedFileSystem fs;
+    private MiniDFSCluster hdfsCluster;
+    private static final String testRoot = "/unittest";
+    Tuple tuple1 = generateTestTuple(1, "First Tuple", "SFO", "CA");
+    Tuple tuple2 = generateTestTuple(1, "Second Tuple", "SJO", "CA");
+
+    @Mock private OutputCollector collector;
+    @Mock private TopologyContext topologyContext;
+    @Rule public ExpectedException thrown = ExpectedException.none();
+
+    @Before
+    public void setup() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        Configuration conf = new Configuration();
+        conf.set("fs.trash.interval", "10");
+        conf.setBoolean("dfs.permissions", true);
+        File baseDir = new File("./target/hdfs/").getAbsoluteFile();
+        FileUtil.fullyDelete(baseDir);
+        conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+
+        MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+        hdfsCluster = builder.build();
+        fs = hdfsCluster.getFileSystem();
+        hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/";
+    }
+
+    @After
+    public void shutDown() throws IOException {
+        fs.close();
+        hdfsCluster.shutdown();
+    }
+
+    @Test
+    public void testTwoTuplesTwoFiles() throws IOException {
+        HdfsBolt bolt = makeHdfsBolt(hdfsURI, 1, .00001f);
+
+        bolt.prepare(new Config(), topologyContext, collector);
+        bolt.execute(tuple1);
+        bolt.execute(tuple2);
+
+        verify(collector).ack(tuple1);
+        verify(collector).ack(tuple2);
+
+        Assert.assertEquals(2, countNonZeroLengthFiles(testRoot));
+    }
+
+    @Test
+    public void testTwoTuplesOneFile() throws IOException
+    {
+        HdfsBolt bolt = makeHdfsBolt(hdfsURI, 2, 10000f);
+        bolt.prepare(new Config(), topologyContext, collector);
+        bolt.execute(tuple1);
+
+        verifyZeroInteractions(collector);
+
+        bolt.execute(tuple2);
+        verify(collector).ack(tuple1);
+        verify(collector).ack(tuple2);
+
+        Assert.assertEquals(1, countNonZeroLengthFiles(testRoot));
+    }
+
+    @Test
+    public void testFailedSync() throws IOException
+    {
+        HdfsBolt bolt = makeHdfsBolt(hdfsURI, 1, .00001f);
+        bolt.prepare(new Config(), topologyContext, collector);
+
+        fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+
+        // All writes/syncs will fail so this should cause a RuntimeException
+        thrown.expect(RuntimeException.class);
+        bolt.execute(tuple1);
+
+    }
+
+    // One tuple and one rotation should yield one file with data and one zero length file
+    // The failed executions should not cause rotations and new zero length files
+    @Test
+    public void testFailureFilecount() throws IOException, InterruptedException
+    {
+        HdfsBolt bolt = makeHdfsBolt(hdfsURI, 1, .000001f);
+        bolt.prepare(new Config(), topologyContext, collector);
+
+        bolt.execute(tuple1);
+        fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+        try
+        {
+            bolt.execute(tuple2);
+        } catch (RuntimeException e) {
+            //
+        }
+        try
+        {
+            bolt.execute(tuple2);
+        } catch (RuntimeException e) {
+            //
+        }
+        try
+        {
+            bolt.execute(tuple2);
+        } catch (RuntimeException e) {
+            //
+        }
+
+        Assert.assertEquals(1, countNonZeroLengthFiles(testRoot));
+        Assert.assertEquals(1, countZeroLengthFiles(testRoot));
+    }
+
+    @Test
+    public void testTickTuples() throws IOException
+    {
+        HdfsBolt bolt = makeHdfsBolt(hdfsURI, 10, 10000f);
+        bolt.prepare(new Config(), topologyContext, collector);
+
+        bolt.execute(tuple1);
+
+        //Should not have flushed to file system yet
+        Assert.assertEquals(0, countNonZeroLengthFiles(testRoot));
+
+        bolt.execute(MockTupleHelpers.mockTickTuple());
+
+        //Tick should have flushed it
+        Assert.assertEquals(1, countNonZeroLengthFiles(testRoot));
+    }
+
+    public void createBaseDirectory(FileSystem passedFs, String path) throws IOException {
+        Path p = new Path(path);
+        passedFs.mkdirs(p);
+    }
+
+    private HdfsBolt makeHdfsBolt(String nameNodeAddr, int countSync, float rotationSizeMB) {
+
+        RecordFormat fieldsFormat = new DelimitedRecordFormat().withFieldDelimiter("|");
+
+        SyncPolicy fieldsSyncPolicy = new CountSyncPolicy(countSync);
+
+        FileRotationPolicy fieldsRotationPolicy =
+                new FileSizeRotationPolicy(rotationSizeMB, FileSizeRotationPolicy.Units.MB);
+
+        FileNameFormat fieldsFileNameFormat = new DefaultFileNameFormat().withPath(testRoot);
+
+        return new HdfsBolt()
+                .withFsUrl(nameNodeAddr)
+                .withFileNameFormat(fieldsFileNameFormat)
+                .withRecordFormat(fieldsFormat)
+                .withRotationPolicy(fieldsRotationPolicy)
+                .withSyncPolicy(fieldsSyncPolicy);
+    }
+
+    private Tuple generateTestTuple(Object id, Object msg,Object city,Object state) {
+        TopologyBuilder builder = new TopologyBuilder();
+        GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(),
+                new Config(), new HashMap(), new HashMap(), new HashMap(), "") {
+            @Override
+            public Fields getComponentOutputFields(String componentId, String streamId) {
+                return new Fields("id", "msg","city","state");
+            }
+        };
+        return new TupleImpl(topologyContext, new Values(id, msg,city,state), 1, "");
+    }
+
+    private void printFiles(String path) throws IOException {
+        Path p = new Path(path);
+        FileStatus[] fileStatuses = fs.listStatus(p);
+        for (FileStatus file : fileStatuses) {
+            System.out.println("@@@ " + file.getPath() + " [" + file.getLen() + "]");
+        }
+    }
+
+    // Generally used to compare how files were actually written and compare to expectations based on total
+    // amount of data written and rotation policies
+    private int countNonZeroLengthFiles(String path) throws IOException {
+        Path p = new Path(path);
+        int nonZero = 0;
+
+        for (FileStatus file : fs.listStatus(p))
+            if (file.getLen() > 0)
+                nonZero++;
+
+        return nonZero;
+    }
+
+    private int countZeroLengthFiles(String path) throws IOException {
+        Path p = new Path(path);
+        int zeroLength = 0;
+
+        for (FileStatus file : fs.listStatus(p))
+            if (file.getLen() == 0)
+                zeroLength++;
+
+        return zeroLength;
+    }
+}
diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/TestSequenceFileBolt.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/TestSequenceFileBolt.java
new file mode 100644
index 0000000..5c760ef
--- /dev/null
+++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/bolt/TestSequenceFileBolt.java
@@ -0,0 +1,186 @@
+/**
+ * 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 org.apache.storm.hdfs.bolt;
+
+import backtype.storm.Config;
+import backtype.storm.Constants;
+import backtype.storm.task.GeneralTopologyContext;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.TupleImpl;
+import backtype.storm.tuple.Values;
+import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.bolt.format.FileNameFormat;
+import org.apache.storm.hdfs.bolt.format.SequenceFormat;
+import org.apache.storm.hdfs.bolt.format.DefaultSequenceFormat;
+import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy;
+import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
+import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
+import org.junit.*;
+
+import org.junit.rules.ExpectedException;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import static org.mockito.Mockito.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+
+public class TestSequenceFileBolt {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestSequenceFileBolt.class);
+
+    private String hdfsURI;
+    private DistributedFileSystem fs;
+    private MiniDFSCluster hdfsCluster;
+    private static final String testRoot = "/unittest";
+    Tuple tuple1 = generateTestTuple(1l, "first tuple");
+    Tuple tuple2 = generateTestTuple(2l, "second tuple");
+
+    @Mock private OutputCollector collector;
+    @Mock private TopologyContext topologyContext;
+    @Rule public ExpectedException thrown = ExpectedException.none();
+
+    @Before
+    public void setup() throws Exception {
+        MockitoAnnotations.initMocks(this);
+        Configuration conf = new Configuration();
+        conf.set("fs.trash.interval", "10");
+        conf.setBoolean("dfs.permissions", true);
+        File baseDir = new File("./target/hdfs/").getAbsoluteFile();
+        FileUtil.fullyDelete(baseDir);
+        conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+
+        MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+        hdfsCluster = builder.build();
+        fs = hdfsCluster.getFileSystem();
+        hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/";
+    }
+
+    @After
+    public void shutDown() throws IOException {
+        fs.close();
+        hdfsCluster.shutdown();
+    }
+
+    @Test
+    public void testTwoTuplesTwoFiles() throws IOException {
+        SequenceFileBolt bolt = makeSeqBolt(hdfsURI, 1, .00001f);
+
+        bolt.prepare(new Config(), topologyContext, collector);
+        bolt.execute(tuple1);
+        bolt.execute(tuple2);
+
+        verify(collector).ack(tuple1);
+        verify(collector).ack(tuple2);
+
+        Assert.assertEquals(2, countNonZeroLengthFiles(testRoot));
+    }
+
+    @Test
+    public void testTwoTuplesOneFile() throws IOException
+    {
+        SequenceFileBolt bolt = makeSeqBolt(hdfsURI, 2, 10000f);
+        bolt.prepare(new Config(), topologyContext, collector);
+        bolt.execute(tuple1);
+
+        verifyZeroInteractions(collector);
+
+        bolt.execute(tuple2);
+        verify(collector).ack(tuple1);
+        verify(collector).ack(tuple2);
+
+        Assert.assertEquals(1, countNonZeroLengthFiles(testRoot));
+    }
+
+    @Test
+    public void testFailedSync() throws IOException
+    {
+        SequenceFileBolt bolt = makeSeqBolt(hdfsURI, 1, .00001f);
+        bolt.prepare(new Config(), topologyContext, collector);
+
+        fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+        // All writes/syncs will fail so this should cause a RuntimeException
+        thrown.expect(RuntimeException.class);
+        bolt.execute(tuple1);
+
+    }
+
+    private SequenceFileBolt makeSeqBolt(String nameNodeAddr, int countSync, float rotationSizeMB) {
+
+        SyncPolicy fieldsSyncPolicy = new CountSyncPolicy(countSync);
+
+        FileRotationPolicy fieldsRotationPolicy =
+                new FileSizeRotationPolicy(rotationSizeMB, FileSizeRotationPolicy.Units.MB);
+
+        FileNameFormat fieldsFileNameFormat = new DefaultFileNameFormat().withPath(testRoot);
+
+        SequenceFormat seqFormat = new DefaultSequenceFormat("key", "value");
+
+        return new SequenceFileBolt()
+                .withFsUrl(nameNodeAddr)
+                .withFileNameFormat(fieldsFileNameFormat)
+                .withRotationPolicy(fieldsRotationPolicy)
+                .withSequenceFormat(seqFormat)
+                .withSyncPolicy(fieldsSyncPolicy);
+    }
+
+    private Tuple generateTestTuple(Long key, String value) {
+        TopologyBuilder builder = new TopologyBuilder();
+        GeneralTopologyContext topologyContext = new GeneralTopologyContext(builder.createTopology(),
+                new Config(), new HashMap(), new HashMap(), new HashMap(), "") {
+            @Override
+            public Fields getComponentOutputFields(String componentId, String streamId) {
+                return new Fields("key", "value");
+            }
+        };
+        return new TupleImpl(topologyContext, new Values(key, value), 1, "");
+    }
+
+    // Generally used to compare how files were actually written and compare to expectations based on total
+    // amount of data written and rotation policies
+    private int countNonZeroLengthFiles(String path) throws IOException {
+        Path p = new Path(path);
+        int nonZero = 0;
+
+        for (FileStatus file : fs.listStatus(p)) {
+            if (file.getLen() > 0) {
+                nonZero++;
+            }
+        }
+
+        return nonZero;
+    }
+
+}
diff --git a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/trident/HdfsStateTest.java b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/trident/HdfsStateTest.java
index e1f45df..51869d2 100644
--- a/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/trident/HdfsStateTest.java
+++ b/external/storm-hdfs/src/test/java/org/apache/storm/hdfs/trident/HdfsStateTest.java
@@ -1,3 +1,20 @@
+/**
+ * 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 org.apache.storm.hdfs.trident;
 
 import backtype.storm.Config;
diff --git a/external/storm-hive/pom.xml b/external/storm-hive/pom.xml
index 6fa4fa9..f842c25 100644
--- a/external/storm-hive/pom.xml
+++ b/external/storm-hive/pom.xml
@@ -45,6 +45,13 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.storm</groupId>
+      <artifactId>storm-core</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hive.hcatalog</groupId>
       <artifactId>hive-hcatalog-streaming</artifactId>
       <version>${hive.version}</version>
diff --git a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java
index e20d31f..d8685b0 100644
--- a/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java
+++ b/external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java
@@ -93,6 +93,15 @@
                                 new ThreadFactoryBuilder().setNameFormat(timeoutName).build());
             heartBeatTimer = new Timer();
             setupHeartBeatTimer();
+
+            // If interval is non-zero then it has already been explicitly set and we should not default it
+            if (conf.containsKey("topology.message.timeout.secs") && options.getTickTupleInterval() == 0)
+            {
+                Integer topologyTimeout = Integer.parseInt(conf.get("topology.message.timeout.secs").toString());
+                int tickTupleInterval = (int) (Math.floor(topologyTimeout / 2));
+                options.withTickTupleInterval(tickTupleInterval);
+                LOG.debug("Setting tick tuple interval to [" + tickTupleInterval + "] based on topology timeout");
+            }
         } catch(Exception e) {
             LOG.warn("unable to make connection to hive ", e);
         }
diff --git a/external/storm-hive/src/test/java/org/apache/storm/hive/bolt/TestHiveBolt.java b/external/storm-hive/src/test/java/org/apache/storm/hive/bolt/TestHiveBolt.java
index 0350c6e..8e79c8c 100644
--- a/external/storm-hive/src/test/java/org/apache/storm/hive/bolt/TestHiveBolt.java
+++ b/external/storm-hive/src/test/java/org/apache/storm/hive/bolt/TestHiveBolt.java
@@ -26,6 +26,7 @@
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.TupleImpl;
 import backtype.storm.tuple.Values;
+import backtype.storm.utils.MockTupleHelpers;
 
 import org.apache.storm.hive.common.HiveOptions;
 import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
@@ -45,7 +46,6 @@
 import org.junit.rules.TemporaryFolder;
 import org.mockito.Mock;
 import org.mockito.Mockito;
-import org.mockito.Spy;
 import org.mockito.MockitoAnnotations;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Mockito.*;
@@ -248,7 +248,6 @@
         bolt = new HiveBolt(hiveOptions);
         bolt.prepare(config, null, new OutputCollector(collector));
         Tuple tuple1 = generateTestTuple(1, "SJC", "Sunnyvale", "CA");
-
         //Tuple tuple2 = generateTestTuple(2,"SFO","San Jose","CA");
         bolt.execute(tuple1);
         verify(collector).ack(tuple1);
@@ -337,6 +336,59 @@
     }
 
     @Test
+    public void testTickTuple()
+    {
+        JsonRecordHiveMapper mapper = new JsonRecordHiveMapper()
+                .withColumnFields(new Fields(colNames1))
+                .withPartitionFields(new Fields(partNames));
+        HiveOptions hiveOptions = new HiveOptions(metaStoreURI,dbName,tblName,mapper)
+                .withTxnsPerBatch(2)
+                .withBatchSize(2);
+
+        bolt = new HiveBolt(hiveOptions);
+        bolt.prepare(config, null, new OutputCollector(collector));
+
+        Tuple tuple1 = generateTestTuple(1, "SJC", "Sunnyvale", "CA");
+        Tuple tuple2 = generateTestTuple(2, "SFO", "San Jose", "CA");
+
+
+        bolt.execute(tuple1);
+
+        //The tick should cause tuple1 to be ack'd
+        Tuple mockTick = MockTupleHelpers.mockTickTuple();
+        bolt.execute(mockTick);
+        verify(collector).ack(tuple1);
+
+        //The second tuple should NOT be ack'd because the batch should be cleared and this will be
+        //the first transaction in the new batch
+        bolt.execute(tuple2);
+        verify(collector, never()).ack(tuple2);
+
+        bolt.cleanup();
+    }
+
+    @Test
+    public void testNoTickEmptyBatches() throws Exception
+    {
+        JsonRecordHiveMapper mapper = new JsonRecordHiveMapper()
+                .withColumnFields(new Fields(colNames1))
+                .withPartitionFields(new Fields(partNames));
+        HiveOptions hiveOptions = new HiveOptions(metaStoreURI,dbName,tblName,mapper)
+                .withTxnsPerBatch(2)
+                .withBatchSize(2);
+
+        bolt = new HiveBolt(hiveOptions);
+        bolt.prepare(config, null, new OutputCollector(collector));
+
+        //The tick should NOT cause any acks since the batch was empty
+        Tuple mockTick = MockTupleHelpers.mockTickTuple();
+        bolt.execute(mockTick);
+        verifyZeroInteractions(collector);
+
+        bolt.cleanup();
+    }
+
+    @Test
     public void testMultiPartitionTuples()
         throws Exception {
         DelimitedRecordHiveMapper mapper = new DelimitedRecordHiveMapper()
diff --git a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/AbstractJdbcBolt.java b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/AbstractJdbcBolt.java
index 15a2345..0c0cca6 100644
--- a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/AbstractJdbcBolt.java
+++ b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/AbstractJdbcBolt.java
@@ -21,6 +21,7 @@
 import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.base.BaseRichBolt;
+import org.apache.commons.lang.Validate;
 import org.apache.storm.jdbc.common.ConnectionProvider;
 import org.apache.storm.jdbc.common.JdbcClient;
 import org.slf4j.Logger;
@@ -52,6 +53,7 @@
     }
 
     public AbstractJdbcBolt(ConnectionProvider connectionProvider) {
+        Validate.notNull(connectionProvider);
         this.connectionProvider = connectionProvider;
     }
 
diff --git a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcInsertBolt.java b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcInsertBolt.java
index 2f29000..c3328f1 100644
--- a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcInsertBolt.java
+++ b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcInsertBolt.java
@@ -20,6 +20,7 @@
 import backtype.storm.task.TopologyContext;
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Tuple;
+import org.apache.commons.lang.Validate;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.storm.jdbc.common.Column;
 import org.apache.storm.jdbc.common.ConnectionProvider;
@@ -45,15 +46,23 @@
 
     public JdbcInsertBolt(ConnectionProvider connectionProvider,  JdbcMapper jdbcMapper) {
         super(connectionProvider);
+
+        Validate.notNull(jdbcMapper);
         this.jdbcMapper = jdbcMapper;
     }
 
     public JdbcInsertBolt withTableName(String tableName) {
+        if (insertQuery != null) {
+            throw new IllegalArgumentException("You can not specify both insertQuery and tableName.");
+        }
         this.tableName = tableName;
         return this;
     }
 
     public JdbcInsertBolt withInsertQuery(String insertQuery) {
+        if (this.tableName != null) {
+            throw new IllegalArgumentException("You can not specify both insertQuery and tableName.");
+        }
         this.insertQuery = insertQuery;
         return this;
     }
diff --git a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcLookupBolt.java b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcLookupBolt.java
index 25122e2..b1dadb7 100644
--- a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcLookupBolt.java
+++ b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/bolt/JdbcLookupBolt.java
@@ -20,6 +20,7 @@
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.Values;
+import org.apache.commons.lang.Validate;
 import org.apache.storm.jdbc.common.Column;
 import org.apache.storm.jdbc.common.ConnectionProvider;
 import org.apache.storm.jdbc.mapper.JdbcLookupMapper;
@@ -40,6 +41,10 @@
 
     public JdbcLookupBolt(ConnectionProvider connectionProvider, String selectQuery, JdbcLookupMapper jdbcLookupMapper) {
         super(connectionProvider);
+
+        Validate.notNull(selectQuery);
+        Validate.notNull(jdbcLookupMapper);
+
         this.selectQuery = selectQuery;
         this.jdbcLookupMapper = jdbcLookupMapper;
     }
diff --git a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcLookupMapper.java b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcLookupMapper.java
index 5a22552..b267bd1 100644
--- a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcLookupMapper.java
+++ b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcLookupMapper.java
@@ -22,6 +22,7 @@
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.ITuple;
 import backtype.storm.tuple.Values;
+import org.apache.commons.lang.Validate;
 import org.apache.storm.jdbc.common.Column;
 
 import java.util.ArrayList;
@@ -33,6 +34,8 @@
 
     public SimpleJdbcLookupMapper(Fields outputFields, List<Column> queryColumns) {
         super(queryColumns);
+
+        Validate.notEmpty(outputFields.toList());
         this.outputFields = outputFields;
     }
 
diff --git a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcMapper.java b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcMapper.java
index c4005e3..9befb1e 100644
--- a/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcMapper.java
+++ b/external/storm-jdbc/src/main/java/org/apache/storm/jdbc/mapper/SimpleJdbcMapper.java
@@ -18,6 +18,7 @@
 package org.apache.storm.jdbc.mapper;
 
 import backtype.storm.tuple.ITuple;
+import org.apache.commons.lang.Validate;
 import org.apache.storm.jdbc.common.Column;
 import org.apache.storm.jdbc.common.ConnectionProvider;
 import org.apache.storm.jdbc.common.JdbcClient;
@@ -35,6 +36,9 @@
     private List<Column> schemaColumns;
 
     public SimpleJdbcMapper(String tableName, ConnectionProvider connectionProvider) {
+        Validate.notEmpty(tableName);
+        Validate.notNull(connectionProvider);
+
         int queryTimeoutSecs = 30;
         connectionProvider.prepare();
         JdbcClient client = new JdbcClient(connectionProvider, queryTimeoutSecs);
@@ -42,6 +46,7 @@
     }
 
     public SimpleJdbcMapper(List<Column> schemaColumns) {
+        Validate.notEmpty(schemaColumns);
         this.schemaColumns = schemaColumns;
     }
 
diff --git a/external/storm-jdbc/src/test/java/org/apache/storm/jdbc/bolt/JdbcInsertBoltTest.java b/external/storm-jdbc/src/test/java/org/apache/storm/jdbc/bolt/JdbcInsertBoltTest.java
new file mode 100644
index 0000000..1b393e9
--- /dev/null
+++ b/external/storm-jdbc/src/test/java/org/apache/storm/jdbc/bolt/JdbcInsertBoltTest.java
@@ -0,0 +1,71 @@
+/**
+ * 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 org.apache.storm.jdbc.bolt;
+
+import com.google.common.collect.Lists;
+import org.apache.storm.jdbc.common.Column;
+import org.apache.storm.jdbc.common.ConnectionProvider;
+import org.apache.storm.jdbc.common.HikariCPConnectionProvider;
+import org.apache.storm.jdbc.mapper.JdbcMapper;
+import org.apache.storm.jdbc.mapper.SimpleJdbcMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+
+/**
+ * Created by pbrahmbhatt on 10/29/15.
+ */
+public class JdbcInsertBoltTest {
+
+    @Test
+    public void testValidation() {
+        ConnectionProvider provider = new HikariCPConnectionProvider(new HashMap<String, Object>());
+        JdbcMapper mapper = new SimpleJdbcMapper(Lists.newArrayList(new Column("test", 0)));
+        expectIllegaArgs(null, mapper);
+        expectIllegaArgs(provider, null);
+
+        try {
+            JdbcInsertBolt bolt = new JdbcInsertBolt(provider, mapper);
+            bolt.withInsertQuery("test");
+            bolt.withTableName("test");
+            Assert.fail("Should have thrown IllegalArgumentException.");
+        } catch(IllegalArgumentException ne) {
+            //expected
+        }
+
+        try {
+            JdbcInsertBolt bolt = new JdbcInsertBolt(provider, mapper);
+            bolt.withTableName("test");
+            bolt.withInsertQuery("test");
+            Assert.fail("Should have thrown IllegalArgumentException.");
+        } catch(IllegalArgumentException ne) {
+            //expected
+        }
+    }
+
+    private void expectIllegaArgs(ConnectionProvider provider, JdbcMapper mapper) {
+        try {
+            JdbcInsertBolt bolt = new JdbcInsertBolt(provider, mapper);
+            Assert.fail("Should have thrown IllegalArgumentException.");
+        } catch(IllegalArgumentException ne) {
+            //expected
+        }
+    }
+
+}
diff --git a/external/storm-jdbc/src/test/java/org/apache/storm/jdbc/bolt/JdbcLookupBoltTest.java b/external/storm-jdbc/src/test/java/org/apache/storm/jdbc/bolt/JdbcLookupBoltTest.java
new file mode 100644
index 0000000..1fda3b1
--- /dev/null
+++ b/external/storm-jdbc/src/test/java/org/apache/storm/jdbc/bolt/JdbcLookupBoltTest.java
@@ -0,0 +1,59 @@
+/**
+ * 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 org.apache.storm.jdbc.bolt;
+
+import backtype.storm.tuple.Fields;
+import com.google.common.collect.Lists;
+import org.apache.storm.jdbc.common.Column;
+import org.apache.storm.jdbc.common.ConnectionProvider;
+import org.apache.storm.jdbc.common.HikariCPConnectionProvider;
+import org.apache.storm.jdbc.mapper.JdbcLookupMapper;
+import org.apache.storm.jdbc.mapper.JdbcMapper;
+import org.apache.storm.jdbc.mapper.SimpleJdbcLookupMapper;
+import org.apache.storm.jdbc.mapper.SimpleJdbcMapper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+
+/**
+ * Created by pbrahmbhatt on 10/29/15.
+ */
+public class JdbcLookupBoltTest {
+
+    @Test
+    public void testValidation() {
+        ConnectionProvider provider = new HikariCPConnectionProvider(new HashMap<String, Object>());
+        JdbcLookupMapper mapper = new SimpleJdbcLookupMapper(new Fields("test"), Lists.newArrayList(new Column("test", 0)));
+        String selectQuery = "select * from dual";
+        expectIllegaArgs(null, selectQuery, mapper);
+        expectIllegaArgs(provider, null, mapper);
+        expectIllegaArgs(provider, selectQuery, null);
+    }
+
+    private void expectIllegaArgs(ConnectionProvider provider, String selectQuery, JdbcLookupMapper mapper) {
+        try {
+            JdbcLookupBolt bolt = new JdbcLookupBolt(provider, selectQuery, mapper);
+            Assert.fail("Should have thrown IllegalArgumentException.");
+        } catch(IllegalArgumentException ne) {
+            //expected
+        }
+    }
+
+}
diff --git a/external/storm-kafka/README.md b/external/storm-kafka/README.md
index 91c64bf..2fe930e 100644
--- a/external/storm-kafka/README.md
+++ b/external/storm-kafka/README.md
@@ -6,16 +6,16 @@
 ##Spouts
 We support both Trident and core Storm spouts. For both spout implementations, we use a BrokerHost interface that
 tracks Kafka broker host to partition mapping and kafkaConfig that controls some Kafka related parameters.
- 
+
 ###BrokerHosts
-In order to initialize your Kafka spout/emitter you need to construct an instance of the marker interface BrokerHosts. 
+In order to initialize your Kafka spout/emitter you need to construct an instance of the marker interface BrokerHosts.
 Currently, we support the following two implementations:
 
 ####ZkHosts
-ZkHosts is what you should use if you want to dynamically track Kafka broker to partition mapping. This class uses 
+ZkHosts is what you should use if you want to dynamically track Kafka broker to partition mapping. This class uses
 Kafka's ZooKeeper entries to track brokerHost -> partition mapping. You can instantiate an object by calling
 ```java
-    public ZkHosts(String brokerZkStr, String brokerZkPath) 
+    public ZkHosts(String brokerZkStr, String brokerZkPath)
     public ZkHosts(String brokerZkStr)
 ```
 Where brokerZkStr is just ip:port (e.g. localhost:2181). brokerZkPath is the root directory under which all the topics and
@@ -40,7 +40,7 @@
 ```
 
 ###KafkaConfig
-The second thing needed for constructing a kafkaSpout is an instance of KafkaConfig. 
+The second thing needed for constructing a kafkaSpout is an instance of KafkaConfig.
 ```java
     public KafkaConfig(BrokerHosts hosts, String topic)
     public KafkaConfig(BrokerHosts hosts, String topic, String clientId)
@@ -103,9 +103,17 @@
   public Fields getOutputFields();
 ```
 
-The default `RawMultiScheme` just takes the `byte[]` and returns a tuple with `byte[]` as is. The name of the
-outputField is "bytes".  There are alternative implementation like `SchemeAsMultiScheme` and
-`KeyValueSchemeAsMultiScheme` which can convert the `byte[]` to `String`.
+The default `RawMultiScheme` just takes the `byte[]` and returns a tuple with `byte[]` as is. The name of the outputField is "bytes". There are alternative implementations like `SchemeAsMultiScheme` and `KeyValueSchemeAsMultiScheme` which can convert the `byte[]` to `String`.
+
+There is also an extension of `SchemeAsMultiScheme`, `MessageMetadataSchemeAsMultiScheme`,
+which has an additional deserialize method that accepts the message `byte[]` in addition to the `Partition` and `offset` associated with the message.
+
+```java
+public Iterable<List<Object>> deserializeMessageWithMetadata(byte[] message, Partition partition, long offset)
+
+```
+
+This is useful for auditing/replaying messages from arbitrary points on a Kafka topic, saving the partition and offset of each message of a discrete stream instead of persisting the entire message.
 
 
 ### Examples
@@ -184,7 +192,7 @@
 Note that the ZooKeeper and log4j dependencies are excluded to prevent version conflicts with Storm's dependencies.
 
 ##Writing to Kafka as part of your topology
-You can create an instance of storm.kafka.bolt.KafkaBolt and attach it as a component to your topology or if you 
+You can create an instance of storm.kafka.bolt.KafkaBolt and attach it as a component to your topology or if you
 are using trident you can use storm.kafka.trident.TridentState, storm.kafka.trident.TridentStateFactory and
 storm.kafka.trident.TridentKafkaUpdater.
 
@@ -199,9 +207,9 @@
 ```
 
 As the name suggests, these methods are called to map a tuple to Kafka key and Kafka message. If you just want one field
-as key and one field as value, then you can use the provided FieldNameBasedTupleToKafkaMapper.java 
-implementation. In the KafkaBolt, the implementation always looks for a field with field name "key" and "message" if you 
-use the default constructor to construct FieldNameBasedTupleToKafkaMapper for backward compatibility 
+as key and one field as value, then you can use the provided FieldNameBasedTupleToKafkaMapper.java
+implementation. In the KafkaBolt, the implementation always looks for a field with field name "key" and "message" if you
+use the default constructor to construct FieldNameBasedTupleToKafkaMapper for backward compatibility
 reasons. Alternatively you could also specify a different key and message field by using the non default constructor.
 In the TridentKafkaState you must specify what is the field name for key and message as there is no default constructor.
 These should be specified while constructing and instance of FieldNameBasedTupleToKafkaMapper.
@@ -213,21 +221,31 @@
     String getTopics(Tuple/TridentTuple tuple);
 }
 ```
-The implementation of this interface should return the topic to which the tuple's key/message mapping needs to be published 
-You can return a null and the message will be ignored. If you have one static topic name then you can use 
+The implementation of this interface should return the topic to which the tuple's key/message mapping needs to be published
+You can return a null and the message will be ignored. If you have one static topic name then you can use
 DefaultTopicSelector.java and set the name of the topic in the constructor.
 
 ### Specifying Kafka producer properties
-You can provide all the produce properties , see http://kafka.apache.org/documentation.html#producerconfigs 
-section "Important configuration properties for the producer", in your Storm topology config by setting the properties
-map with key kafka.broker.properties.
+You can provide all the produce properties in your Storm topology by calling `KafkaBolt.withProducerProperties()` and `TridentKafkaStateFactory.withProducerProperties()`. Please see  http://kafka.apache.org/documentation.html#newproducerconfigs
+Section "Important configuration properties for the producer" for more details.
+
+###Using wildcard kafka topic match
+You can do a wildcard topic match by adding the following config
+```
+     Config config = new Config();
+     config.put("kafka.topic.wildcard.match",true);
+
+```
+
+After this you can specify a wildcard topic for matching e.g. clickstream.*.log.  This will match all streams matching clickstream.my.log, clickstream.cart.log etc
+
 
 ###Putting it all together
 
 For the bolt :
 ```java
         TopologyBuilder builder = new TopologyBuilder();
-    
+
         Fields fields = new Fields("key", "message");
         FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
                     new Values("storm", "1"),
@@ -237,19 +255,21 @@
         );
         spout.setCycle(true);
         builder.setSpout("spout", spout, 5);
+        //set producer properties.
+        Properties props = new Properties();
+        props.put("bootstrap.servers", "localhost:9092");
+        props.put("acks", "1");
+        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+
         KafkaBolt bolt = new KafkaBolt()
+                .withProducerProperties(props)
                 .withTopicSelector(new DefaultTopicSelector("test"))
                 .withTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper());
         builder.setBolt("forwardToKafka", bolt, 8).shuffleGrouping("spout");
-        
+
         Config conf = new Config();
-        //set producer properties.
-        Properties props = new Properties();
-        props.put("metadata.broker.list", "localhost:9092");
-        props.put("request.required.acks", "1");
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
-        conf.put(KafkaBolt.KAFKA_BROKER_PROPERTIES, props);
-        
+
         StormSubmitter.submitTopology("kafkaboltTest", conf, builder.createTopology());
 ```
 
@@ -268,18 +288,20 @@
         TridentTopology topology = new TridentTopology();
         Stream stream = topology.newStream("spout1", spout);
 
+        //set producer properties.
+        Properties props = new Properties();
+        props.put("bootstrap.servers", "localhost:9092");
+        props.put("acks", "1");
+        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+
         TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
+                .withProducerProperties(props)
                 .withKafkaTopicSelector(new DefaultTopicSelector("test"))
                 .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count"));
         stream.partitionPersist(stateFactory, fields, new TridentKafkaUpdater(), new Fields());
 
         Config conf = new Config();
-        //set producer properties.
-        Properties props = new Properties();
-        props.put("metadata.broker.list", "localhost:9092");
-        props.put("request.required.acks", "1");
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
-        conf.put(TridentKafkaState.KAFKA_BROKER_PROPERTIES, props);
         StormSubmitter.submitTopology("kafkaTridentTest", conf, topology.build());
 ```
 
diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml
index c731a0f..8f2867b 100644
--- a/external/storm-kafka/pom.xml
+++ b/external/storm-kafka/pom.xml
@@ -57,6 +57,11 @@
     </build>
     <dependencies>
         <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
             <version>1.9.0</version>
diff --git a/external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java b/external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java
index d379061..d0f6724 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java
@@ -30,9 +30,11 @@
 
 import java.io.UnsupportedEncodingException;
 import java.net.SocketTimeoutException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+
 public class DynamicBrokersReader {
 
     public static final Logger LOG = LoggerFactory.getLogger(DynamicBrokersReader.class);
@@ -40,6 +42,7 @@
     private CuratorFramework _curator;
     private String _zkPath;
     private String _topic;
+    private Boolean _isWildcardTopic;
 
     public DynamicBrokersReader(Map conf, String zkStr, String zkPath, String topic) {
         // Check required parameters
@@ -53,6 +56,7 @@
 
         _zkPath = zkPath;
         _topic = topic;
+        _isWildcardTopic = Utils.getBoolean(conf.get("kafka.topic.wildcard.match"), false);
         try {
             _curator = CuratorFrameworkFactory.newClient(
                     zkStr,
@@ -70,35 +74,40 @@
     /**
      * Get all partitions with their current leaders
      */
-    public GlobalPartitionInformation getBrokerInfo() throws SocketTimeoutException {
-      GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation();
-        try {
-            int numPartitionsForTopic = getNumPartitions();
-            String brokerInfoPath = brokerPath();
-            for (int partition = 0; partition < numPartitionsForTopic; partition++) {
-                int leader = getLeaderFor(partition);
-                String path = brokerInfoPath + "/" + leader;
-                try {
-                    byte[] brokerData = _curator.getData().forPath(path);
-                    Broker hp = getBrokerHost(brokerData);
-                    globalPartitionInformation.addPartition(partition, hp);
-                } catch (org.apache.zookeeper.KeeperException.NoNodeException e) {
-                    LOG.error("Node {} does not exist ", path);
-                }
-            }
-        } catch (SocketTimeoutException e) {
-					throw e;
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-        LOG.info("Read partition info from zookeeper: " + globalPartitionInformation);
-        return globalPartitionInformation;
+    public List<GlobalPartitionInformation> getBrokerInfo() throws SocketTimeoutException {
+      List<String> topics =  getTopics();
+      List<GlobalPartitionInformation> partitions =  new ArrayList<GlobalPartitionInformation>();
+
+      for (String topic : topics) {
+          GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(topic, this._isWildcardTopic);
+          try {
+              int numPartitionsForTopic = getNumPartitions(topic);
+              String brokerInfoPath = brokerPath();
+              for (int partition = 0; partition < numPartitionsForTopic; partition++) {
+                  int leader = getLeaderFor(topic,partition);
+                  String path = brokerInfoPath + "/" + leader;
+                  try {
+                      byte[] brokerData = _curator.getData().forPath(path);
+                      Broker hp = getBrokerHost(brokerData);
+                      globalPartitionInformation.addPartition(partition, hp);
+                  } catch (org.apache.zookeeper.KeeperException.NoNodeException e) {
+                      LOG.error("Node {} does not exist ", path);
+                  }
+              }
+          } catch (SocketTimeoutException e) {
+              throw e;
+          } catch (Exception e) {
+              throw new RuntimeException(e);
+          }
+          LOG.info("Read partition info from zookeeper: " + globalPartitionInformation);
+          partitions.add(globalPartitionInformation);
+      }
+        return partitions;
     }
 
-
-    private int getNumPartitions() {
+    private int getNumPartitions(String topic) {
         try {
-            String topicBrokersPath = partitionPath();
+            String topicBrokersPath = partitionPath(topic);
             List<String> children = _curator.getChildren().forPath(topicBrokersPath);
             return children.size();
         } catch (Exception e) {
@@ -106,24 +115,50 @@
         }
     }
 
-    public String partitionPath() {
-        return _zkPath + "/topics/" + _topic + "/partitions";
+    private List<String> getTopics() {
+        List<String> topics = new ArrayList<String>();
+        if (!_isWildcardTopic) {
+            topics.add(_topic);
+            return topics;
+        } else {
+            try {
+                List<String> children = _curator.getChildren().forPath(topicsPath());
+                for (String t : children) {
+                    if (t.matches(_topic)) {
+                        LOG.info(String.format("Found matching topic %s", t));
+                        topics.add(t);
+                    }
+                }
+                return topics;
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    public String topicsPath () {
+        return _zkPath + "/topics";
+    }
+    public String partitionPath(String topic) {
+        return topicsPath() + "/" + topic + "/partitions";
     }
 
     public String brokerPath() {
         return _zkPath + "/ids";
     }
 
+
+
     /**
      * get /brokers/topics/distributedTopic/partitions/1/state
      * { "controller_epoch":4, "isr":[ 1, 0 ], "leader":1, "leader_epoch":1, "version":1 }
-     *
+     * @param topic
      * @param partition
      * @return
      */
-    private int getLeaderFor(long partition) {
+    private int getLeaderFor(String topic, long partition) {
         try {
-            String topicBrokersPath = partitionPath();
+            String topicBrokersPath = partitionPath(topic);
             byte[] hostPortData = _curator.getData().forPath(topicBrokersPath + "/" + partition + "/state");
             Map<Object, Object> value = (Map<Object, Object>) JSONValue.parse(new String(hostPortData, "UTF-8"));
             Integer leader = ((Number) value.get("leader")).intValue();
diff --git a/external/storm-kafka/src/jvm/storm/kafka/DynamicPartitionConnections.java b/external/storm-kafka/src/jvm/storm/kafka/DynamicPartitionConnections.java
index f550858..e237a7a 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/DynamicPartitionConnections.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/DynamicPartitionConnections.java
@@ -34,7 +34,7 @@
 
     static class ConnectionInfo {
         SimpleConsumer consumer;
-        Set<Integer> partitions = new HashSet();
+        Set<String> partitions = new HashSet<String>();
 
         public ConnectionInfo(SimpleConsumer consumer) {
             this.consumer = consumer;
@@ -51,16 +51,16 @@
     }
 
     public SimpleConsumer register(Partition partition) {
-        Broker broker = _reader.getCurrentBrokers().getBrokerFor(partition.partition);
-        return register(broker, partition.partition);
+        Broker broker = _reader.getBrokerForTopic(partition.topic).getBrokerFor(partition.partition);
+        return register(broker, partition.topic, partition.partition);
     }
 
-    public SimpleConsumer register(Broker host, int partition) {
+    public SimpleConsumer register(Broker host, String topic, int partition) {
         if (!_connections.containsKey(host)) {
             _connections.put(host, new ConnectionInfo(new SimpleConsumer(host.host, host.port, _config.socketTimeoutMs, _config.bufferSizeBytes, _config.clientId)));
         }
         ConnectionInfo info = _connections.get(host);
-        info.partitions.add(partition);
+        info.partitions.add(getHashKey(topic,partition));
         return info.consumer;
     }
 
@@ -72,9 +72,9 @@
         return null;
     }
 
-    public void unregister(Broker port, int partition) {
+    public void unregister(Broker port, String topic, int partition) {
         ConnectionInfo info = _connections.get(port);
-        info.partitions.remove(partition);
+        info.partitions.remove(getHashKey(topic,partition));
         if (info.partitions.isEmpty()) {
             info.consumer.close();
             _connections.remove(port);
@@ -82,7 +82,7 @@
     }
 
     public void unregister(Partition partition) {
-        unregister(partition.host, partition.partition);
+        unregister(partition.host, partition.topic, partition.partition);
     }
 
     public void clear() {
@@ -91,4 +91,8 @@
         }
         _connections.clear();
     }
+
+    private String getHashKey(String topic, int partition) {
+        return topic + "_" + partition;
+    }
 }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java
index dd71b5a..49c7526 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java
@@ -23,7 +23,8 @@
 import java.io.Serializable;
 
 public class KafkaConfig implements Serializable {
-
+    private static final long serialVersionUID = 5276718734571623855L;
+    
     public final BrokerHosts hosts;
     public final String topic;
     public final String clientId;
diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaSpout.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaSpout.java
index 3260ad1..d16659f 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/KafkaSpout.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaSpout.java
@@ -6,9 +6,9 @@
  * 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.
@@ -51,7 +51,6 @@
 
     public static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class);
 
-    String _uuid = UUID.randomUUID().toString();
     SpoutConfig _spoutConfig;
     SpoutOutputCollector _collector;
     PartitionCoordinator _coordinator;
@@ -69,7 +68,7 @@
     @Override
     public void open(Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
         _collector = collector;
-
+        String topologyInstanceId = context.getStormId();
         Map stateConf = new HashMap(conf);
         List<String> zkServers = _spoutConfig.zkServers;
         if (zkServers == null) {
@@ -89,13 +88,17 @@
         // using TransactionalState like this is a hack
         int totalTasks = context.getComponentTasks(context.getThisComponentId()).size();
         if (_spoutConfig.hosts instanceof StaticHosts) {
-            _coordinator = new StaticCoordinator(_connections, conf, _spoutConfig, _state, context.getThisTaskIndex(), totalTasks, _uuid);
+            _coordinator = new StaticCoordinator(_connections, conf,
+                    _spoutConfig, _state, context.getThisTaskIndex(),
+                    totalTasks, topologyInstanceId);
         } else {
-            _coordinator = new ZkCoordinator(_connections, conf, _spoutConfig, _state, context.getThisTaskIndex(), totalTasks, _uuid);
+            _coordinator = new ZkCoordinator(_connections, conf,
+                    _spoutConfig, _state, context.getThisTaskIndex(),
+                    totalTasks, topologyInstanceId);
         }
 
         context.registerMetric("kafkaOffset", new IMetric() {
-            KafkaUtils.KafkaOffsetMetric _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_spoutConfig.topic, _connections);
+            KafkaUtils.KafkaOffsetMetric _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_connections);
 
             @Override
             public Object getValueAndReset() {
@@ -151,8 +154,13 @@
             }
         }
 
-        long now = System.currentTimeMillis();
-        if ((now - _lastUpdateMs) > _spoutConfig.stateUpdateIntervalMs) {
+        long diffWithNow = System.currentTimeMillis() - _lastUpdateMs;
+
+        /*
+             As far as the System.currentTimeMillis() is dependent on System clock,
+             additional check on negative value of diffWithNow in case of external changes.
+         */
+        if (diffWithNow > _spoutConfig.stateUpdateIntervalMs || diffWithNow < 0) {
             commit();
         }
     }
@@ -182,11 +190,11 @@
 
     @Override
     public void declareOutputFields(OutputFieldsDeclarer declarer) {
-	if (_spoutConfig.topicAsStreamId) {
-	    declarer.declareStream(_spoutConfig.topic, _spoutConfig.scheme.getOutputFields());
-	} else {
+        if (_spoutConfig.topicAsStreamId) {
+            declarer.declareStream(_spoutConfig.topic, _spoutConfig.scheme.getOutputFields());
+        } else {
             declarer.declare(_spoutConfig.scheme.getOutputFields());
-	}
+        }
     }
 
     private void commit() {
diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java
index b324d79..52cdde8 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java
@@ -52,7 +52,7 @@
 
     public static IBrokerReader makeBrokerReader(Map stormConf, KafkaConfig conf) {
         if (conf.hosts instanceof StaticHosts) {
-            return new StaticBrokerReader(((StaticHosts) conf.hosts).getPartitionInformation());
+            return new StaticBrokerReader(conf.topic, ((StaticHosts) conf.hosts).getPartitionInformation());
         } else {
             return new ZkBrokerReader(stormConf, conf.topic, (ZkHosts) conf.hosts);
         }
@@ -82,11 +82,9 @@
     public static class KafkaOffsetMetric implements IMetric {
         Map<Partition, Long> _partitionToOffset = new HashMap<Partition, Long>();
         Set<Partition> _partitions;
-        String _topic;
         DynamicPartitionConnections _connections;
 
-        public KafkaOffsetMetric(String topic, DynamicPartitionConnections connections) {
-            _topic = topic;
+        public KafkaOffsetMetric(DynamicPartitionConnections connections) {
             _connections = connections;
         }
 
@@ -94,15 +92,19 @@
             _partitionToOffset.put(partition, offset);
         }
 
+        private class TopicMetrics {
+            long totalSpoutLag = 0;
+            long totalEarliestTimeOffset = 0;
+            long totalLatestTimeOffset = 0;
+            long totalLatestEmittedOffset = 0;
+        }
+
         @Override
         public Object getValueAndReset() {
             try {
-                long totalSpoutLag = 0;
-                long totalEarliestTimeOffset = 0;
-                long totalLatestTimeOffset = 0;
-                long totalLatestEmittedOffset = 0;
                 HashMap ret = new HashMap();
                 if (_partitions != null && _partitions.size() == _partitionToOffset.size()) {
+                    Map<String,TopicMetrics> topicMetricsMap = new TreeMap<String, TopicMetrics>();
                     for (Map.Entry<Partition, Long> e : _partitionToOffset.entrySet()) {
                         Partition partition = e.getKey();
                         SimpleConsumer consumer = _connections.getConnection(partition);
@@ -110,27 +112,45 @@
                             LOG.warn("partitionToOffset contains partition not found in _connections. Stale partition data?");
                             return null;
                         }
-                        long latestTimeOffset = getOffset(consumer, _topic, partition.partition, kafka.api.OffsetRequest.LatestTime());
-                        long earliestTimeOffset = getOffset(consumer, _topic, partition.partition, kafka.api.OffsetRequest.EarliestTime());
+                        long latestTimeOffset = getOffset(consumer, partition.topic, partition.partition, kafka.api.OffsetRequest.LatestTime());
+                        long earliestTimeOffset = getOffset(consumer, partition.topic, partition.partition, kafka.api.OffsetRequest.EarliestTime());
                         if (latestTimeOffset == KafkaUtils.NO_OFFSET) {
                             LOG.warn("No data found in Kafka Partition " + partition.getId());
                             return null;
                         }
                         long latestEmittedOffset = e.getValue();
                         long spoutLag = latestTimeOffset - latestEmittedOffset;
-                        ret.put(_topic + "/" + partition.getId() + "/" + "spoutLag", spoutLag);
-                        ret.put(_topic + "/" + partition.getId() + "/" + "earliestTimeOffset", earliestTimeOffset);
-                        ret.put(_topic + "/" + partition.getId() + "/" + "latestTimeOffset", latestTimeOffset);
-                        ret.put(_topic + "/" + partition.getId() + "/" + "latestEmittedOffset", latestEmittedOffset);
-                        totalSpoutLag += spoutLag;
-                        totalEarliestTimeOffset += earliestTimeOffset;
-                        totalLatestTimeOffset += latestTimeOffset;
-                        totalLatestEmittedOffset += latestEmittedOffset;
+                        String topic = partition.topic;
+                        String metricPath = partition.getId();
+                        //Handle the case where Partition Path Id does not contain topic name Partition.getId() == "partition_" + partition
+                        if (!metricPath.startsWith(topic + "/")) {
+                            metricPath = topic + "/" + metricPath;
+                        }
+                        ret.put(metricPath + "/" + "spoutLag", spoutLag);
+                        ret.put(metricPath + "/" + "earliestTimeOffset", earliestTimeOffset);
+                        ret.put(metricPath + "/" + "latestTimeOffset", latestTimeOffset);
+                        ret.put(metricPath + "/" + "latestEmittedOffset", latestEmittedOffset);
+
+                        if (!topicMetricsMap.containsKey(partition.topic)) {
+                            topicMetricsMap.put(partition.topic,new TopicMetrics());
+                        }
+
+                        TopicMetrics topicMetrics = topicMetricsMap.get(partition.topic);
+                        topicMetrics.totalSpoutLag += spoutLag;
+                        topicMetrics.totalEarliestTimeOffset += earliestTimeOffset;
+                        topicMetrics.totalLatestTimeOffset += latestTimeOffset;
+                        topicMetrics.totalLatestEmittedOffset += latestEmittedOffset;
                     }
-                    ret.put(_topic + "/" + "totalSpoutLag", totalSpoutLag);
-                    ret.put(_topic + "/" + "totalEarliestTimeOffset", totalEarliestTimeOffset);
-                    ret.put(_topic + "/" + "totalLatestTimeOffset", totalLatestTimeOffset);
-                    ret.put(_topic + "/" + "totalLatestEmittedOffset", totalLatestEmittedOffset);
+
+                    for(Map.Entry<String, TopicMetrics> e : topicMetricsMap.entrySet()) {
+                        String topic = e.getKey();
+                        TopicMetrics topicMetrics = e.getValue();
+                        ret.put(topic + "/" + "totalSpoutLag", topicMetrics.totalSpoutLag);
+                        ret.put(topic + "/" + "totalEarliestTimeOffset", topicMetrics.totalEarliestTimeOffset);
+                        ret.put(topic + "/" + "totalLatestTimeOffset", topicMetrics.totalLatestTimeOffset);
+                        ret.put(topic + "/" + "totalLatestEmittedOffset", topicMetrics.totalLatestEmittedOffset);
+                    }
+
                     return ret;
                 } else {
                     LOG.info("Metrics Tick: Not enough data to calculate spout lag.");
@@ -155,7 +175,7 @@
     public static ByteBufferMessageSet fetchMessages(KafkaConfig config, SimpleConsumer consumer, Partition partition, long offset)
             throws TopicOffsetOutOfRangeException, FailedFetchException,RuntimeException {
         ByteBufferMessageSet msgs = null;
-        String topic = config.topic;
+        String topic = partition.topic;
         int partitionId = partition.partition;
         FetchRequestBuilder builder = new FetchRequestBuilder();
         FetchRequest fetchRequest = builder.addFetch(topic, partitionId, offset, config.fetchSizeBytes).
@@ -178,7 +198,7 @@
         if (fetchResponse.hasError()) {
             KafkaError error = KafkaError.getError(fetchResponse.errorCode(topic, partitionId));
             if (error.equals(KafkaError.OFFSET_OUT_OF_RANGE) && config.useStartOffsetTimeIfOffsetOutOfRange) {
-                String msg = "Got fetch request with offset out of range: [" + offset + "]";
+                String msg = partition + " Got fetch request with offset out of range: [" + offset + "]";
                 LOG.warn(msg);
                 throw new TopicOffsetOutOfRangeException(msg);
             } else {
@@ -193,7 +213,7 @@
     }
 
 
-    public static Iterable<List<Object>> generateTuples(KafkaConfig kafkaConfig, Message msg) {
+    public static Iterable<List<Object>> generateTuples(KafkaConfig kafkaConfig, Message msg, String topic) {
         Iterable<List<Object>> tups;
         ByteBuffer payload = msg.payload();
         if (payload == null) {
@@ -201,22 +221,37 @@
         }
         ByteBuffer key = msg.key();
         if (key != null && kafkaConfig.scheme instanceof KeyValueSchemeAsMultiScheme) {
-            tups = ((KeyValueSchemeAsMultiScheme) kafkaConfig.scheme).deserializeKeyAndValue(Utils.toByteArray(key), Utils.toByteArray(payload));
+            tups = ((KeyValueSchemeAsMultiScheme) kafkaConfig.scheme).deserializeKeyAndValue(key, payload);
         } else {
-            tups = kafkaConfig.scheme.deserialize(Utils.toByteArray(payload));
+            if (kafkaConfig.scheme instanceof StringMultiSchemeWithTopic) {
+                tups = ((StringMultiSchemeWithTopic)kafkaConfig.scheme).deserializeWithTopic(topic, payload);
+            } else {
+                tups = kafkaConfig.scheme.deserialize(payload);
+            }
         }
         return tups;
     }
+    
+    public static Iterable<List<Object>> generateTuples(MessageMetadataSchemeAsMultiScheme scheme, Message msg, Partition partition, long offset) {
+        ByteBuffer payload = msg.payload();
+        if (payload == null) {
+            return null;
+        }
+        return scheme.deserializeMessageWithMetadata(payload, partition, offset);
+    }
 
 
-    public static List<Partition> calculatePartitionsForTask(GlobalPartitionInformation partitionInformation, int totalTasks, int taskIndex) {
+    public static List<Partition> calculatePartitionsForTask(List<GlobalPartitionInformation> partitons, int totalTasks, int taskIndex) {
         Preconditions.checkArgument(taskIndex < totalTasks, "task index must be less that total tasks");
-        List<Partition> partitions = partitionInformation.getOrderedPartitions();
+        List<Partition> taskPartitions = new ArrayList<Partition>();
+        List<Partition> partitions = new ArrayList<Partition>();
+        for(GlobalPartitionInformation partitionInformation : partitons) {
+            partitions.addAll(partitionInformation.getOrderedPartitions());
+        }
         int numPartitions = partitions.size();
         if (numPartitions < totalTasks) {
             LOG.warn("there are more tasks than partitions (tasks: " + totalTasks + "; partitions: " + numPartitions + "), some tasks will be idle");
         }
-        List<Partition> taskPartitions = new ArrayList<Partition>();
         for (int i = taskIndex; i < numPartitions; i += totalTasks) {
             Partition taskPartition = partitions.get(i);
             taskPartitions.add(taskPartition);
diff --git a/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java b/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java
index f42f7c8..7c0dc6c 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/KeyValueScheme.java
@@ -19,10 +19,9 @@
 
 import backtype.storm.spout.Scheme;
 
+import java.nio.ByteBuffer;
 import java.util.List;
 
 public interface KeyValueScheme extends Scheme {
-
-    public List<Object> deserializeKeyAndValue(byte[] key, byte[] value);
-
+    List<Object> deserializeKeyAndValue(ByteBuffer key, ByteBuffer value);
 }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/KeyValueSchemeAsMultiScheme.java b/external/storm-kafka/src/jvm/storm/kafka/KeyValueSchemeAsMultiScheme.java
index 7def6ac..d27ae7e 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/KeyValueSchemeAsMultiScheme.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/KeyValueSchemeAsMultiScheme.java
@@ -19,16 +19,17 @@
 
 import backtype.storm.spout.SchemeAsMultiScheme;
 
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 
-public class KeyValueSchemeAsMultiScheme extends SchemeAsMultiScheme{
+public class KeyValueSchemeAsMultiScheme extends SchemeAsMultiScheme {
 
     public KeyValueSchemeAsMultiScheme(KeyValueScheme scheme) {
         super(scheme);
     }
 
-    public Iterable<List<Object>> deserializeKeyAndValue(final byte[] key, final byte[] value) {
+    public Iterable<List<Object>> deserializeKeyAndValue(final ByteBuffer key, final ByteBuffer value) {
         List<Object> o = ((KeyValueScheme)scheme).deserializeKeyAndValue(key, value);
         if(o == null) return null;
         else return Arrays.asList(o);
diff --git a/external/storm-kafka/src/jvm/storm/kafka/MessageMetadataScheme.java b/external/storm-kafka/src/jvm/storm/kafka/MessageMetadataScheme.java
new file mode 100644
index 0000000..62f652f
--- /dev/null
+++ b/external/storm-kafka/src/jvm/storm/kafka/MessageMetadataScheme.java
@@ -0,0 +1,27 @@
+/**
+ * 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 storm.kafka;
+
+import backtype.storm.spout.Scheme;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public interface MessageMetadataScheme extends Scheme {
+    List<Object> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset);
+}
diff --git a/external/storm-kafka/src/jvm/storm/kafka/MessageMetadataSchemeAsMultiScheme.java b/external/storm-kafka/src/jvm/storm/kafka/MessageMetadataSchemeAsMultiScheme.java
new file mode 100644
index 0000000..f23a101
--- /dev/null
+++ b/external/storm-kafka/src/jvm/storm/kafka/MessageMetadataSchemeAsMultiScheme.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package storm.kafka;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import backtype.storm.spout.SchemeAsMultiScheme;
+
+public class MessageMetadataSchemeAsMultiScheme extends SchemeAsMultiScheme {
+    private static final long serialVersionUID = -7172403703813625116L;
+
+    public MessageMetadataSchemeAsMultiScheme(MessageMetadataScheme scheme) {
+        super(scheme);
+    }
+
+    public Iterable<List<Object>> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset) {
+        List<Object> o = ((MessageMetadataScheme) scheme).deserializeMessageWithMetadata(message, partition, offset);
+        if (o == null) {
+            return null;
+        } else {
+            return Arrays.asList(o);
+        }
+    }
+}
diff --git a/external/storm-kafka/src/jvm/storm/kafka/Partition.java b/external/storm-kafka/src/jvm/storm/kafka/Partition.java
index 24d6a41..5f683ef 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/Partition.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/Partition.java
@@ -25,20 +25,32 @@
 
     public Broker host;
     public int partition;
+    public String topic;
+
+    //Flag to keep the Partition Path Id backward compatible with Old implementation of Partition.getId() == "partition_" + partition
+    private Boolean bUseTopicNameForPartitionPathId;
 
     // for kryo compatibility
     private Partition() {
 	
     }
-    
-    public Partition(Broker host, int partition) {
+    public Partition(Broker host, String topic, int partition) {
+        this.topic = topic;
         this.host = host;
         this.partition = partition;
+        this.bUseTopicNameForPartitionPathId = false;
+    }
+    
+    public Partition(Broker host, String topic, int partition,Boolean bUseTopicNameForPartitionPathId) {
+        this.topic = topic;
+        this.host = host;
+        this.partition = partition;
+        this.bUseTopicNameForPartitionPathId = bUseTopicNameForPartitionPathId;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hashCode(host, partition);
+        return Objects.hashCode(host, topic, partition);
     }
 
     @Override
@@ -50,20 +62,26 @@
             return false;
         }
         final Partition other = (Partition) obj;
-        return Objects.equal(this.host, other.host) && Objects.equal(this.partition, other.partition);
+        return Objects.equal(this.host, other.host) && Objects.equal(this.topic, other.topic) && Objects.equal(this.partition, other.partition);
     }
 
     @Override
     public String toString() {
         return "Partition{" +
                 "host=" + host +
+                ", topic=" + topic +
                 ", partition=" + partition +
                 '}';
     }
 
     @Override
     public String getId() {
-        return "partition_" + partition;
+        if (bUseTopicNameForPartitionPathId) {
+            return  topic  + "/partition_" + partition;
+        } else {
+            //Keep the Partition Id backward compatible with Old implementation of Partition.getId() == "partition_" + partition
+            return "partition_" + partition;
+        }
     }
 
 }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java
index 052d525..d1b9f48 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java
@@ -23,12 +23,16 @@
 import backtype.storm.metric.api.MeanReducer;
 import backtype.storm.metric.api.ReducedMetric;
 import backtype.storm.spout.SpoutOutputCollector;
+
 import com.google.common.collect.ImmutableMap;
+
 import kafka.javaapi.consumer.SimpleConsumer;
 import kafka.javaapi.message.ByteBufferMessageSet;
 import kafka.message.MessageAndOffset;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import storm.kafka.KafkaSpout.EmitState;
 import storm.kafka.KafkaSpout.MessageAndRealOffset;
 import storm.kafka.trident.MaxMetric;
@@ -63,7 +67,7 @@
         _connections = connections;
         _spoutConfig = spoutConfig;
         _topologyInstanceId = topologyInstanceId;
-        _consumer = connections.register(id.host, id.partition);
+        _consumer = connections.register(id.host, id.topic, id.partition);
         _state = state;
         _stormConf = stormConf;
         numberAcked = numberFailed = 0;
@@ -86,13 +90,14 @@
             LOG.warn("Error reading and/or parsing at ZkNode: " + path, e);
         }
 
-        Long currentOffset = KafkaUtils.getOffset(_consumer, spoutConfig.topic, id.partition, spoutConfig);
+        String topic = _partition.topic;
+        Long currentOffset = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig);
 
         if (jsonTopologyId == null || jsonOffset == null) { // failed to parse JSON?
             _committedTo = currentOffset;
             LOG.info("No partition information found, using configuration to determine offset");
         } else if (!topologyInstanceId.equals(jsonTopologyId) && spoutConfig.ignoreZkOffsets) {
-            _committedTo = KafkaUtils.getOffset(_consumer, spoutConfig.topic, id.partition, spoutConfig.startOffsetTime);
+            _committedTo = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig.startOffsetTime);
             LOG.info("Topology change detected and ignore zookeeper offsets set to true, using configuration to determine offset");
         } else {
             _committedTo = jsonOffset;
@@ -135,16 +140,23 @@
             if (toEmit == null) {
                 return EmitState.NO_EMITTED;
             }
-            Iterable<List<Object>> tups = KafkaUtils.generateTuples(_spoutConfig, toEmit.msg);
-            if (tups != null) {
-		if(_spoutConfig.topicAsStreamId) {
-	            for (List<Object> tup : tups) {
-			collector.emit(_spoutConfig.topic, tup, new KafkaMessageId(_partition, toEmit.offset));
-		    }
-		} else {
-		    for (List<Object> tup : tups) {
-			collector.emit(tup, new KafkaMessageId(_partition, toEmit.offset));
-		    }
+
+            Iterable<List<Object>> tups;
+            if (_spoutConfig.scheme instanceof MessageMetadataSchemeAsMultiScheme) {
+                tups = KafkaUtils.generateTuples((MessageMetadataSchemeAsMultiScheme) _spoutConfig.scheme, toEmit.msg, _partition, toEmit.offset);
+            } else {
+                tups = KafkaUtils.generateTuples(_spoutConfig, toEmit.msg, _partition.topic);
+            }
+            
+            if ((tups != null) && tups.iterator().hasNext()) {
+                if(_spoutConfig.topicAsStreamId) {
+                    for (List<Object> tup : tups) {
+                        collector.emit(_spoutConfig.topic, tup, new KafkaMessageId(_partition, toEmit.offset));
+                    }
+                } else {
+                    for (List<Object> tup : tups) {
+                        collector.emit(tup, new KafkaMessageId(_partition, toEmit.offset));
+                    }
                 }
                 break;
             } else {
@@ -174,8 +186,8 @@
         try {
             msgs = KafkaUtils.fetchMessages(_spoutConfig, _consumer, _partition, offset);
         } catch (TopicOffsetOutOfRangeException e) {
-            _emittedToOffset = KafkaUtils.getOffset(_consumer, _spoutConfig.topic, _partition.partition, kafka.api.OffsetRequest.EarliestTime());
-            LOG.warn("Using new offset: {}", _emittedToOffset);
+            _emittedToOffset = KafkaUtils.getOffset(_consumer, _partition.topic, _partition.partition, kafka.api.OffsetRequest.EarliestTime());
+            LOG.warn("{} Using new offset: {}", _partition.partition, _emittedToOffset);
             // fetch failed, so don't update the metrics
             
             //fix bug [STORM-643] : remove outdated failed offsets
@@ -260,7 +272,7 @@
                     .put("partition", _partition.partition)
                     .put("broker", ImmutableMap.of("host", _partition.host.host,
                             "port", _partition.host.port))
-                    .put("topic", _spoutConfig.topic).build();
+                    .put("topic", _partition.topic).build();
             _state.writeJSON(committedPath(), data);
 
             _committedTo = lastCompletedOffset;
@@ -288,13 +300,14 @@
 
     public void close() {
         commit();
-        _connections.unregister(_partition.host, _partition.partition);
+        _connections.unregister(_partition.host, _partition.topic , _partition.partition);
     }
 
     static class KafkaMessageId {
         public Partition partition;
         public long offset;
 
+
         public KafkaMessageId(Partition partition, long offset) {
             this.partition = partition;
             this.offset = offset;
diff --git a/external/storm-kafka/src/jvm/storm/kafka/StaticCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/StaticCoordinator.java
index de4d0fd..4b20d84 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/StaticCoordinator.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/StaticCoordinator.java
@@ -17,10 +17,9 @@
  */
 package storm.kafka;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import storm.kafka.trident.GlobalPartitionInformation;
+
+import java.util.*;
 
 
 public class StaticCoordinator implements PartitionCoordinator {
@@ -29,7 +28,9 @@
 
     public StaticCoordinator(DynamicPartitionConnections connections, Map stormConf, SpoutConfig config, ZkState state, int taskIndex, int totalTasks, String topologyInstanceId) {
         StaticHosts hosts = (StaticHosts) config.hosts;
-        List<Partition> myPartitions = KafkaUtils.calculatePartitionsForTask(hosts.getPartitionInformation(), totalTasks, taskIndex);
+        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
+        partitions.add(hosts.getPartitionInformation());
+        List<Partition> myPartitions = KafkaUtils.calculatePartitionsForTask(partitions, totalTasks, taskIndex);
         for (Partition myPartition : myPartitions) {
             _managers.put(myPartition, new PartitionManager(connections, topologyInstanceId, state, stormConf, config, myPartition));
         }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/StringKeyValueScheme.java b/external/storm-kafka/src/jvm/storm/kafka/StringKeyValueScheme.java
index 41cacb6..6f6d339 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/StringKeyValueScheme.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/StringKeyValueScheme.java
@@ -20,12 +20,13 @@
 import backtype.storm.tuple.Values;
 import com.google.common.collect.ImmutableMap;
 
+import java.nio.ByteBuffer;
 import java.util.List;
 
 public class StringKeyValueScheme extends StringScheme implements KeyValueScheme {
 
     @Override
-    public List<Object> deserializeKeyAndValue(byte[] key, byte[] value) {
+    public List<Object> deserializeKeyAndValue(ByteBuffer key, ByteBuffer value) {
         if ( key == null ) {
             return deserialize(value);
         }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/StringMessageAndMetadataScheme.java b/external/storm-kafka/src/jvm/storm/kafka/StringMessageAndMetadataScheme.java
new file mode 100644
index 0000000..1708b97
--- /dev/null
+++ b/external/storm-kafka/src/jvm/storm/kafka/StringMessageAndMetadataScheme.java
@@ -0,0 +1,43 @@
+/**
+ * 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 storm.kafka;
+
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class StringMessageAndMetadataScheme extends StringScheme implements MessageMetadataScheme {
+    private static final long serialVersionUID = -5441841920447947374L;
+
+    public static final String STRING_SCHEME_PARTITION_KEY = "partition";
+    public static final String STRING_SCHEME_OFFSET = "offset";
+
+    @Override
+    public List<Object> deserializeMessageWithMetadata(ByteBuffer message, Partition partition, long offset) {
+        String stringMessage = StringScheme.deserializeString(message);
+        return new Values(stringMessage, partition.partition, offset);
+    }
+
+    @Override
+    public Fields getOutputFields() {
+        return new Fields(STRING_SCHEME_KEY, STRING_SCHEME_PARTITION_KEY, STRING_SCHEME_OFFSET);
+    }
+
+}
diff --git a/external/storm-kafka/src/jvm/storm/kafka/StringMultiSchemeWithTopic.java b/external/storm-kafka/src/jvm/storm/kafka/StringMultiSchemeWithTopic.java
new file mode 100644
index 0000000..1e7f216
--- /dev/null
+++ b/external/storm-kafka/src/jvm/storm/kafka/StringMultiSchemeWithTopic.java
@@ -0,0 +1,48 @@
+/**
+ * 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 storm.kafka;
+
+import backtype.storm.spout.MultiScheme;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+public class StringMultiSchemeWithTopic
+        implements MultiScheme {
+    public static final String STRING_SCHEME_KEY = "str";
+
+    public static final String TOPIC_KEY = "topic";
+
+    @Override
+    public Iterable<List<Object>> deserialize(ByteBuffer bytes) {
+        throw new NotImplementedException();
+    }
+
+    public Iterable<List<Object>> deserializeWithTopic(String topic, ByteBuffer bytes) {
+        List<Object> items = new Values(StringScheme.deserializeString(bytes), topic);
+        return Collections.singletonList(items);
+    }
+
+    public Fields getOutputFields() {
+        return new Fields(STRING_SCHEME_KEY, TOPIC_KEY);
+    }
+}
diff --git a/external/storm-kafka/src/jvm/storm/kafka/StringScheme.java b/external/storm-kafka/src/jvm/storm/kafka/StringScheme.java
index 286dc9b..1071e60 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/StringScheme.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/StringScheme.java
@@ -20,23 +20,27 @@
 import backtype.storm.spout.Scheme;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
 
-import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 
 public class StringScheme implements Scheme {
-
+    private static final Charset UTF8_CHARSET = StandardCharsets.UTF_8;
     public static final String STRING_SCHEME_KEY = "str";
 
-    public List<Object> deserialize(byte[] bytes) {
+    public List<Object> deserialize(ByteBuffer bytes) {
         return new Values(deserializeString(bytes));
     }
 
-    public static String deserializeString(byte[] string) {
-        try {
-            return new String(string, "UTF-8");
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException(e);
+    public static String deserializeString(ByteBuffer string) {
+        if (string.hasArray()) {
+            int base = string.arrayOffset();
+            return new String(string.array(), base + string.position(), string.remaining());
+        } else {
+            return new String(Utils.toByteArray(string), UTF8_CHARSET);
         }
     }
 
diff --git a/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java
index 3af648c..8650e6f 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java
@@ -76,7 +76,7 @@
     public void refresh() {
         try {
             LOG.info(taskId(_taskIndex, _totalTasks) + "Refreshing partition manager connections");
-            GlobalPartitionInformation brokerInfo = _reader.getBrokerInfo();
+            List<GlobalPartitionInformation> brokerInfo = _reader.getBrokerInfo();
             List<Partition> mine = KafkaUtils.calculatePartitionsForTask(brokerInfo, _totalTasks, _taskIndex);
 
             Set<Partition> curr = _managers.keySet();
diff --git a/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java
index 2cca826..1ebe142 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java
@@ -57,7 +57,6 @@
     private static final Logger LOG = LoggerFactory.getLogger(KafkaBolt.class);
 
     public static final String TOPIC = "topic";
-    public static final String KAFKA_BROKER_PROPERTIES = "kafka.broker.properties";
 
     private KafkaProducer<K, V> producer;
     private OutputCollector collector;
@@ -73,8 +72,7 @@
     private boolean fireAndForget = false;
     private boolean async = true;
 
-    public KafkaBolt() {
-    }
+    public KafkaBolt() {}
 
     public KafkaBolt<K,V> withTupleToKafkaMapper(TupleToKafkaMapper<K,V> mapper) {
         this.mapper = mapper;
@@ -103,14 +101,7 @@
             this.topicSelector = new DefaultTopicSelector((String) stormConf.get(TOPIC));
         }
 
-        Map configMap = (Map) stormConf.get(KAFKA_BROKER_PROPERTIES);
-        Properties properties = new Properties();
-        if(configMap!= null)
-            properties.putAll(configMap);
-        if(boltSpecfiedProperties != null)
-            properties.putAll(boltSpecfiedProperties);
-
-        producer = new KafkaProducer<K, V>(properties);
+        producer = new KafkaProducer<>(boltSpecfiedProperties);
         this.collector = collector;
     }
 
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/Coordinator.java b/external/storm-kafka/src/jvm/storm/kafka/trident/Coordinator.java
index c395f8c..bd786b3 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/Coordinator.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/Coordinator.java
@@ -21,9 +21,10 @@
 import storm.trident.spout.IOpaquePartitionedTridentSpout;
 import storm.trident.spout.IPartitionedTridentSpout;
 
+import java.util.List;
 import java.util.Map;
 
-class Coordinator implements IPartitionedTridentSpout.Coordinator<GlobalPartitionInformation>, IOpaquePartitionedTridentSpout.Coordinator<GlobalPartitionInformation> {
+class Coordinator implements IPartitionedTridentSpout.Coordinator<List<GlobalPartitionInformation>>, IOpaquePartitionedTridentSpout.Coordinator<List<GlobalPartitionInformation>> {
 
     private IBrokerReader reader;
     private TridentKafkaConfig config;
@@ -44,7 +45,7 @@
     }
 
     @Override
-    public GlobalPartitionInformation getPartitionsForBatch() {
-        return reader.getCurrentBrokers();
+    public List<GlobalPartitionInformation> getPartitionsForBatch() {
+        return reader.getAllBrokers();
     }
 }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/GlobalPartitionInformation.java b/external/storm-kafka/src/jvm/storm/kafka/trident/GlobalPartitionInformation.java
index 76bec9e..b0d97fc 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/GlobalPartitionInformation.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/GlobalPartitionInformation.java
@@ -28,9 +28,21 @@
 public class GlobalPartitionInformation implements Iterable<Partition>, Serializable {
 
     private Map<Integer, Broker> partitionMap;
+    public String topic;
 
-    public GlobalPartitionInformation() {
-        partitionMap = new TreeMap<Integer, Broker>();
+    //Flag to keep the Partition Path Id backward compatible with Old implementation of Partition.getId() == "partition_" + partition
+    private Boolean bUseTopicNameForPartitionPathId;
+
+    public GlobalPartitionInformation(String topic, Boolean bUseTopicNameForPartitionPathId) {
+        this.topic = topic;
+        this.partitionMap = new TreeMap<Integer, Broker>();
+        this.bUseTopicNameForPartitionPathId = bUseTopicNameForPartitionPathId;
+    }
+
+    public GlobalPartitionInformation(String topic) {
+        this.topic = topic;
+        this.partitionMap = new TreeMap<Integer, Broker>();
+        this.bUseTopicNameForPartitionPathId = false;
     }
 
     public void addPartition(int partitionId, Broker broker) {
@@ -40,7 +52,8 @@
     @Override
     public String toString() {
         return "GlobalPartitionInformation{" +
-                "partitionMap=" + partitionMap +
+                "topic=" + topic +
+                ", partitionMap=" + partitionMap +
                 '}';
     }
 
@@ -51,7 +64,7 @@
     public List<Partition> getOrderedPartitions() {
         List<Partition> partitions = new LinkedList<Partition>();
         for (Map.Entry<Integer, Broker> partition : partitionMap.entrySet()) {
-            partitions.add(new Partition(partition.getValue(), partition.getKey()));
+            partitions.add(new Partition(partition.getValue(), this.topic, partition.getKey(), this.bUseTopicNameForPartitionPathId));
         }
         return partitions;
     }
@@ -59,7 +72,8 @@
     @Override
     public Iterator<Partition> iterator() {
         final Iterator<Map.Entry<Integer, Broker>> iterator = partitionMap.entrySet().iterator();
-
+        final String topic = this.topic;
+        final Boolean bUseTopicNameForPartitionPathId = this.bUseTopicNameForPartitionPathId;
         return new Iterator<Partition>() {
             @Override
             public boolean hasNext() {
@@ -69,7 +83,7 @@
             @Override
             public Partition next() {
                 Map.Entry<Integer, Broker> next = iterator.next();
-                return new Partition(next.getValue(), next.getKey());
+                return new Partition(next.getValue(), topic , next.getKey(), bUseTopicNameForPartitionPathId);
             }
 
             @Override
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java b/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java
index 3e018d9..afba659 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/IBrokerReader.java
@@ -17,9 +17,14 @@
  */
 package storm.kafka.trident;
 
+import java.util.List;
+import java.util.Map;
+
 public interface IBrokerReader {
 
-    GlobalPartitionInformation getCurrentBrokers();
+    GlobalPartitionInformation getBrokerForTopic(String topic);
+
+    List<GlobalPartitionInformation> getAllBrokers();
 
     void close();
 }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java b/external/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java
index 136e7d2..fbd1d7a 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/OpaqueTridentKafkaSpout.java
@@ -22,23 +22,24 @@
 import storm.kafka.Partition;
 import storm.trident.spout.IOpaquePartitionedTridentSpout;
 
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 
 
-public class OpaqueTridentKafkaSpout implements IOpaquePartitionedTridentSpout<GlobalPartitionInformation, Partition, Map> {
+public class OpaqueTridentKafkaSpout implements IOpaquePartitionedTridentSpout<List<GlobalPartitionInformation>, Partition, Map> {
 
 
     TridentKafkaConfig _config;
-    String _topologyInstanceId = UUID.randomUUID().toString();
 
     public OpaqueTridentKafkaSpout(TridentKafkaConfig config) {
         _config = config;
     }
 
     @Override
-    public IOpaquePartitionedTridentSpout.Emitter<GlobalPartitionInformation, Partition, Map> getEmitter(Map conf, TopologyContext context) {
-        return new TridentKafkaEmitter(conf, context, _config, _topologyInstanceId).asOpaqueEmitter();
+    public IOpaquePartitionedTridentSpout.Emitter<List<GlobalPartitionInformation>, Partition, Map> getEmitter(Map conf, TopologyContext context) {
+        return new TridentKafkaEmitter(conf, context, _config, context
+                .getStormId()).asOpaqueEmitter();
     }
 
     @Override
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/StaticBrokerReader.java b/external/storm-kafka/src/jvm/storm/kafka/trident/StaticBrokerReader.java
index d1673f1..ca83c06 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/StaticBrokerReader.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/StaticBrokerReader.java
@@ -17,17 +17,30 @@
  */
 package storm.kafka.trident;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
 public class StaticBrokerReader implements IBrokerReader {
 
-    private GlobalPartitionInformation brokers = new GlobalPartitionInformation();
+    private Map<String,GlobalPartitionInformation> brokers = new TreeMap<String,GlobalPartitionInformation>();
 
-    public StaticBrokerReader(GlobalPartitionInformation partitionInformation) {
-        this.brokers = partitionInformation;
+    public StaticBrokerReader(String topic, GlobalPartitionInformation partitionInformation) {
+        this.brokers.put(topic, partitionInformation);
     }
 
     @Override
-    public GlobalPartitionInformation getCurrentBrokers() {
-        return brokers;
+    public GlobalPartitionInformation getBrokerForTopic(String topic) {
+        if (brokers.containsKey(topic)) return brokers.get(topic);
+        return null;
+    }
+
+    @Override
+    public List<GlobalPartitionInformation> getAllBrokers () {
+        List<GlobalPartitionInformation> list = new ArrayList<GlobalPartitionInformation>();
+        list.addAll(brokers.values());
+        return list;
     }
 
     @Override
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java b/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java
index 8c10bed..9feffc8 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/TransactionalTridentKafkaSpout.java
@@ -29,7 +29,6 @@
 public class TransactionalTridentKafkaSpout implements IPartitionedTridentSpout<GlobalPartitionInformation, Partition, Map> {
 
     TridentKafkaConfig _config;
-    String _topologyInstanceId = UUID.randomUUID().toString();
 
     public TransactionalTridentKafkaSpout(TridentKafkaConfig config) {
         _config = config;
@@ -43,7 +42,8 @@
 
     @Override
     public IPartitionedTridentSpout.Emitter getEmitter(Map conf, TopologyContext context) {
-        return new TridentKafkaEmitter(conf, context, _config, _topologyInstanceId).asTransactionalEmitter();
+        return new TridentKafkaEmitter(conf, context, _config, context
+                .getStormId()).asTransactionalEmitter();
     }
 
     @Override
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaEmitter.java b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaEmitter.java
index 61c79a5..a97d2cb 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaEmitter.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaEmitter.java
@@ -36,10 +36,7 @@
 import storm.trident.spout.IPartitionedTridentSpout;
 import storm.trident.topology.TransactionAttempt;
 
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 public class TridentKafkaEmitter {
 
@@ -58,7 +55,7 @@
         _topologyInstanceId = topologyInstanceId;
         _connections = new DynamicPartitionConnections(_config, KafkaUtils.makeBrokerReader(conf, _config));
         _topologyName = (String) conf.get(Config.TOPOLOGY_NAME);
-        _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_config.topic, _connections);
+        _kafkaOffsetMetric = new KafkaUtils.KafkaOffsetMetric(_connections);
         context.registerMetric("kafkaOffset", _kafkaOffsetMetric, _config.metricsTimeBucketSizeInSecs);
         _kafkaMeanFetchLatencyMetric = context.registerMetric("kafkaFetchAvg", new MeanReducer(), _config.metricsTimeBucketSizeInSecs);
         _kafkaMaxFetchLatencyMetric = context.registerMetric("kafkaFetchMax", new MaxMetric(), _config.metricsTimeBucketSizeInSecs);
@@ -85,7 +82,7 @@
                 ret.put("nextOffset", lastMeta.get("nextOffset"));
                 ret.put("partition", partition.partition);
                 ret.put("broker", ImmutableMap.of("host", partition.host.host, "port", partition.host.port));
-                ret.put("topic", _config.topic);
+                ret.put("topic", partition.topic);
                 ret.put("topology", ImmutableMap.of("name", _topologyName, "id", _topologyInstanceId));
                 return ret;
             }
@@ -101,19 +98,19 @@
                 lastInstanceId = (String) lastTopoMeta.get("id");
             }
             if (_config.ignoreZkOffsets && !_topologyInstanceId.equals(lastInstanceId)) {
-                offset = KafkaUtils.getOffset(consumer, _config.topic, partition.partition, _config.startOffsetTime);
+                offset = KafkaUtils.getOffset(consumer, partition.topic, partition.partition, _config.startOffsetTime);
             } else {
                 offset = (Long) lastMeta.get("nextOffset");
             }
         } else {
-            offset = KafkaUtils.getOffset(consumer, _config.topic, partition.partition, _config);
+            offset = KafkaUtils.getOffset(consumer, partition.topic, partition.partition, _config);
         }
 
         ByteBufferMessageSet msgs = null;
         try {
             msgs = fetchMessages(consumer, partition, offset);
         } catch (TopicOffsetOutOfRangeException e) {
-            long newOffset = KafkaUtils.getOffset(consumer, _config.topic, partition.partition, kafka.api.OffsetRequest.EarliestTime());
+            long newOffset = KafkaUtils.getOffset(consumer, partition.topic, partition.partition, kafka.api.OffsetRequest.EarliestTime());
             LOG.warn("OffsetOutOfRange: Updating offset from offset = " + offset + " to offset = " + newOffset);
             offset = newOffset;
             msgs = KafkaUtils.fetchMessages(_config, consumer, partition, offset);
@@ -121,7 +118,7 @@
 
         long endoffset = offset;
         for (MessageAndOffset msg : msgs) {
-            emit(collector, msg.message());
+            emit(collector, msg.message(), partition, msg.offset());
             endoffset = msg.nextOffset();
         }
         Map newMeta = new HashMap();
@@ -130,7 +127,7 @@
         newMeta.put("instanceId", _topologyInstanceId);
         newMeta.put("partition", partition.partition);
         newMeta.put("broker", ImmutableMap.of("host", partition.host.host, "port", partition.host.port));
-        newMeta.put("topic", _config.topic);
+        newMeta.put("topic", partition.topic);
         newMeta.put("topology", ImmutableMap.of("name", _topologyName, "id", _topologyInstanceId));
         return newMeta;
     }
@@ -172,15 +169,21 @@
                     if (offset > nextOffset) {
                         throw new RuntimeException("Error when re-emitting batch. overshot the end offset");
                     }
-                    emit(collector, msg.message());
+                    emit(collector, msg.message(), partition, msg.offset());
                     offset = msg.nextOffset();
                 }
             }
         }
     }
 
-    private void emit(TridentCollector collector, Message msg) {
-        Iterable<List<Object>> values = KafkaUtils.generateTuples(_config, msg);
+    private void emit(TridentCollector collector, Message msg, Partition partition, long offset) {
+        Iterable<List<Object>> values;
+        if (_config.scheme instanceof MessageMetadataSchemeAsMultiScheme) {
+            values = KafkaUtils.generateTuples((MessageMetadataSchemeAsMultiScheme) _config.scheme, msg, partition, offset);
+        } else {
+            values = KafkaUtils.generateTuples(_config, msg, partition.topic);
+        }
+
         if (values != null) {
             for (List<Object> value : values) {
                 collector.emit(value);
@@ -192,8 +195,11 @@
         _connections.clear();
     }
 
-    private List<Partition> orderPartitions(GlobalPartitionInformation partitions) {
-        return partitions.getOrderedPartitions();
+    private List<Partition> orderPartitions(List<GlobalPartitionInformation> partitions) {
+        List<Partition> part = new ArrayList<Partition>();
+        for (GlobalPartitionInformation globalPartitionInformation : partitions)
+            part.addAll(globalPartitionInformation.getOrderedPartitions());
+        return part;
     }
 
     private void refresh(List<Partition> list) {
@@ -202,9 +208,9 @@
     }
 
 
-    public IOpaquePartitionedTridentSpout.Emitter<GlobalPartitionInformation, Partition, Map> asOpaqueEmitter() {
+    public IOpaquePartitionedTridentSpout.Emitter<List<GlobalPartitionInformation>, Partition, Map> asOpaqueEmitter() {
 
-        return new IOpaquePartitionedTridentSpout.Emitter<GlobalPartitionInformation, Partition, Map>() {
+        return new IOpaquePartitionedTridentSpout.Emitter<List<GlobalPartitionInformation>, Partition, Map>() {
 
             /**
              * Emit a batch of tuples for a partition/transaction.
@@ -223,7 +229,7 @@
             }
 
             @Override
-            public List<Partition> getOrderedPartitions(GlobalPartitionInformation partitionInformation) {
+            public List<Partition> getOrderedPartitions(List<GlobalPartitionInformation> partitionInformation) {
                 return orderPartitions(partitionInformation);
             }
 
@@ -235,7 +241,7 @@
     }
 
     public IPartitionedTridentSpout.Emitter asTransactionalEmitter() {
-        return new IPartitionedTridentSpout.Emitter<GlobalPartitionInformation, Partition, Map>() {
+        return new IPartitionedTridentSpout.Emitter<List<GlobalPartitionInformation>, Partition, Map>() {
 
             /**
              * Emit a batch of tuples for a partition/transaction that's never been emitted before.
@@ -265,7 +271,7 @@
             }
 
             @Override
-            public List<Partition> getOrderedPartitions(GlobalPartitionInformation partitionInformation) {
+            public List<Partition> getOrderedPartitions(List<GlobalPartitionInformation> partitionInformation) {
                 return orderPartitions(partitionInformation);
             }
 
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java
index 402ffb1..84b6a6a 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java
@@ -19,10 +19,10 @@
 
 import backtype.storm.task.OutputCollector;
 import backtype.storm.topology.FailedException;
-import kafka.javaapi.producer.Producer;
-import kafka.producer.KeyedMessage;
-import kafka.producer.ProducerConfig;
 import org.apache.commons.lang.Validate;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
@@ -32,15 +32,14 @@
 import storm.trident.tuple.TridentTuple;
 
 import java.util.List;
-import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 
 public class TridentKafkaState implements State {
     private static final Logger LOG = LoggerFactory.getLogger(TridentKafkaState.class);
 
-    public static final String KAFKA_BROKER_PROPERTIES = "kafka.broker.properties";
-
-    private Producer producer;
+    private KafkaProducer producer;
     private OutputCollector collector;
 
     private TridentTupleToKafkaMapper mapper;
@@ -66,14 +65,10 @@
         LOG.debug("commit is Noop.");
     }
 
-    public void prepare(Map stormConf) {
+    public void prepare(Properties options) {
         Validate.notNull(mapper, "mapper can not be null");
         Validate.notNull(topicSelector, "topicSelector can not be null");
-        Map configMap = (Map) stormConf.get(KAFKA_BROKER_PROPERTIES);
-        Properties properties = new Properties();
-        properties.putAll(configMap);
-        ProducerConfig config = new ProducerConfig(properties);
-        producer = new Producer(config);
+        producer = new KafkaProducer(options);
     }
 
     public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
@@ -83,8 +78,16 @@
                 topic = topicSelector.getTopic(tuple);
 
                 if(topic != null) {
-                    producer.send(new KeyedMessage(topic, mapper.getKeyFromTuple(tuple),
-                            mapper.getMessageFromTuple(tuple)));
+                    Future<RecordMetadata> result = producer.send(new ProducerRecord(topic,
+                            mapper.getKeyFromTuple(tuple), mapper.getMessageFromTuple(tuple)));
+                    try {
+                        result.get();
+                    } catch (ExecutionException e) {
+                        String errorMsg = "Could not retrieve result for message with key = "
+                                + mapper.getKeyFromTuple(tuple) + " from topic = " + topic;
+                        LOG.error(errorMsg, e);
+                        throw new FailedException(errorMsg, e);
+                    }
                 } else {
                     LOG.warn("skipping key = " + mapper.getKeyFromTuple(tuple) + ", topic selector returned null.");
                 }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaStateFactory.java b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaStateFactory.java
index adca69e..a5d9d42 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaStateFactory.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaStateFactory.java
@@ -26,6 +26,7 @@
 import storm.trident.state.StateFactory;
 
 import java.util.Map;
+import java.util.Properties;
 
 public class TridentKafkaStateFactory implements StateFactory {
 
@@ -33,7 +34,7 @@
 
     private TridentTupleToKafkaMapper mapper;
     private KafkaTopicSelector topicSelector;
-
+    private Properties producerProperties = new Properties();
 
     public TridentKafkaStateFactory withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) {
         this.mapper = mapper;
@@ -45,13 +46,18 @@
         return this;
     }
 
+    public TridentKafkaStateFactory withProducerProperties(Properties props) {
+        this.producerProperties = props;
+        return this;
+    }
+
     @Override
     public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
         LOG.info("makeState(partitonIndex={}, numpartitions={}", partitionIndex, numPartitions);
         TridentKafkaState state = new TridentKafkaState()
                 .withKafkaTopicSelector(this.topicSelector)
                 .withTridentTupleToKafkaMapper(this.mapper);
-        state.prepare(conf);
+        state.prepare(producerProperties);
         return state;
     }
 }
diff --git a/external/storm-kafka/src/jvm/storm/kafka/trident/ZkBrokerReader.java b/external/storm-kafka/src/jvm/storm/kafka/trident/ZkBrokerReader.java
index 1a26705..b480bdd 100644
--- a/external/storm-kafka/src/jvm/storm/kafka/trident/ZkBrokerReader.java
+++ b/external/storm-kafka/src/jvm/storm/kafka/trident/ZkBrokerReader.java
@@ -22,6 +22,8 @@
 import storm.kafka.DynamicBrokersReader;
 import storm.kafka.ZkHosts;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
 
@@ -29,7 +31,7 @@
 
 	public static final Logger LOG = LoggerFactory.getLogger(ZkBrokerReader.class);
 
-	GlobalPartitionInformation cachedBrokers;
+	List<GlobalPartitionInformation> cachedBrokers = new ArrayList<GlobalPartitionInformation>();
 	DynamicBrokersReader reader;
 	long lastRefreshTimeMs;
 
@@ -48,8 +50,7 @@
 
 	}
 
-	@Override
-	public GlobalPartitionInformation getCurrentBrokers() {
+	private void refresh() {
 		long currTime = System.currentTimeMillis();
 		if (currTime > lastRefreshTimeMs + refreshMillis) {
 			try {
@@ -60,6 +61,19 @@
 				LOG.warn("Failed to update brokers", e);
 			}
 		}
+	}
+	@Override
+	public GlobalPartitionInformation getBrokerForTopic(String topic) {
+		refresh();
+        for(GlobalPartitionInformation partitionInformation : cachedBrokers) {
+            if (partitionInformation.topic.equals(topic)) return partitionInformation;
+        }
+		return null;
+	}
+
+	@Override
+	public List<GlobalPartitionInformation> getAllBrokers() {
+		refresh();
 		return cachedBrokers;
 	}
 
diff --git a/external/storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java b/external/storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java
index 941ac9e..d871924 100644
--- a/external/storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java
+++ b/external/storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java
@@ -29,18 +29,24 @@
 import storm.kafka.trident.GlobalPartitionInformation;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Date: 16/05/2013
  * Time: 20:35
  */
 public class DynamicBrokersReaderTest {
-    private DynamicBrokersReader dynamicBrokersReader;
+    private DynamicBrokersReader dynamicBrokersReader, wildCardBrokerReader;
     private String masterPath = "/brokers";
-    private String topic = "testing";
+    private String topic = "testing1";
+    private String secondTopic = "testing2";
+    private String thirdTopic = "testing3";
+
     private CuratorFramework zookeeper;
     private TestingServer server;
 
@@ -53,9 +59,16 @@
         conf.put(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT, 1000);
         conf.put(Config.STORM_ZOOKEEPER_RETRY_TIMES, 4);
         conf.put(Config.STORM_ZOOKEEPER_RETRY_INTERVAL, 5);
+
         ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(1000, 3);
         zookeeper = CuratorFrameworkFactory.newClient(connectionString, retryPolicy);
         dynamicBrokersReader = new DynamicBrokersReader(conf, connectionString, masterPath, topic);
+
+        Map conf2 = new HashMap();
+        conf2.putAll(conf);
+        conf2.put("kafka.topic.wildcard.match",true);
+
+        wildCardBrokerReader = new DynamicBrokersReader(conf2, connectionString, masterPath, "^test.*$");
         zookeeper.start();
     }
 
@@ -66,20 +79,20 @@
         server.close();
     }
 
-    private void addPartition(int id, String host, int port) throws Exception {
-        writePartitionId(id);
-        writeLeader(id, 0);
+    private void addPartition(int id, String host, int port, String topic) throws Exception {
+        writePartitionId(id, topic);
+        writeLeader(id, 0, topic);
         writeLeaderDetails(0, host, port);
     }
 
-    private void addPartition(int id, int leader, String host, int port) throws Exception {
-        writePartitionId(id);
-        writeLeader(id, leader);
+    private void addPartition(int id, int leader, String host, int port, String topic) throws Exception {
+        writePartitionId(id, topic);
+        writeLeader(id, leader, topic);
         writeLeaderDetails(leader, host, port);
     }
 
-    private void writePartitionId(int id) throws Exception {
-        String path = dynamicBrokersReader.partitionPath();
+    private void writePartitionId(int id, String topic) throws Exception {
+        String path = dynamicBrokersReader.partitionPath(topic);
         writeDataToPath(path, ("" + id));
     }
 
@@ -88,8 +101,8 @@
         zookeeper.setData().forPath(path, data.getBytes());
     }
 
-    private void writeLeader(int id, int leaderId) throws Exception {
-        String path = dynamicBrokersReader.partitionPath() + "/" + id + "/state";
+    private void writeLeader(int id, int leaderId, String topic) throws Exception {
+        String path = dynamicBrokersReader.partitionPath(topic) + "/" + id + "/state";
         String value = " { \"controller_epoch\":4, \"isr\":[ 1, 0 ], \"leader\":" + leaderId + ", \"leader_epoch\":1, \"version\":1 }";
         writeDataToPath(path, value);
     }
@@ -100,18 +113,59 @@
         writeDataToPath(path, value);
     }
 
+
+    private GlobalPartitionInformation getByTopic(List<GlobalPartitionInformation> partitions, String topic){
+        for(GlobalPartitionInformation partitionInformation : partitions) {
+            if (partitionInformation.topic.equals(topic)) return partitionInformation;
+        }
+        return null;
+    }
+
     @Test
     public void testGetBrokerInfo() throws Exception {
         String host = "localhost";
         int port = 9092;
         int partition = 0;
-        addPartition(partition, host, port);
-        GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo();
+        addPartition(partition, host, port, topic);
+        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
+
+        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
+        assertNotNull(brokerInfo);
         assertEquals(1, brokerInfo.getOrderedPartitions().size());
         assertEquals(port, brokerInfo.getBrokerFor(partition).port);
         assertEquals(host, brokerInfo.getBrokerFor(partition).host);
     }
 
+    @Test
+    public void testGetBrokerInfoWildcardMatch() throws Exception {
+        String host = "localhost";
+        int port = 9092;
+        int partition = 0;
+        addPartition(partition, host, port, topic);
+        addPartition(partition, host, port, secondTopic);
+
+        List<GlobalPartitionInformation> partitions = wildCardBrokerReader.getBrokerInfo();
+
+        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
+        assertNotNull(brokerInfo);
+        assertEquals(1, brokerInfo.getOrderedPartitions().size());
+        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
+        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
+
+        brokerInfo = getByTopic(partitions, secondTopic);
+        assertNotNull(brokerInfo);
+        assertEquals(1, brokerInfo.getOrderedPartitions().size());
+        assertEquals(port, brokerInfo.getBrokerFor(partition).port);
+        assertEquals(host, brokerInfo.getBrokerFor(partition).host);
+
+        addPartition(partition, host, port, thirdTopic);
+        //Discover newly added topic
+        partitions = wildCardBrokerReader.getBrokerInfo();
+        assertNotNull(getByTopic(partitions, topic));
+        assertNotNull(getByTopic(partitions, secondTopic));
+        assertNotNull(getByTopic(partitions, secondTopic));
+    }
+
 
     @Test
     public void testMultiplePartitionsOnDifferentHosts() throws Exception {
@@ -120,10 +174,13 @@
         int secondPort = 9093;
         int partition = 0;
         int secondPartition = partition + 1;
-        addPartition(partition, 0, host, port);
-        addPartition(secondPartition, 1, host, secondPort);
+        addPartition(partition, 0, host, port, topic);
+        addPartition(secondPartition, 1, host, secondPort, topic);
 
-        GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo();
+        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
+
+        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
+        assertNotNull(brokerInfo);
         assertEquals(2, brokerInfo.getOrderedPartitions().size());
 
         assertEquals(port, brokerInfo.getBrokerFor(partition).port);
@@ -140,10 +197,13 @@
         int port = 9092;
         int partition = 0;
         int secondPartition = partition + 1;
-        addPartition(partition, 0, host, port);
-        addPartition(secondPartition, 0, host, port);
+        addPartition(partition, 0, host, port, topic);
+        addPartition(secondPartition, 0, host, port, topic);
 
-        GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo();
+        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
+
+        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
+        assertNotNull(brokerInfo);
         assertEquals(2, brokerInfo.getOrderedPartitions().size());
 
         assertEquals(port, brokerInfo.getBrokerFor(partition).port);
@@ -158,15 +218,21 @@
         String host = "localhost";
         int port = 9092;
         int partition = 0;
-        addPartition(partition, host, port);
-        GlobalPartitionInformation brokerInfo = dynamicBrokersReader.getBrokerInfo();
+        addPartition(partition, host, port, topic);
+        List<GlobalPartitionInformation> partitions = dynamicBrokersReader.getBrokerInfo();
+
+        GlobalPartitionInformation brokerInfo = getByTopic(partitions, topic);
+        assertNotNull(brokerInfo);
         assertEquals(port, brokerInfo.getBrokerFor(partition).port);
         assertEquals(host, brokerInfo.getBrokerFor(partition).host);
 
         String newHost = host + "switch";
         int newPort = port + 1;
-        addPartition(partition, newHost, newPort);
-        brokerInfo = dynamicBrokersReader.getBrokerInfo();
+        addPartition(partition, newHost, newPort, topic);
+        partitions = dynamicBrokersReader.getBrokerInfo();
+
+        brokerInfo = getByTopic(partitions, topic);
+        assertNotNull(brokerInfo);
         assertEquals(newPort, brokerInfo.getBrokerFor(partition).port);
         assertEquals(newHost, brokerInfo.getBrokerFor(partition).host);
     }
diff --git a/external/storm-kafka/src/test/storm/kafka/KafkaUtilsTest.java b/external/storm-kafka/src/test/storm/kafka/KafkaUtilsTest.java
index a2d2af8..eb694bb 100644
--- a/external/storm-kafka/src/test/storm/kafka/KafkaUtilsTest.java
+++ b/external/storm-kafka/src/test/storm/kafka/KafkaUtilsTest.java
@@ -17,33 +17,37 @@
  */
 package storm.kafka;
 
-import backtype.storm.spout.SchemeAsMultiScheme;
-import backtype.storm.utils.Utils;
-import com.google.common.collect.ImmutableMap;
-import kafka.api.OffsetRequest;
-import kafka.javaapi.consumer.SimpleConsumer;
-import kafka.javaapi.message.ByteBufferMessageSet;
-import kafka.message.MessageAndOffset;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.Assert;
-import storm.kafka.trident.GlobalPartitionInformation;
-
-import java.util.List;
-import java.util.Properties;
-
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import kafka.api.OffsetRequest;
+import kafka.javaapi.consumer.SimpleConsumer;
+import kafka.javaapi.message.ByteBufferMessageSet;
+import kafka.message.MessageAndOffset;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-public class KafkaUtilsTest {
 
+import storm.kafka.trident.GlobalPartitionInformation;
+import backtype.storm.spout.SchemeAsMultiScheme;
+import backtype.storm.utils.Utils;
+
+import com.google.common.collect.ImmutableMap;
+public class KafkaUtilsTest {
+    private String TEST_TOPIC = "testTopic";
     private static final Logger LOG = LoggerFactory.getLogger(KafkaUtilsTest.class);
     private KafkaTestBroker broker;
     private SimpleConsumer simpleConsumer;
@@ -53,10 +57,10 @@
     @Before
     public void setup() {
         broker = new KafkaTestBroker();
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation();
+        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TEST_TOPIC);
         globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString()));
         brokerHosts = new StaticHosts(globalPartitionInformation);
-        config = new KafkaConfig(brokerHosts, "testTopic");
+        config = new KafkaConfig(brokerHosts, TEST_TOPIC);
         simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient");
     }
 
@@ -69,7 +73,7 @@
 
     @Test(expected = FailedFetchException.class)
     public void topicDoesNotExist() throws Exception {
-        KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), 0);
+        KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0), 0);
     }
 
     @Test(expected = FailedFetchException.class)
@@ -78,7 +82,7 @@
         broker.shutdown();
         SimpleConsumer simpleConsumer = new SimpleConsumer("localhost", port, 100, 1024, "testClient");
         try {
-            KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), OffsetRequest.LatestTime());
+            KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0), OffsetRequest.LatestTime());
         } finally {
             simpleConsumer.close();
         }
@@ -90,7 +94,7 @@
         createTopicAndSendMessage(value);
         long offset = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.LatestTime()) - 1;
         ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(config, simpleConsumer,
-                new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), offset);
+                new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0), offset);
         String message = new String(Utils.toByteArray(messageAndOffsets.iterator().next().message().payload()));
         assertThat(message, is(equalTo(value)));
     }
@@ -99,7 +103,7 @@
     public void fetchMessagesWithInvalidOffsetAndDefaultHandlingDisabled() throws Exception {
         config.useStartOffsetTimeIfOffsetOutOfRange = false;
         KafkaUtils.fetchMessages(config, simpleConsumer,
-                new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), -99);
+                new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0), -99);
     }
 
     @Test(expected = TopicOffsetOutOfRangeException.class)
@@ -108,7 +112,7 @@
         String value = "test";
         createTopicAndSendMessage(value);
         KafkaUtils.fetchMessages(config, simpleConsumer,
-                new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), -99);
+                new Partition(Broker.fromString(broker.getBrokerConnectionString()), "newTopic", 0), -99);
     }
 
     @Test
@@ -146,7 +150,7 @@
         createTopicAndSendMessage(key, value);
         ByteBufferMessageSet messageAndOffsets = getLastMessage();
         for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message());
+            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
             assertEquals(ImmutableMap.of(key, value), lists.iterator().next().get(0));
         }
     }
@@ -158,6 +162,20 @@
     }
 
     @Test
+    public void generateTuplesWithValueAndStringMultiSchemeWithTopic() {
+        config.scheme = new StringMultiSchemeWithTopic();
+        String value = "value";
+        createTopicAndSendMessage(value);
+        ByteBufferMessageSet messageAndOffsets = getLastMessage();
+        for (MessageAndOffset msg : messageAndOffsets) {
+            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
+            List<Object> list = lists.iterator().next();
+            assertEquals(value, list.get(0));
+            assertEquals(config.topic, list.get(1));
+        }
+    }
+
+    @Test
     public void generateTuplesWithValueSchemeAndKeyValueMessage() {
         config.scheme = new SchemeAsMultiScheme(new StringScheme());
         String value = "value";
@@ -165,22 +183,43 @@
         createTopicAndSendMessage(key, value);
         ByteBufferMessageSet messageAndOffsets = getLastMessage();
         for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message());
+            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
             assertEquals(value, lists.iterator().next().get(0));
         }
     }
+    
+    @Test
+    public void generateTuplesWithMessageAndMetadataScheme() {
+        String value = "value";
+        Partition mockPartition = Mockito.mock(Partition.class);
+        mockPartition.partition = 0;
+        long offset = 0L;
+        
+        MessageMetadataSchemeAsMultiScheme scheme = new MessageMetadataSchemeAsMultiScheme(new StringMessageAndMetadataScheme());
+        
+        createTopicAndSendMessage(null, value);
+        ByteBufferMessageSet messageAndOffsets = getLastMessage();
+        for (MessageAndOffset msg : messageAndOffsets) {
+            Iterable<List<Object>> lists = KafkaUtils.generateTuples(scheme, msg.message(), mockPartition, offset);
+            List<Object> values = lists.iterator().next(); 
+            assertEquals("Message is incorrect", value, values.get(0));
+            assertEquals("Partition is incorrect", mockPartition.partition, values.get(1));
+            assertEquals("Offset is incorrect", offset, values.get(2));
+        }
+    }
 
     private ByteBufferMessageSet getLastMessage() {
         long offsetOfLastMessage = KafkaUtils.getOffset(simpleConsumer, config.topic, 0, OffsetRequest.LatestTime()) - 1;
-        return KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), offsetOfLastMessage);
+        return KafkaUtils.fetchMessages(config, simpleConsumer, new Partition(Broker.fromString(broker.getBrokerConnectionString()), TEST_TOPIC, 0), offsetOfLastMessage);
     }
 
     private void runGetValueOnlyTuplesTest() {
         String value = "value";
+        
         createTopicAndSendMessage(null, value);
         ByteBufferMessageSet messageAndOffsets = getLastMessage();
         for (MessageAndOffset msg : messageAndOffsets) {
-            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message());
+            Iterable<List<Object>> lists = KafkaUtils.generateTuples(config, msg.message(), config.topic);
             assertEquals(value, lists.iterator().next().get(0));
         }
     }
@@ -195,7 +234,7 @@
 
     private void createTopicAndSendMessage(String key, String value) {
         Properties p = new Properties();
-        p.put("serializer.class", "kafka.serializer.StringEncoder");
+        p.put("acks", "1");
         p.put("bootstrap.servers", broker.getBrokerConnectionString());
         p.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
         p.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
@@ -225,27 +264,32 @@
     public void assignAllPartitionsToOneTask() {
         runPartitionToTaskMappingTest(32, 32);
     }
-
-
+    
     public void runPartitionToTaskMappingTest(int numPartitions, int partitionsPerTask) {
         GlobalPartitionInformation globalPartitionInformation = TestUtils.buildPartitionInfo(numPartitions);
+        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
+        partitions.add(globalPartitionInformation);
         int numTasks = numPartitions / partitionsPerTask;
         for (int i = 0 ; i < numTasks ; i++) {
-            assertEquals(partitionsPerTask, KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, numTasks, i).size());
+            assertEquals(partitionsPerTask, KafkaUtils.calculatePartitionsForTask(partitions, numTasks, i).size());
         }
     }
 
     @Test
     public void moreTasksThanPartitions() {
         GlobalPartitionInformation globalPartitionInformation = TestUtils.buildPartitionInfo(1);
+        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
+        partitions.add(globalPartitionInformation);
         int numTasks = 2;
-        assertEquals(1, KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, numTasks, 0).size());
-        assertEquals(0, KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, numTasks, 1).size());
+        assertEquals(1, KafkaUtils.calculatePartitionsForTask(partitions, numTasks, 0).size());
+        assertEquals(0, KafkaUtils.calculatePartitionsForTask(partitions, numTasks, 1).size());
     }
 
     @Test (expected = IllegalArgumentException.class )
     public void assignInvalidTask() {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation();
-        KafkaUtils.calculatePartitionsForTask(globalPartitionInformation, 1, 1);
+        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TEST_TOPIC);
+        List<GlobalPartitionInformation> partitions = new ArrayList<GlobalPartitionInformation>();
+        partitions.add(globalPartitionInformation);
+        KafkaUtils.calculatePartitionsForTask(partitions, 1, 1);
     }
 }
diff --git a/external/storm-kafka/src/test/storm/kafka/StringKeyValueSchemeTest.java b/external/storm-kafka/src/test/storm/kafka/StringKeyValueSchemeTest.java
index 0b786ba..eddb900 100644
--- a/external/storm-kafka/src/test/storm/kafka/StringKeyValueSchemeTest.java
+++ b/external/storm-kafka/src/test/storm/kafka/StringKeyValueSchemeTest.java
@@ -21,7 +21,9 @@
 import com.google.common.collect.ImmutableMap;
 import org.junit.Test;
 
-import java.util.Arrays;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.Collections;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -32,7 +34,7 @@
 
     @Test
     public void testDeserialize() throws Exception {
-        assertEquals(Arrays.asList("test"), scheme.deserialize("test".getBytes()));
+        assertEquals(Collections.singletonList("test"), scheme.deserialize(wrapString("test")));
     }
 
     @Test
@@ -44,12 +46,17 @@
 
     @Test
     public void testDeserializeWithNullKeyAndValue() throws Exception {
-        assertEquals(Arrays.asList("test"), scheme.deserializeKeyAndValue(null, "test".getBytes()));
+        assertEquals(Collections.singletonList("test"),
+            scheme.deserializeKeyAndValue(null, wrapString("test")));
     }
 
     @Test
     public void testDeserializeWithKeyAndValue() throws Exception {
-        assertEquals(Arrays.asList(ImmutableMap.of("key", "test")),
-                scheme.deserializeKeyAndValue("key".getBytes(), "test".getBytes()));
+        assertEquals(Collections.singletonList(ImmutableMap.of("key", "test")),
+                scheme.deserializeKeyAndValue(wrapString("key"), wrapString("test")));
+    }
+
+    private static ByteBuffer wrapString(String s) {
+        return ByteBuffer.wrap(s.getBytes(Charset.defaultCharset()));
     }
 }
diff --git a/external/storm-kafka/src/test/storm/kafka/TestStringScheme.java b/external/storm-kafka/src/test/storm/kafka/TestStringScheme.java
new file mode 100644
index 0000000..ae36409
--- /dev/null
+++ b/external/storm-kafka/src/test/storm/kafka/TestStringScheme.java
@@ -0,0 +1,40 @@
+/*
+ * 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 storm.kafka;
+
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStringScheme {
+  @Test
+  public void testDeserializeString() {
+    String s = "foo";
+    byte[] bytes = s.getBytes(StandardCharsets.UTF_8);
+    ByteBuffer direct = ByteBuffer.allocateDirect(bytes.length);
+    direct.put(bytes);
+    direct.flip();
+    String s1 = StringScheme.deserializeString(ByteBuffer.wrap(bytes));
+    String s2 = StringScheme.deserializeString(direct);
+    assertEquals(s, s1);
+    assertEquals(s, s2);
+  }
+}
diff --git a/external/storm-kafka/src/test/storm/kafka/TestUtils.java b/external/storm-kafka/src/test/storm/kafka/TestUtils.java
index fa29992..3e69160 100644
--- a/external/storm-kafka/src/test/storm/kafka/TestUtils.java
+++ b/external/storm-kafka/src/test/storm/kafka/TestUtils.java
@@ -28,7 +28,7 @@
 import storm.kafka.trident.GlobalPartitionInformation;
 
 import java.nio.ByteBuffer;
-import java.util.Properties;
+import java.util.*;
 
 import static org.junit.Assert.assertEquals;
 
@@ -40,9 +40,14 @@
         return buildPartitionInfo(numPartitions, 9092);
     }
 
+    public static List<GlobalPartitionInformation> buildPartitionInfoList(GlobalPartitionInformation partitionInformation) {
+        List<GlobalPartitionInformation> map = new ArrayList<GlobalPartitionInformation>();
+        map.add(partitionInformation);
+        return map;
+    }
 
     public static GlobalPartitionInformation buildPartitionInfo(int numPartitions, int brokerPort) {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation();
+        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TOPIC);
         for (int i = 0; i < numPartitions; i++) {
             globalPartitionInformation.addPartition(i, Broker.fromString("broker-" + i + " :" + brokerPort));
         }
@@ -63,27 +68,24 @@
     }
 
     private static BrokerHosts getBrokerHosts(KafkaTestBroker broker) {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation();
+        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TOPIC);
         globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString()));
         return new StaticHosts(globalPartitionInformation);
     }
 
-    public static Config getConfig(String brokerConnectionString) {
-        Config config = new Config();
+    public static Properties getProducerProperties(String brokerConnectionString) {
         Properties props = new Properties();
-        props.put("metadata.broker.list", brokerConnectionString);
-        props.put("request.required.acks", "1");
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
-        config.put(KafkaBolt.KAFKA_BROKER_PROPERTIES, props);
-        config.put(KafkaBolt.TOPIC, TOPIC);
-
-        return config;
+        props.put("bootstrap.servers", brokerConnectionString);
+        props.put("acks", "1");
+        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        return props;
     }
 
     public static boolean verifyMessage(String key, String message, KafkaTestBroker broker, SimpleConsumer simpleConsumer) {
         long lastMessageOffset = KafkaUtils.getOffset(simpleConsumer, TestUtils.TOPIC, 0, OffsetRequest.LatestTime()) - 1;
         ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(TestUtils.getKafkaConfig(broker), simpleConsumer,
-                new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), lastMessageOffset);
+                new Partition(Broker.fromString(broker.getBrokerConnectionString()),TestUtils.TOPIC, 0), lastMessageOffset);
         MessageAndOffset messageAndOffset = messageAndOffsets.iterator().next();
         Message kafkaMessage = messageAndOffset.message();
         ByteBuffer messageKeyBuffer = kafkaMessage.key();
diff --git a/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java b/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java
index d8a5e24..8213b07 100644
--- a/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java
+++ b/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java
@@ -17,7 +17,6 @@
  */
 package storm.kafka;
 
-import backtype.storm.Config;
 import backtype.storm.tuple.Fields;
 import kafka.javaapi.consumer.SimpleConsumer;
 import org.junit.After;
@@ -37,22 +36,18 @@
 public class TridentKafkaTest {
     private KafkaTestBroker broker;
     private TridentKafkaState state;
-    private Config config;
     private SimpleConsumer simpleConsumer;
-    private TridentTupleToKafkaMapper mapper;
-    private KafkaTopicSelector topicSelector;
 
     @Before
     public void setup() {
         broker = new KafkaTestBroker();
         simpleConsumer = TestUtils.getKafkaConsumer(broker);
-        config = TestUtils.getConfig(broker.getBrokerConnectionString());
-        mapper = new FieldNameBasedTupleToKafkaMapper("key", "message");
-        topicSelector = new DefaultTopicSelector(TestUtils.TOPIC);
+        TridentTupleToKafkaMapper mapper = new FieldNameBasedTupleToKafkaMapper("key", "message");
+        KafkaTopicSelector topicSelector = new DefaultTopicSelector(TestUtils.TOPIC);
         state = new TridentKafkaState()
                 .withKafkaTopicSelector(topicSelector)
                 .withTridentTupleToKafkaMapper(mapper);
-        state.prepare(config);
+        state.prepare(TestUtils.getProducerProperties(broker.getBrokerConnectionString()));
     }
 
     @Test
@@ -71,7 +66,7 @@
     }
 
     private List<TridentTuple> generateTupleBatch(String key, String message, int batchsize) {
-        List<TridentTuple> batch = new ArrayList<TridentTuple>();
+        List<TridentTuple> batch = new ArrayList<>();
         for(int i =0 ; i < batchsize; i++) {
             batch.add(TridentTupleView.createFreshTuple(new Fields("key", "message"), key, message));
         }
diff --git a/external/storm-kafka/src/test/storm/kafka/TridentKafkaTopology.java b/external/storm-kafka/src/test/storm/kafka/TridentKafkaTopology.java
index 9cb7bbf..b9e25e4 100644
--- a/external/storm-kafka/src/test/storm/kafka/TridentKafkaTopology.java
+++ b/external/storm-kafka/src/test/storm/kafka/TridentKafkaTopology.java
@@ -22,7 +22,7 @@
 import backtype.storm.generated.StormTopology;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Values;
-import storm.kafka.trident.TridentKafkaState;
+import com.google.common.collect.ImmutableMap;
 import storm.kafka.trident.TridentKafkaStateFactory;
 import storm.kafka.trident.TridentKafkaUpdater;
 import storm.kafka.trident.mapper.FieldNameBasedTupleToKafkaMapper;
@@ -31,14 +31,11 @@
 import storm.trident.TridentTopology;
 import storm.trident.testing.FixedBatchSpout;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Properties;
 
-
 public class TridentKafkaTopology {
 
-    private static StormTopology buildTopology() {
+    private static StormTopology buildTopology(String brokerConnectionString) {
         Fields fields = new Fields("word", "count");
         FixedBatchSpout spout = new FixedBatchSpout(fields, 4,
                 new Values("storm", "1"),
@@ -51,9 +48,16 @@
         TridentTopology topology = new TridentTopology();
         Stream stream = topology.newStream("spout1", spout);
 
+        Properties props = new Properties();
+        props.put("bootstrap.servers", brokerConnectionString);
+        props.put("acks", "1");
+        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+
         TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
-                .withKafkaTopicSelector(new DefaultTopicSelector("test"))
-                .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count"));
+            .withProducerProperties(props)
+            .withKafkaTopicSelector(new DefaultTopicSelector("test"))
+            .withTridentTupleToKafkaMapper(new FieldNameBasedTupleToKafkaMapper("word", "count"));
         stream.partitionPersist(stateFactory, fields, new TridentKafkaUpdater(), new Fields());
 
         return topology.build();
@@ -77,24 +81,11 @@
             System.out.println("Please provide kafka broker url ,e.g. localhost:9092");
         }
 
-        Config conf = getConfig(args[0]);
         LocalCluster cluster = new LocalCluster();
-        cluster.submitTopology("wordCounter", conf, buildTopology());
+        cluster.submitTopology("wordCounter", new Config(), buildTopology(args[0]));
         Thread.sleep(60 * 1000);
         cluster.killTopology("wordCounter");
 
         cluster.shutdown();
     }
-
-    private  static Config getConfig(String brokerConnectionString) {
-        Config conf = new Config();
-        Map config = new HashMap();
-        Properties props = new Properties();
-        props.put("metadata.broker.list", brokerConnectionString);
-        props.put("request.required.acks", "1");
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
-        conf.put(TridentKafkaState.KAFKA_BROKER_PROPERTIES, props);
-        return conf;
-    }
-
 }
diff --git a/external/storm-kafka/src/test/storm/kafka/ZkCoordinatorTest.java b/external/storm-kafka/src/test/storm/kafka/ZkCoordinatorTest.java
index 3259eea..48ca60f 100644
--- a/external/storm-kafka/src/test/storm/kafka/ZkCoordinatorTest.java
+++ b/external/storm-kafka/src/test/storm/kafka/ZkCoordinatorTest.java
@@ -61,7 +61,7 @@
         Map conf = buildZookeeperConfig(server);
         state = new ZkState(conf);
         simpleConsumer = new SimpleConsumer("localhost", broker.getPort(), 60000, 1024, "testClient");
-        when(dynamicPartitionConnections.register(any(Broker.class), anyInt())).thenReturn(simpleConsumer);
+        when(dynamicPartitionConnections.register(any(Broker.class), any(String.class) ,anyInt())).thenReturn(simpleConsumer);
     }
 
     private Map buildZookeeperConfig(TestingServer server) {
@@ -87,7 +87,7 @@
         int totalTasks = 64;
         int partitionsPerTask = 1;
         List<ZkCoordinator> coordinatorList = buildCoordinators(totalTasks / partitionsPerTask);
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfo(totalTasks));
+        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks)));
         for (ZkCoordinator coordinator : coordinatorList) {
             List<PartitionManager> myManagedPartitions = coordinator.getMyManagedPartitions();
             assertEquals(partitionsPerTask, myManagedPartitions.size());
@@ -101,10 +101,10 @@
         final int totalTasks = 64;
         int partitionsPerTask = 2;
         List<ZkCoordinator> coordinatorList = buildCoordinators(totalTasks / partitionsPerTask);
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfo(totalTasks, 9092));
+        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9092)));
         List<List<PartitionManager>> partitionManagersBeforeRefresh = getPartitionManagers(coordinatorList);
         waitForRefresh();
-        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfo(totalTasks, 9093));
+        when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9093)));
         List<List<PartitionManager>> partitionManagersAfterRefresh = getPartitionManagers(coordinatorList);
         assertEquals(partitionManagersAfterRefresh.size(), partitionManagersAfterRefresh.size());
         Iterator<List<PartitionManager>> iterator = partitionManagersAfterRefresh.iterator();
diff --git a/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java
index 53d7c50..87daab0 100644
--- a/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java
+++ b/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java
@@ -79,7 +79,7 @@
     }
 
     private void setupKafkaConsumer() {
-        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation();
+        GlobalPartitionInformation globalPartitionInformation = new GlobalPartitionInformation(TEST_TOPIC);
         globalPartitionInformation.addPartition(0, Broker.fromString(broker.getBrokerConnectionString()));
         BrokerHosts brokerHosts = new StaticHosts(globalPartitionInformation);
         kafkaConfig = new KafkaConfig(brokerHosts, TEST_TOPIC);
@@ -183,31 +183,27 @@
     }
 
     private KafkaBolt generateStringSerializerBolt() {
-        KafkaBolt bolt = new KafkaBolt();
         Properties props = new Properties();
-        props.put("request.required.acks", "1");
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
+        props.put("acks", "1");
         props.put("bootstrap.servers", broker.getBrokerConnectionString());
         props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
         props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
         props.put("metadata.fetch.timeout.ms", 1000);
-        config.put(KafkaBolt.KAFKA_BROKER_PROPERTIES, props);
+        KafkaBolt bolt = new KafkaBolt().withProducerProperties(props);
         bolt.prepare(config, null, new OutputCollector(collector));
         bolt.setAsync(false);
         return bolt;
     }
 
     private KafkaBolt generateDefaultSerializerBolt(boolean async, boolean fireAndForget) {
-        KafkaBolt bolt = new KafkaBolt();
         Properties props = new Properties();
-        props.put("request.required.acks", "1");
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
+        props.put("acks", "1");
         props.put("bootstrap.servers", broker.getBrokerConnectionString());
         props.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
         props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
         props.put("metadata.fetch.timeout.ms", 1000);
         props.put("linger.ms", 0);
-        config.put(KafkaBolt.KAFKA_BROKER_PROPERTIES, props);
+        KafkaBolt bolt = new KafkaBolt().withProducerProperties(props);
         bolt.prepare(config, null, new OutputCollector(collector));
         bolt.setAsync(async);
         bolt.setFireAndForget(fireAndForget);
@@ -216,8 +212,7 @@
 
     private KafkaBolt defaultSerializerBoltWithSpecifiedProperties(boolean async, boolean fireAndForget) {
         Properties props = new Properties();
-        props.put("request.required.acks", "1");
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
+        props.put("acks", "1");
         props.put("bootstrap.servers", broker.getBrokerConnectionString());
         props.put("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
         props.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer");
@@ -252,7 +247,7 @@
     private boolean verifyMessage(String key, String message) {
         long lastMessageOffset = KafkaUtils.getOffset(simpleConsumer, kafkaConfig.topic, 0, OffsetRequest.LatestTime()) - 1;
         ByteBufferMessageSet messageAndOffsets = KafkaUtils.fetchMessages(kafkaConfig, simpleConsumer,
-                new Partition(Broker.fromString(broker.getBrokerConnectionString()), 0), lastMessageOffset);
+                new Partition(Broker.fromString(broker.getBrokerConnectionString()),kafkaConfig.topic, 0), lastMessageOffset);
         MessageAndOffset messageAndOffset = messageAndOffsets.iterator().next();
         Message kafkaMessage = messageAndOffset.message();
         ByteBuffer messageKeyBuffer = kafkaMessage.key();
diff --git a/external/storm-metrics/pom.xml b/external/storm-metrics/pom.xml
new file mode 100644
index 0000000..a2fa3ec
--- /dev/null
+++ b/external/storm-metrics/pom.xml
@@ -0,0 +1,107 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+      <artifactId>storm</artifactId>
+      <groupId>org.apache.storm</groupId>
+      <version>0.11.0-SNAPSHOT</version>
+      <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.storm</groupId>
+  <artifactId>storm-metrics</artifactId>
+  <packaging>jar</packaging>
+
+  <name>storm-metrics</name>
+
+  <properties>
+    <!-- settings for downloading the sigar native binary complete archive, which is not available in Maven central-->
+    <sigar.version>1.6.4</sigar.version>
+    <sigar.download.url>https://magelan.googlecode.com/files/hyperic-sigar-${sigar.version}.zip</sigar.download.url>
+    <sigar.SHA1>8f79d4039ca3ec6c88039d5897a80a268213e6b7</sigar.SHA1>
+    <!-- this will download the sigar ZIP to the local maven repository next to the sigar dependencies,
+         so we only download it once -->
+    <sigar.download.path>${settings.localRepository}/org/fusesource/sigar/${sigar.version}</sigar.download.path>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.hdrhistogram</groupId>
+      <artifactId>HdrHistogram</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.fusesource</groupId>
+      <artifactId>sigar</artifactId>
+      <version>${sigar.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>log4j</groupId>
+          <artifactId>log4j</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.storm</groupId>
+      <artifactId>storm-core</artifactId>
+      <version>${project.version}</version>
+      <!-- keep storm out of the jar-with-dependencies -->
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <version>1.6</version>
+        <executions>
+          <execution>
+            <id>prepare</id>
+            <phase>validate</phase>
+            <configuration>
+              <tasks>
+                <echo message="Downloading sigar native binaries..."/>
+                <get src="${sigar.download.url}"
+                     dest="${sigar.download.path}/" skipExisting="true"/>
+                <checksum file="${sigar.download.path}/hyperic-sigar-${sigar.version}.zip" algorithm="SHA1"
+                          property="${sigar.SHA1}" verifyProperty="validChecksum"/>
+                <fail message="Checksum validation failed for hyperic-sigar-${sigar.version}.zip">
+                  <condition>
+                    <isfalse value="${validChecksum}"/>
+                  </condition>
+                </fail>
+                <unzip src="${sigar.download.path}/hyperic-sigar-${sigar.version}.zip"
+                       dest="${project.build.directory}/classes/resources">
+                  <patternset>
+                    <include name="**/lib/libsigar-*"/>
+                    <include name="**/lib/sigar-*"/>
+                  </patternset>
+                  <mapper type="flatten"/>
+                </unzip>
+              </tasks>
+            </configuration>
+            <goals>
+              <goal>run</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>
diff --git a/external/storm-metrics/src/main/java/org/apache/storm/metrics/hdrhistogram/HistogramMetric.java b/external/storm-metrics/src/main/java/org/apache/storm/metrics/hdrhistogram/HistogramMetric.java
new file mode 100644
index 0000000..4adc500
--- /dev/null
+++ b/external/storm-metrics/src/main/java/org/apache/storm/metrics/hdrhistogram/HistogramMetric.java
@@ -0,0 +1,79 @@
+/**
+ * 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 org.apache.storm.metrics.hdrhistogram;
+
+import backtype.storm.metric.api.IMetric;
+import org.HdrHistogram.Histogram;
+
+/**
+ * A metric wrapping an HdrHistogram.
+ */
+public class HistogramMetric implements IMetric {
+    private final Histogram _histo;
+
+
+    public HistogramMetric(final int numberOfSignificantValueDigits) {
+        this(null, numberOfSignificantValueDigits);
+    }
+
+    public HistogramMetric(Long highestTrackableValue, final int numberOfSignificantValueDigits) {
+        this(null, highestTrackableValue, numberOfSignificantValueDigits);
+    }
+
+    /**
+     * (From the Constructor of Histogram)
+     * Construct a Histogram given the Lowest and Highest values to be tracked and a number of significant
+     * decimal digits. Providing a lowestDiscernibleValue is useful is situations where the units used
+     * for the histogram's values are much smaller that the minimal accuracy required. E.g. when tracking
+     * time values stated in nanosecond units, where the minimal accuracy required is a microsecond, the
+     * proper value for lowestDiscernibleValue would be 1000.
+     *
+     * @param lowestDiscernibleValue         The lowest value that can be discerned (distinguished from 0) by the
+     *                                       histogram. Must be a positive integer that is {@literal >=} 1. May be
+     *                                       internally rounded down to nearest power of 2
+     *                                       (if null 1 is used).
+     * @param highestTrackableValue          The highest value to be tracked by the histogram. Must be a positive
+     *                                       integer that is {@literal >=} (2 * lowestDiscernibleValue).
+     *                                       (if null 2 * lowestDiscernibleValue is used and auto-resize is enabled)
+     * @param numberOfSignificantValueDigits Specifies the precision to use. This is the number of significant
+     *                                       decimal digits to which the histogram will maintain value resolution
+     *                                       and separation. Must be a non-negative integer between 0 and 5.
+     */
+    public HistogramMetric(Long lowestDiscernibleValue, Long highestTrackableValue,
+                     final int numberOfSignificantValueDigits) {
+        boolean autoResize = false;
+        if (lowestDiscernibleValue == null) lowestDiscernibleValue = 1L;
+        if (highestTrackableValue == null) {
+            highestTrackableValue = 2 * lowestDiscernibleValue;
+            autoResize = true;
+        }
+        _histo = new Histogram(lowestDiscernibleValue, highestTrackableValue, numberOfSignificantValueDigits);
+        if (autoResize) _histo.setAutoResize(true);
+    }
+
+    public void recordValue(long val) {
+        _histo.recordValue(val);
+    }
+
+    @Override
+    public Object getValueAndReset() {
+          Histogram copy = _histo.copy();
+          _histo.reset();
+          return copy;
+    }
+}
diff --git a/external/storm-metrics/src/main/java/org/apache/storm/metrics/sigar/CPUMetric.java b/external/storm-metrics/src/main/java/org/apache/storm/metrics/sigar/CPUMetric.java
new file mode 100644
index 0000000..a3addc9
--- /dev/null
+++ b/external/storm-metrics/src/main/java/org/apache/storm/metrics/sigar/CPUMetric.java
@@ -0,0 +1,60 @@
+/**
+ * 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 org.apache.storm.metrics.sigar;
+
+import org.hyperic.sigar.Sigar;
+import org.hyperic.sigar.ProcCpu;
+
+import backtype.storm.metric.api.IMetric;
+
+import java.util.HashMap;
+
+/**
+ * A metric using Sigar to get User and System CPU utilization for a worker.
+ */
+public class CPUMetric implements IMetric {
+    private long _prevUser = 0;
+    private long _prevSys = 0;
+    private final Sigar _sigar;
+    private final long _pid;
+
+    public CPUMetric() {
+        _sigar = new Sigar();
+        _pid = _sigar.getPid();
+    }
+
+    @Override
+    public Object getValueAndReset() {
+        try {
+          ProcCpu cpu = _sigar.getProcCpu(_pid);
+          long userTotal = cpu.getUser();
+          long sysTotal = cpu.getSys();
+          long user = userTotal - _prevUser;
+          long sys = sysTotal - _prevSys;
+          _prevUser = userTotal;
+          _prevSys = sysTotal;
+
+          HashMap<String, Long> ret = new HashMap<String, Long>();
+          ret.put("user-ms", user);
+          ret.put("sys-ms", sys);
+          return ret;
+      } catch (Exception e) {
+          throw new RuntimeException(e);
+      }
+    }
+}
diff --git a/external/storm-solr/pom.xml b/external/storm-solr/pom.xml
index 2b5c742..d5b7622 100644
--- a/external/storm-solr/pom.xml
+++ b/external/storm-solr/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--/**
+        * 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.
+        */-->
 <project xmlns="http://maven.apache.org/POM/4.0.0"
          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
@@ -21,7 +38,7 @@
     </developers>
 
     <dependencies>
-        <dependency>
+       <dependency>
             <groupId>org.apache.storm</groupId>
             <artifactId>storm-core</artifactId>
             <version>${project.version}</version>
@@ -95,4 +112,4 @@
             </plugin>
         </plugins>
     </build>
-</project>
\ No newline at end of file
+</project>
diff --git a/log4j2/cluster.xml b/log4j2/cluster.xml
index bb96646..f349d8c 100644
--- a/log4j2/cluster.xml
+++ b/log4j2/cluster.xml
@@ -19,12 +19,12 @@
 <configuration monitorInterval="60">
 <properties>
     <property name="pattern">%d{yyyy-MM-dd HH:mm:ss.SSS} %c{1.} [%p] %msg%n</property>
-    <property name="patternMetris">%d %-8r %m%n</property>
+    <property name="patternMetrics">%d %-8r %m%n</property>
 </properties>
 <appenders>
-    <RollingFile name="A1"
+    <RollingFile name="A1" immediateFlush="false"
                  fileName="${sys:storm.log.dir}/${sys:logfile.name}"
-                 filePattern="${sys:storm.log.dir}/${sys:logfile.name}.%i">
+                 filePattern="${sys:storm.log.dir}/${sys:logfile.name}.%i.gz">
         <PatternLayout>
             <pattern>${pattern}</pattern>
         </PatternLayout>
@@ -33,9 +33,9 @@
         </Policies>
         <DefaultRolloverStrategy max="9"/>
     </RollingFile>
-    <RollingFile name="ACCESS"
-                 fileName="${sys:storm.log.dir}/access.log"
-                 filePattern="${sys:storm.log.dir}/access.log.%i">
+    <RollingFile name="WEB-ACCESS" immediateFlush="false"
+                 fileName="${sys:storm.log.dir}/access-web-${sys:daemon.name}.log"
+                 filePattern="${sys:storm.log.dir}/access-web-${sys:daemon.name}.log.%i.gz">
         <PatternLayout>
             <pattern>${pattern}</pattern>
         </PatternLayout>
@@ -44,26 +44,42 @@
         </Policies>
         <DefaultRolloverStrategy max="9"/>
     </RollingFile>
-    <RollingFile name="METRICS"
+    <RollingFile name="THRIFT-ACCESS" immediateFlush="false"
+                 fileName="${sys:storm.log.dir}/access-${sys:logfile.name}"
+                 filePattern="${sys:storm.log.dir}/access-${sys:logfile.name}.%i.gz">
+    <PatternLayout>
+        <pattern>${pattern}</pattern>
+    </PatternLayout>
+        <Policies>
+            <SizeBasedTriggeringPolicy size="100 MB"/> <!-- Or every 100 MB -->
+        </Policies>
+        <DefaultRolloverStrategy max="9"/>
+    </RollingFile>
+    <RollingFile name="METRICS" immediateFlush="false"
                  fileName="${sys:storm.log.dir}/metrics.log"
-                 filePattern="${sys:storm.log.dir}/metrics.log.%i">
+                 filePattern="${sys:storm.log.dir}/metrics.log.%i.gz">
         <PatternLayout>
-            <pattern>${patternMetris}</pattern>
+            <pattern>${patternMetrics}</pattern>
         </PatternLayout>
         <Policies>
             <SizeBasedTriggeringPolicy size="2 MB"/> <!-- Or every 100 MB -->
         </Policies>
         <DefaultRolloverStrategy max="9"/>
     </RollingFile>
-    <Syslog name="syslog" format="RFC5424" host="localhost" port="514"
+    <Syslog name="syslog" format="RFC5424" charset="UTF-8" host="localhost" port="514"
             protocol="UDP" appName="[${sys:daemon.name}]" mdcId="mdc" includeMDC="true"
             facility="LOCAL5" enterpriseNumber="18060" newLine="true" exceptionPattern="%rEx{full}"
-            messageId="[${sys:user.name}:S0]" id="storm"/>
+            messageId="[${sys:user.name}:S0]" id="storm" immediateFlush="false" immediateFail="true"/>
 </appenders>
 <loggers>
 
-    <Logger name="backtype.storm.security.auth.authorizer" level="info">
-        <AppenderRef ref="ACCESS"/>
+    <Logger name="backtype.storm.logging.filters.AccessLoggingFilter" level="info" additivity="false">
+        <AppenderRef ref="WEB-ACCESS"/>
+        <AppenderRef ref="syslog"/>
+    </Logger>
+    <Logger name="backtype.storm.logging.ThriftAccessLogger" level="info" additivity="false">
+        <AppenderRef ref="THRIFT-ACCESS"/>
+        <AppenderRef ref="syslog"/>
     </Logger>
     <Logger name="backtype.storm.metric.LoggingMetricsConsumer" level="info">
         <AppenderRef ref="METRICS"/>
diff --git a/log4j2/worker.xml b/log4j2/worker.xml
index 2017699..df368c6 100644
--- a/log4j2/worker.xml
+++ b/log4j2/worker.xml
@@ -22,9 +22,9 @@
     <property name="patternNoTime">%msg%n</property>
 </properties>
 <appenders>
-    <RollingFile name="A1"
-                 fileName="${sys:storm.log.dir}/${sys:logfile.name}"
-                 filePattern="${sys:storm.log.dir}/${sys:logfile.name}.%i.gz">
+    <RollingFile name="A1" immediateFlush="false"
+		fileName="${sys:workers.artifacts}/${sys:storm.id}/${sys:worker.port}/${sys:logfile.name}"
+		filePattern="${sys:workers.artifacts}/${sys:storm.id}/${sys:worker.port}/${sys:logfile.name}.%i.gz">
         <PatternLayout>
             <pattern>${pattern}</pattern>
         </PatternLayout>
@@ -33,9 +33,9 @@
         </Policies>
         <DefaultRolloverStrategy max="9"/>
     </RollingFile>
-    <RollingFile name="STDOUT"
-                 fileName="${sys:storm.log.dir}/${sys:logfile.name}.out"
-                 filePattern="${sys:storm.log.dir}/${sys:logfile.name}.out.%i.gz">
+    <RollingFile name="STDOUT" immediateFlush="false"
+		fileName="${sys:workers.artifacts}/${sys:storm.id}/${sys:worker.port}/${sys:logfile.name}.out"
+		filePattern="${sys:workers.artifacts}/${sys:storm.id}/${sys:worker.port}/${sys:logfile.name}.out.%i.gz">
         <PatternLayout>
             <pattern>${patternNoTime}</pattern>
         </PatternLayout>
@@ -44,9 +44,9 @@
         </Policies>
         <DefaultRolloverStrategy max="4"/>
     </RollingFile>
-    <RollingFile name="STDERR"
-                 fileName="${sys:storm.log.dir}/${sys:logfile.name}.err"
-                 filePattern="${sys:storm.log.dir}/${sys:logfile.name}.err.%i.gz">
+    <RollingFile name="STDERR" immediateFlush="false"
+		fileName="${sys:workers.artifacts}/${sys:storm.id}/${sys:worker.port}/${sys:logfile.name}.err"
+		filePattern="${sys:workers.artifacts}/${sys:storm.id}/${sys:worker.port}/${sys:logfile.name}.err.%i.gz">
         <PatternLayout>
             <pattern>${patternNoTime}</pattern>
         </PatternLayout>
@@ -55,10 +55,10 @@
         </Policies>
         <DefaultRolloverStrategy max="4"/>
     </RollingFile>
-    <Syslog name="syslog" format="RFC5424" host="localhost" port="514"
+    <Syslog name="syslog" format="RFC5424" charset="UTF-8" host="localhost" port="514"
         protocol="UDP" appName="[${sys:storm.id}:${sys:worker.port}]" mdcId="mdc" includeMDC="true"
         facility="LOCAL5" enterpriseNumber="18060" newLine="true" exceptionPattern="%rEx{full}"
-        messageId="[${sys:user.name}:${sys:logging.sensitivity}]" id="storm"/>
+        messageId="[${sys:user.name}:${sys:logging.sensitivity}]" id="storm" immediateFlush="false" immediateFail="true"/>
 </appenders>
 <loggers>
     <root level="info"> <!-- We log everything -->
diff --git a/pom.xml b/pom.xml
index faf94ab..3ed8e3f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -154,27 +154,6 @@
         <maven>3.0.0</maven>
     </prerequisites>
 
-    <modules>
-        <module>storm-buildtools/maven-shade-clojure-transformer</module>
-        <module>storm-buildtools/storm-maven-plugins</module>
-        <module>storm-multilang/javascript</module>
-        <module>storm-multilang/python</module>
-        <module>storm-multilang/ruby</module>
-        <module>storm-core</module>
-        <module>external/storm-kafka</module>
-        <module>external/storm-hdfs</module>
-        <module>external/storm-hbase</module>
-        <module>external/storm-hive</module>
-        <module>external/storm-jdbc</module>
-        <module>external/storm-redis</module>
-        <module>external/storm-eventhubs</module>
-        <module>external/flux</module>
-        <module>external/storm-elasticsearch</module>
-        <module>external/storm-solr</module>
-        <module>external/sql</module>
-        <module>examples/storm-starter</module>
-    </modules>
-
     <scm>
         <connection>scm:git:https://git-wip-us.apache.org/repos/asf/storm.git</connection>
         <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/storm.git</developerConnection>
@@ -192,16 +171,20 @@
         <test.extra.args>-Djava.net.preferIPv4Stack=true</test.extra.args>
 
         <!-- dependency versions -->
-        <clojure.version>1.6.0</clojure.version>
+        <clojure.version>1.7.0</clojure.version>
+        <java_jmx.version>0.3.1</java_jmx.version>
         <compojure.version>1.1.3</compojure.version>
         <hiccup.version>0.3.6</hiccup.version>
         <commons-io.version>2.4</commons-io.version>
         <commons-lang.version>2.5</commons-lang.version>
         <commons-exec.version>1.1</commons-exec.version>
+        <commons-fileupload.version>1.2.1</commons-fileupload.version>
+        <commons-codec.version>1.6</commons-codec.version>
         <clj-time.version>0.8.0</clj-time.version>
-        <curator.version>2.5.0</curator.version>
+        <curator.version>2.9.0</curator.version>
         <json-simple.version>1.1</json-simple.version>
         <ring.version>1.3.0</ring.version>
+        <ring-json.version>0.3.1</ring-json.version>
         <jetty.version>7.6.13.v20130916</jetty.version>
         <clojure.tools.logging.version>0.2.3</clojure.tools.logging.version>
         <clojure.math.numeric-tower.version>0.0.1</clojure.math.numeric-tower.version>
@@ -209,12 +192,14 @@
         <snakeyaml.version>1.11</snakeyaml.version>
         <httpclient.version>4.3.3</httpclient.version>
         <clojure.tools.cli.version>0.2.4</clojure.tools.cli.version>
-        <disruptor.version>2.10.4</disruptor.version>
+        <disruptor.version>3.3.2</disruptor.version>
         <jgrapht.version>0.9.0</jgrapht.version>
         <guava.version>16.0.1</guava.version>
         <netty.version>3.9.0.Final</netty.version>
         <log4j-over-slf4j.version>1.6.6</log4j-over-slf4j.version>
         <log4j.version>2.1</log4j.version>
+        <slf4j.version>1.7.7</slf4j.version>
+        <metrics.version>3.1.0</metrics.version>
         <clojure.tools.nrepl.version>0.2.3</clojure.tools.nrepl.version>
         <clojure-complete.version>0.2.3</clojure-complete.version>
         <mockito.version>1.9.5</mockito.version>
@@ -225,10 +210,39 @@
         <hive.version>0.14.0</hive.version>
         <hadoop.version>2.6.0</hadoop.version>
         <kryo.version>2.21</kryo.version>
+        <servlet.version>2.5</servlet.version>
+        <joda-time.version>2.3</joda-time.version>
+        <jackson.version>2.3.1</jackson.version>
+        <thrift.version>0.9.3</thrift.version>
+        <junit.version>4.11</junit.version>
+        <metrics-clojure.version>2.5.1</metrics-clojure.version>
+        <hdrhistogram.version>2.1.7</hdrhistogram.version>
         <calcite.version>1.4.0-incubating</calcite.version>
-        <jackson.version>2.6.3</jackson.version>
     </properties>
 
+    <modules>
+        <module>storm-multilang/javascript</module>
+        <module>storm-multilang/python</module>
+        <module>storm-multilang/ruby</module>
+        <module>storm-buildtools/maven-shade-clojure-transformer</module>
+        <module>storm-buildtools/storm-maven-plugins</module>
+        <module>storm-core</module>
+        <module>external/storm-kafka</module>
+        <module>external/storm-hdfs</module>
+        <module>external/storm-hbase</module>
+        <module>external/storm-hive</module>
+        <module>external/storm-jdbc</module>
+        <module>external/storm-redis</module>
+        <module>external/storm-eventhubs</module>
+        <module>external/flux</module>
+        <module>external/storm-elasticsearch</module>
+        <module>external/storm-solr</module>
+        <module>external/storm-metrics</module>
+        <module>external/sql</module>
+        <module>examples/storm-starter</module>
+    </modules>
+
+
     <profiles>
         <profile>
             <id>sign</id>
@@ -322,6 +336,11 @@
     <dependencyManagement>
         <dependencies>
             <dependency>
+                <groupId>org.hdrhistogram</groupId>
+                <artifactId>HdrHistogram</artifactId>
+                <version>${hdrhistogram.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>org.clojure</groupId>
                 <artifactId>clojure</artifactId>
                 <version>${clojure.version}</version>
@@ -383,6 +402,41 @@
                 </exclusions>
             </dependency>
             <dependency>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+                <version>${servlet.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.slf4j</groupId>
+                <artifactId>slf4j-api</artifactId>
+                <version>${slf4j.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>joda-time</groupId>
+                <artifactId>joda-time</artifactId>
+                <version>${joda-time.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.fasterxml.jackson.core</groupId>
+                <artifactId>jackson-core</artifactId>
+                <version>${jackson.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>com.fasterxml.jackson.dataformat</groupId>
+                <artifactId>jackson-dataformat-smile</artifactId>
+                <version>${jackson.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-fileupload</groupId>
+                <artifactId>commons-fileupload</artifactId>
+                <version>${commons-fileupload.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>commons-codec</groupId>
+                <artifactId>commons-codec</artifactId>
+                <version>${commons-codec.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>org.apache.curator</groupId>
                 <artifactId>curator-framework</artifactId>
                 <version>${curator.version}</version>
@@ -412,6 +466,11 @@
                     </exclusion>
                 </exclusions>
             </dependency>
+            <dependency>
+                <groupId>org.apache.curator</groupId>
+                <artifactId>curator-client</artifactId>
+                <version>${curator.version}</version>
+            </dependency>
 
             <dependency>
                 <groupId>com.googlecode.json-simple</groupId>
@@ -439,6 +498,11 @@
                 <version>${ring.version}</version>
             </dependency>
             <dependency>
+                <groupId>ring</groupId>
+                <artifactId>ring-json</artifactId>
+                <version>${ring-json.version}</version>
+            </dependency>
+            <dependency>
 	      <groupId>org.eclipse.jetty</groupId>
 	      <artifactId>jetty-servlet</artifactId>
 	      <version>${jetty.version}</version>
@@ -449,6 +513,17 @@
 	      <version>${jetty.version}</version>
             </dependency>
             <dependency>
+                <groupId>org.eclipse.jetty</groupId>
+                <artifactId>jetty-server</artifactId>
+                <version>${jetty.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.eclipse.jetty</groupId>
+                <artifactId>jetty-util</artifactId>
+                <version>${jetty.version}</version>
+            </dependency>
+
+            <dependency>
                 <groupId>org.clojure</groupId>
                 <artifactId>tools.logging</artifactId>
                 <version>${clojure.tools.logging.version}</version>
@@ -479,7 +554,7 @@
                 <version>${clojure.tools.cli.version}</version>
             </dependency>
             <dependency>
-                <groupId>com.googlecode.disruptor</groupId>
+                <groupId>com.lmax</groupId>
                 <artifactId>disruptor</artifactId>
                 <version>${disruptor.version}</version>
             </dependency>
@@ -524,6 +599,16 @@
                 <version>${netty.version}</version>
             </dependency>
             <dependency>
+                <groupId>io.dropwizard.metrics</groupId>
+                <artifactId>metrics-core</artifactId>
+                <version>${metrics.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>metrics-clojure</groupId>
+                <artifactId>metrics-clojure</artifactId>
+                <version>${metrics-clojure.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>org.clojure</groupId>
                 <artifactId>clojure-contrib</artifactId>
                 <version>${clojure-contrib.version}</version>
@@ -573,26 +658,25 @@
             <dependency>
                 <groupId>org.apache.thrift</groupId>
                 <artifactId>libthrift</artifactId>
-                <version>0.9.2</version>
+                <version>${thrift.version}</version>
                 <scope>compile</scope>
             </dependency>
             <dependency>
-              <groupId>org.apache.calcite</groupId>
-              <artifactId>calcite-core</artifactId>
-              <version>${calcite.version}</version>
+                <groupId>org.apache.calcite</groupId>
+                <artifactId>calcite-core</artifactId>
+                <version>${calcite.version}</version>
             </dependency>
             <dependency>
-              <groupId>com.fasterxml.jackson.core</groupId>
-              <artifactId>jackson-databind</artifactId>
-              <version>${jackson.version}</version>
+                <groupId>com.fasterxml.jackson.core</groupId>
+                <artifactId>jackson-databind</artifactId>
+                <version>${jackson.version}</version>
             </dependency>
-			<!-- used by examples/storm-starter -->
-		    <dependency>
-		      <groupId>junit</groupId>
-		      <artifactId>junit</artifactId>
-		      <version>4.11</version>
-		      <scope>test</scope>
-		    </dependency>
+            <dependency>
+                <groupId>junit</groupId>
+                <artifactId>junit</artifactId>
+                <version>${junit.version}</version>
+                <scope>test</scope>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 
@@ -686,7 +770,7 @@
                 <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-shade-plugin</artifactId>
-                    <version>2.2</version>
+                    <version>2.4.1</version>
                 </plugin>
                 <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
@@ -724,6 +808,76 @@
                 <artifactId>clojure-maven-plugin</artifactId>
                 <extensions>true</extensions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <version>0.11</version>
+                <executions>
+                    <execution>
+                        <phase>verify</phase>
+                        <goals>
+                            <goal>check</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <excludeSubProjects>false</excludeSubProjects>
+                    <excludes>
+                        <!-- exclude maven artifacts -->
+                        <exclude>**/target/**</exclude>
+                        <!-- exclude intellij projects -->
+                        <exclude>**/*.iml</exclude>
+                        <exclude>**/.idea/**</exclude>
+                        <!-- module specific testing artifacts -->
+                        <exclude>**/metastore_db/**</exclude>
+
+                        <!-- exclude CHANGELOG, VERSION, AND TODO files -->
+                        <exclude>**/CHANGELOG.md</exclude>
+                        <exclude>**/README.md</exclude>
+                        <exclude>**/README.markdown</exclude>
+                        <exclude>**/DEVELOPER.md</exclude>
+                        <exclude>**/BYLAWS.md</exclude>
+                        <exclude>**/STORM-UI-REST-API.md</exclude>
+                        <exclude>SECURITY.md</exclude>
+                        <exclude>VERSION</exclude>
+                        <exclude>TODO</exclude>
+                        <!-- thrift-generated code -->
+                        <exclude>**/src/py/**</exclude>
+
+                        <!-- the following are in the LICENSE file -->
+                        <exclude>**/src/ui/public/js/jquery.dataTables.1.10.4.min.js</exclude>
+                        <exclude>**/src/ui/public/css/jquery.dataTables.1.10.4.min.css</exclude>
+                        <exclude>**/src/ui/public/images/*</exclude>
+                        <exclude>**/src/ui/public/js/bootstrap-3.3.1.min.js</exclude>
+                        <exclude>**/src/ui/public/css/bootstrap-3.3.1.min.css</exclude>
+                        <exclude>**/src/ui/public/js/dataTables.bootstrap.min.js</exclude>
+                        <exclude>**/src/ui/public/css/dataTables.bootstrap.css</exclude>
+                        <exclude>**/src/ui/public/js/jsonFormatter.min.js</exclude>
+                        <exclude>**/src/ui/public/css/jsonFormatter.min.css</exclude>
+                        <exclude>**/src/ui/public/js/jquery-1.11.1.min.js</exclude>
+                        <exclude>**/src/ui/public/js/jquery.cookies.2.2.0.min.js</exclude>
+                        <exclude>**/src/ui/public/js/moment.min.js</exclude>
+                        <exclude>**/src/ui/public/js/jquery.blockUI.min.js</exclude>
+                        <exclude>**/src/ui/public/js/url.min.js</exclude>
+                        <exclude>**/src/ui/public/js/arbor.js</exclude>
+                        <exclude>**/src/ui/public/js/arbor-graphics.js</exclude>
+                        <exclude>**/src/ui/public/js/arbor-tween.js</exclude>
+                        <exclude>**/src/ui/public/js/jquery.mustache.js</exclude>
+                        <exclude>**/src/ui/public/js/typeahead.jquery.min.js</exclude>
+
+                        <!-- generated by shade plugin -->
+                        <exclude>**/dependency-reduced-pom.xml</exclude>
+
+                        <exclude>**/docs/**</exclude>
+                        <exclude>**/.git/**</exclude>
+                        <exclude>**/derby.log</exclude>
+                        <exclude>**/src/dev/**</exclude>
+                        <!-- Storm SQL -->
+                        <exclude>**/src/codegen/config.fmpp</exclude>
+                        <exclude>**/src/codegen/data/Parser.tdd</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 
@@ -784,60 +938,6 @@
                     </tagListOptions>
                 </configuration>
             </plugin>
-            <plugin>
-                <groupId>org.apache.rat</groupId>
-                <artifactId>apache-rat-plugin</artifactId>
-                <version>0.11</version>
-                <configuration>
-                    <excludeSubProjects>false</excludeSubProjects>
-                    <excludes>
-                        <!-- exclude maven artifacts -->
-                        <exclude>**/target/**</exclude>
-                        <!-- exclude intellij projects -->
-                        <exclude>**/*.iml</exclude>
-
-                        <!-- exclude CHANGELOG, VERSION, AND TODO files -->
-                        <exclude>**/CHANGELOG.md</exclude>
-                        <exclude>**/README.md</exclude>
-                        <exclude>**/README.markdown</exclude>
-                        <exclude>**/DEVELOPER.md</exclude>
-                        <exclude>**/BYLAWS.md</exclude>
-                        <exclude>**/STORM-UI-REST-API.md</exclude>
-                        <exclude>SECURITY.md</exclude>
-                        <exclude>VERSION</exclude>
-                        <exclude>TODO</exclude>
-                        <!-- thrift-generated code -->
-                        <exclude>**/src/py/**</exclude>
-
-                        <!-- the following are in the LICENSE file -->
-                        <exclude>**/src/ui/public/js/jquery.dataTables.1.10.4.min.js</exclude>
-                        <exclude>**/src/ui/public/css/jquery.dataTables.1.10.4.min.css</exclude>
-                        <exclude>**/src/ui/public/images/*</exclude>
-                        <exclude>**/src/ui/public/js/bootstrap-3.3.1.min.js</exclude>
-                        <exclude>**/src/ui/public/css/bootstrap-3.3.1.min.css</exclude>
-                        <exclude>**/src/ui/public/js/dataTables.bootstrap.min.js</exclude>
-                        <exclude>**/src/ui/public/css/dataTables.bootstrap.css</exclude>
-                        <exclude>**/src/ui/public/js/jsonFormatter.min.js</exclude>
-                        <exclude>**/src/ui/public/css/jsonFormatter.min.css</exclude>
-                        <exclude>**/src/ui/public/js/jquery-1.11.1.min.js</exclude>
-                        <exclude>**/src/ui/public/js/jquery.cookies.2.2.0.min.js</exclude>
-                        <exclude>**/src/ui/public/js/moment.min.js</exclude>
-                        <exclude>**/src/ui/public/js/jquery.blockUI.min.js</exclude>
-                        <exclude>**/src/ui/public/js/url.min.js</exclude>
-                        <exclude>**/src/ui/public/js/arbor.js</exclude>
-                        <exclude>**/src/ui/public/js/arbor-graphics.js</exclude>
-                        <exclude>**/src/ui/public/js/arbor-tween.js</exclude>
-                        <exclude>**/src/ui/public/js/jquery.mustache.js</exclude>
-
-                        <!-- generated by shade plugin -->
-                        <exclude>**/dependency-reduced-pom.xml</exclude>
-
-                        <exclude>**/docs/**</exclude>
-                        <exclude>**/.git/**</exclude>
-                        <exclude>**/derby.log</exclude>
-                    </excludes>
-                </configuration>
-            </plugin>
         </plugins>
     </reporting>
 </project>
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 3eb4112..083cdca 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -39,6 +39,12 @@
         <dependency>
             <groupId>com.esotericsoftware.kryo</groupId>
             <artifactId>kryo</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.objenesis</groupId>
+                    <artifactId>objenesis</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <!--clojure-->
         <dependency>
@@ -74,7 +80,6 @@
         <dependency>
             <groupId>ring</groupId>
             <artifactId>ring-json</artifactId>
-            <version>0.3.1</version>
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
@@ -121,7 +126,12 @@
             <artifactId>data.codec</artifactId>
             <scope>test</scope>
         </dependency>
-
+        <dependency>
+          <groupId>org.clojure</groupId>
+          <artifactId>java.jmx</artifactId>
+          <version>${java_jmx.version}</version>
+        </dependency>
+        
         <!--java-->
         <dependency>
             <groupId>commons-io</groupId>
@@ -141,7 +151,6 @@
         <dependency>
             <groupId>org.apache.thrift</groupId>
             <artifactId>libthrift</artifactId>
-            <version>0.9.2</version>
             <scope>compile</scope>
             <exclusions>
                 <exclusion>
@@ -154,8 +163,6 @@
                 </exclusion>
             </exclusions>
         </dependency>
-
-
         <dependency>
             <groupId>org.apache.curator</groupId>
             <artifactId>curator-framework</artifactId>
@@ -192,8 +199,6 @@
             <artifactId>json-simple</artifactId>
             <scope>compile</scope>
         </dependency>
-
-
         <dependency>
             <groupId>com.twitter</groupId>
             <artifactId>carbonite</artifactId>
@@ -205,7 +210,7 @@
             <scope>compile</scope>
         </dependency>
         <dependency>
-            <groupId>com.googlecode.disruptor</groupId>
+            <groupId>com.lmax</groupId>
             <artifactId>disruptor</artifactId>
         </dependency>
         <dependency>
@@ -239,15 +244,66 @@
             <artifactId>netty</artifactId>
         </dependency>
         <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>metrics-clojure</groupId>
+            <artifactId>metrics-clojure</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
             <scope>test</scope>
         </dependency>
         <dependency>
-        	<groupId>junit</groupId>
-        	<artifactId>junit</artifactId>
-        	<version>4.11</version>
-        	<scope>test</scope>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>servlet-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>joda-time</groupId>
+            <artifactId>joda-time</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-util</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.dataformat</groupId>
+            <artifactId>jackson-dataformat-smile</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-client</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-fileupload</groupId>
+            <artifactId>commons-fileupload</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
         </dependency>
     </dependencies>
     <build>
@@ -360,23 +416,22 @@
                     <minimizeJar>false</minimizeJar>
                     <artifactSet>
                         <includes>
+                            <include>ns-tracker:ns-tracker</include>
+                            <include>hiccup:hiccup</include>
+                            <include>ring:*</include>
+                            <include>compojure:compojure</include>
+                            <include>clj-time:clj-time</include>
                             <include>org.apache.thrift:*</include>
                             <include>io.netty:netty</include>
                             <include>com.google.guava:guava</include>
                             <include>org.apache.httpcomponents:http*</include>
                             <include>org.apache.zookeeper:zookeeper</include>
                             <include>org.apache.curator:*</include>
-                            <include>org.apache.httpcomponents:http*</include>
-                            <include>org.apache.zookeeper:zookeeper</include>
-                            <include>org.apache.curator:*</include>
                             <include>com.twitter:carbonite</include>
                             <include>com.twitter:chill-java</include>
-                            <include>org.objenesis:objenesis</include>
                             <include>org.tukaani:xz</include>
                             <include>org.yaml:snakeyaml</include>
                             <include>org.jgrapht:jgrapht-core</include>
-                            <include>commons-httpclient:commons-httpclient</include>
-                            <include>org.apache.commons:commons-compress</include>
                             <include>org.apache.commons:commons-exec</include>
                             <include>commons-io:commons-io</include>
                             <include>commons-codec:commons-codec</include>
@@ -385,168 +440,244 @@
                             <include>com.googlecode.json-simple:json-simple</include>
                             <include>org.clojure:math.numeric-tower</include>
                             <include>org.clojure:tools.cli</include>
+                            <include>org.clojure:tools.logging</include>
                             <include>org.clojure:tools.macro</include>
+                            <include>org.clojure:java.jmx</include>
                             <include>joda-time:joda-time</include>
                             <include>org.eclipse.jetty:*</include>
                             <include>com.fasterxml.jackson.core:*</include>
                             <include>com.fasterxml.jackson.dataformat:*</include>
+                            <include>clout:clout</include>
+                            <include>org.clojure:tools.namespace</include>
+                            <include>cheshire:cheshire</include>
+                            <include>org.clojure:core.incubator</include>
+                            <include>io.dropwizard.metrics:*</include>
+                            <include>metrics-clojure:*</include>
                         </includes>
                     </artifactSet>
-
                     <relocations>
                         <relocation>
-                            <pattern>com.fasterxml.jackson.core</pattern>
-                            <shadedPattern>com.fasterxml.jackson.storm.core</shadedPattern>
+                            <pattern>cheshire</pattern>
+                            <shadedPattern>org.apache.storm.shade.cheshire</shadedPattern>
                         </relocation>
                         <relocation>
-                            <pattern>com.fasterxml.jackson.dataformat</pattern>
-                            <shadedPattern>com.fasterxml.jackson.storm.dataformat</shadedPattern>
+                            <pattern>clojure.tools.logging</pattern>
+                            <shadedPattern>org.apache.storm.shade.clojure.tools.logging</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>clojure.core.incubator</pattern>
+                            <shadedPattern>org.apache.storm.shade.clojure.core.incubator</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>clojure.tools.namespace</pattern>
+                            <shadedPattern>org.apache.storm.shade.clojure.tools.namespace</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>clout</pattern>
+                            <shadedPattern>org.apache.storm.shade.clout</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>compojure</pattern>
+                            <shadedPattern>org.apache.storm.shade.compojure</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>ns_tracker</pattern>
+                            <shadedPattern>org.apache.storm.shade.ns_tracker</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>ns-tracker</pattern>
+                            <shadedPattern>org.apache.storm.shade.ns-tracker</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>hiccup</pattern>
+                            <shadedPattern>org.apache.storm.shade.hiccup</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>ring</pattern>
+                            <shadedPattern>org.apache.storm.shade.ring</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>clj_time</pattern>
+                            <shadedPattern>org.apache.storm.shade.clj_time</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>clj-time</pattern>
+                            <shadedPattern>org.apache.storm.shade.clj-time</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>com.fasterxml</pattern>
+                            <shadedPattern>org.apache.storm.shade.com.fasterxml</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>org.apache.thrift</pattern>
+                            <!-- This pattern is inconsistent for backwards compatibility purposes. -->
                             <shadedPattern>org.apache.thrift7</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>org.jboss.netty</pattern>
-                            <shadedPattern>org.apache.storm.netty</shadedPattern>
+                            <shadedPattern>org.apache.storm.shade.org.jboss.netty</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>com.google.common</pattern>
-                            <shadedPattern>org.apache.storm.guava</shadedPattern>
+                            <shadedPattern>org.apache.storm.shade.com.google.common</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>com.google.thirdparty</pattern>
-                            <shadedPattern>org.apache.storm.guava.thirdparty</shadedPattern>
+                            <shadedPattern>org.apache.storm.shade.com.google.thirdparty</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>org.apache.http</pattern>
-                            <shadedPattern>org.apache.storm.http</shadedPattern>
-                        </relocation>
-                        <relocation>
-                            <pattern>org.apache.zookeeper</pattern>
-                            <shadedPattern>org.apache.storm.zookeeper</shadedPattern>
+                            <shadedPattern>org.apache.storm.shade.org.apache.http</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>org.apache.curator</pattern>
-                            <shadedPattern>org.apache.storm.curator</shadedPattern>
-                        </relocation>
-                        <relocation>
-                            <pattern>org.jboss.netty</pattern>
-                            <shadedPattern>org.apache.storm.netty</shadedPattern>
-                        </relocation>
-                        <relocation>
-                            <pattern>com.google.common</pattern>
-                            <shadedPattern>org.apache.storm.guava</shadedPattern>
-                        </relocation>
-                        <relocation>
-                            <pattern>com.google.thirdparty</pattern>
-                            <shadedPattern>org.apache.storm.guava.thirdparty</shadedPattern>
-                        </relocation>
-                        <relocation>
-                            <pattern>org.apache.http</pattern>
-                            <shadedPattern>org.apache.storm.http</shadedPattern>
+                            <shadedPattern>org.apache.storm.shade.org.apache.curator</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>org.apache.zookeeper</pattern>
-                            <shadedPattern>org.apache.storm.zookeeper</shadedPattern>
+                            <shadedPattern>org.apache.storm.shade.org.apache.zookeeper</shadedPattern>
                         </relocation>
                         <relocation>
                             <pattern>org.apache.jute</pattern>
-                            <shadedPattern>org.apache.storm.jute</shadedPattern>
-                        </relocation>
-                        <relocation>
-                            <pattern>org.apache.curator</pattern>
-                            <shadedPattern>org.apache.storm.curator</shadedPattern>
+                            <shadedPattern>org.apache.storm.shade.org.apache.jute</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>carbonite</pattern>
-                          <shadedPattern>org.apache.storm.carbonite</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.carbonite</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>com.twitter.chill</pattern>
-                          <shadedPattern>org.apache.storm.chill</shadedPattern>
-                        </relocation>
-                        <relocation>
-                          <pattern>org.objenesis</pattern>
-                          <shadedPattern>org.apache.storm.objenesis</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.com.twitter.chill</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.tukaani.xz</pattern>
-                          <shadedPattern>org.apache.storm.xz</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.tukaani.xz</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.yaml.snakeyaml</pattern>
-                          <shadedPattern>org.apache.storm.snakeyaml</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.yaml.snakeyaml</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.jgrapht</pattern>
-                          <shadedPattern>org.apache.storm.jgrapht</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.jgrapht</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.fusesource</pattern>
-                          <shadedPattern>org.apache.storm.fusesource</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.fusesource</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>com.metamx.http.client</pattern>
-                          <shadedPattern>org.apache.storm.metamx.http.client</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.com.metamx.http.client</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.apache.commons.io</pattern>
-                          <shadedPattern>org.apache.storm.commons.io</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.apache.commons.io</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.apache.commons.codec</pattern>
-                          <shadedPattern>org.apache.storm.commons.codec</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.apache.commons.codec</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.apache.commons.fileupload</pattern>
-                          <shadedPattern>org.apache.storm.commons.fileupload</shadedPattern>
-                        </relocation>
-                        <relocation>
-                          <pattern>org.apache.commons.compress</pattern>
-                          <shadedPattern>org.apache.storm.commons.compress</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.apache.commons.fileupload</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.apache.commons.exec</pattern>
-                          <shadedPattern>org.apache.storm.commons.exec</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.apache.commons.exec</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.apache.commons.lang</pattern>
-                          <shadedPattern>org.apache.storm.commons.lang</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.apache.commons.lang</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.json.simple</pattern>
-                          <shadedPattern>org.apache.storm.json.simple</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.json.simple</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>clojure.math</pattern>
-                          <shadedPattern>org.apache.storm.clojure.math</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.clojure.math</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>clojure.tools.cli</pattern>
-                          <shadedPattern>org.apache.storm.clojure.tools.cli</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.clojure.tools.cli</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>cljs.tools.cli</pattern>
-                          <shadedPattern>org.apache.storm.cljs.tools.cli</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.cljs.tools.cli</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>clojure.tools.macro</pattern>
-                          <shadedPattern>org.apache.storm.clojure.tools.macro</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.clojure.tools.macro</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.joda.time</pattern>
-                          <shadedPattern>org.apache.storm.joda.time</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.joda.time</shadedPattern>
                         </relocation>
                         <relocation>
                           <pattern>org.eclipse.jetty</pattern>
-                          <shadedPattern>org.apache.storm.jetty</shadedPattern>
+                          <shadedPattern>org.apache.storm.shade.org.eclipse.jetty</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>com.codahale.metrics</pattern>
+                            <shadedPattern>org.apache.storm.shade.com.codahale.metrics</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.core</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.core</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.counters</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.counters</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.gauges</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.gauges</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.histograms</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.histograms</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.meters</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.meters</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.reporters</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.reporters</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.timers</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.timers</shadedPattern>
+                        </relocation>
+                        <relocation>
+                            <pattern>metrics.utils</pattern>
+                            <shadedPattern>org.apache.storm.shade.metrics.utils</shadedPattern>
                         </relocation>
                     </relocations>
                     <transformers>
                         <transformer implementation="org.apache.storm.maven.shade.clojure.ClojureTransformer" />
                     </transformers>
                     <filters>
+                        <!-- Several of these filters remove the .clj files from the shaded dependencies, even though only .clj files are in these jars.
+                             The reason for this is a bit complex, but intentional.  During the build process all of the dependency .clj files are
+                             compiled down into .class files, and included in storm-core.jar.  The regular shade transformer handles these in 
+                             the majority of cases correctly.  However, the Clojure-Transformer does not shade everything correctly all the
+                             time.  Instead of spending a lot of time to get the Clojure-Transformer to parse Clojure correctly we opted to remove
+                             the .clj files from the uber jar. -->
+                        <filter><artifact>metrics-clojure:*</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>org.clojure:core.incubator</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>cheshire:cheshire</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>org.clojure:tools.logging</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>org.clojure:tools.namespace</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>org.clojure:math.numeric-tower</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>org.clojure:tools.macro</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>org.clojure:tools.cli</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>ns-tracker:ns-tracker</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>clout:clout</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>hiccup:hiccup</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>clj-time:clj-time</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>ring:*</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
+                        <filter><artifact>compojure:compojure</artifact><excludes><exclude>**/*.clj</exclude></excludes></filter>
                         <filter>
                             <artifact>org.apache.thrift:*</artifact>
                             <excludes>
@@ -562,48 +693,12 @@
                             </excludes>
                         </filter>
                         <filter>
-                            <artifact>commons-httpclient:commons-httpclient</artifact>
-                            <excludes>
-                                <exclude>META-INF/LICENSE.txt</exclude>
-                                <exclude>META-INF/NOTICE.txt</exclude>
-                                <exclude>META-INF/README.txt</exclude>
-                            </excludes>
-                        </filter>
-                        <filter>
                             <artifact>org.apache.zookeeper:zookeeper</artifact>
                             <excludes>
                                 <exclude>LICENSE.txt</exclude>
                             </excludes>
                         </filter>
                         <filter>
-                            <artifact>commons-httpclient:commons-httpclient</artifact>
-                            <excludes>
-                                <exclude>META-INF/LICENSE.txt</exclude>
-                                <exclude>META-INF/NOTICE.txt</exclude>
-                                <exclude>META-INF/README.txt</exclude>
-                            </excludes>
-                        </filter>
-                        <filter>
-                            <artifact>org.apache.zookeeper:zookeeper</artifact>
-                            <excludes>
-                                <exclude>LICENSE.txt</exclude>
-                            </excludes>
-                        </filter>
-                        <filter>
-                            <artifact>org.objenesis:objenesis</artifact>
-                            <excludes>
-                                <exclude>META-INF/LICENSE.txt</exclude>
-                                <exclude>META-INF/NOTICE.txt</exclude>
-                            </excludes>
-                        </filter>
-                        <filter>
-                            <artifact>org.apache.commons:commons-compress</artifact>
-                            <excludes>
-                                <exclude>META-INF/LICENSE.txt</exclude>
-                                <exclude>META-INF/NOTICE.txt</exclude>
-                            </excludes>
-                        </filter>
-                        <filter>
                             <artifact>org.apache.commons:commons-exec</artifact>
                             <excludes>
                                 <exclude>META-INF/LICENSE.txt</exclude>
@@ -740,6 +835,18 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.6</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 
diff --git a/storm-core/src/clj/backtype/storm/cluster.clj b/storm-core/src/clj/backtype/storm/cluster.clj
index 99fb49e..35aa8c8 100644
--- a/storm-core/src/clj/backtype/storm/cluster.clj
+++ b/storm-core/src/clj/backtype/storm/cluster.clj
@@ -16,12 +16,13 @@
 
 (ns backtype.storm.cluster
   (:import [org.apache.zookeeper.data Stat ACL Id]
-           [backtype.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials NimbusSummary]
+           [backtype.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials NimbusSummary
+            LogConfig ProfileAction ProfileRequest NodeInfo]
            [java.io Serializable])
   (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
-  (:import [org.apache.curator.framework.state ConnectionStateListener ConnectionState])
   (:import [org.apache.curator.framework CuratorFramework])
   (:import [backtype.storm.utils Utils])
+  (:import [backtype.storm.cluster ClusterState ClusterStateContext ClusterStateListener ConnectionState])
   (:import [java.security MessageDigest])
   (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
   (:import [backtype.storm.nimbus NimbusInfo])
@@ -29,128 +30,22 @@
   (:require [backtype.storm [zookeeper :as zk]])
   (:require [backtype.storm.daemon [common :as common]]))
 
-(defprotocol ClusterState
-  (set-ephemeral-node [this path data acls])
-  (delete-node [this path])
-  (create-sequential [this path data acls])
-  ;; if node does not exist, create persistent with this data
-  (set-data [this path data acls])
-  (get-data [this path watch?])
-  (get-version [this path watch?])
-  (get-data-with-version [this path watch?])
-  (get-children [this path watch?])
-  (mkdirs [this path acls])
-  (exists-node? [this path watch?])
-  (close [this])
-  (register [this callback])
-  (unregister [this id])
-  (add-listener [this listener])
-  (sync-path [this path]))
-
 (defn mk-topo-only-acls
   [topo-conf]
   (let [payload (.get topo-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)]
     (when (Utils/isZkAuthenticationConfiguredTopology topo-conf)
       [(first ZooDefs$Ids/CREATOR_ALL_ACL)
        (ACL. ZooDefs$Perms/READ (Id. "digest" (DigestAuthenticationProvider/generateDigest payload)))])))
-
+ 
 (defnk mk-distributed-cluster-state
-  [conf :auth-conf nil :acls nil]
-  (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf auth-conf)]
-    (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT) acls)
-    (.close zk))
-  (let [callbacks (atom {})
-        active (atom true)
-        zk (zk/mk-client conf
-                         (conf STORM-ZOOKEEPER-SERVERS)
-                         (conf STORM-ZOOKEEPER-PORT)
-                         :auth-conf auth-conf
-                         :root (conf STORM-ZOOKEEPER-ROOT)
-                         :watcher (fn [state type path]
-                                    (when @active
-                                      (when-not (= :connected state)
-                                        (log-warn "Received event " state ":" type ":" path " with disconnected Zookeeper."))
-                                      (when-not (= :none type)
-                                        (doseq [callback (vals @callbacks)]
-                                          (callback type path))))))]
-    (reify
-     ClusterState
-
-     (register
-       [this callback]
-       (let [id (uuid)]
-         (swap! callbacks assoc id callback)
-         id))
-
-     (unregister
-       [this id]
-       (swap! callbacks dissoc id))
-
-     (set-ephemeral-node
-       [this path data acls]
-       (zk/mkdirs zk (parent-path path) acls)
-       (if (zk/exists zk path false)
-         (try-cause
-           (zk/set-data zk path data) ; should verify that it's ephemeral
-           (catch KeeperException$NoNodeException e
-             (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
-             (zk/create-node zk path data :ephemeral acls)))
-         (zk/create-node zk path data :ephemeral acls)))
-
-     (create-sequential
-       [this path data acls]
-       (zk/create-node zk path data :sequential acls))
-
-     (set-data
-       [this path data acls]
-       ;; note: this does not turn off any existing watches
-       (if (zk/exists zk path false)
-         (zk/set-data zk path data)
-         (do
-           (zk/mkdirs zk (parent-path path) acls)
-           (zk/create-node zk path data :persistent acls))))
-
-     (delete-node
-       [this path]
-       (zk/delete-node zk path))
-
-     (get-data
-       [this path watch?]
-       (zk/get-data zk path watch?))
-
-     (get-data-with-version
-       [this path watch?]
-       (zk/get-data-with-version zk path watch?))
-
-     (get-version 
-       [this path watch?]
-       (zk/get-version zk path watch?))
-
-     (get-children
-       [this path watch?]
-       (zk/get-children zk path watch?))
-
-     (mkdirs
-       [this path acls]
-       (zk/mkdirs zk path acls))
-
-     (exists-node?
-       [this path watch?]
-       (zk/exists-node? zk path watch?))
-
-     (close
-       [this]
-       (reset! active false)
-       (.close zk))
-
-      (add-listener
-        [this listener]
-        (zk/add-listener zk listener))
-
-      (sync-path
-        [this path]
-        (zk/sync-path zk path))
-      )))
+  [conf :auth-conf nil :acls nil :context (ClusterStateContext.)]
+  (let [clazz (Class/forName (or (conf STORM-CLUSTER-STATE-STORE)
+                                 "backtype.storm.cluster_state.zookeeper_state_factory"))
+        state-instance (.newInstance clazz)]
+    (log-debug "Creating cluster state: " (.toString clazz))
+    (or (.mkState state-instance conf auth-conf acls context)
+        nil)))
+  
 
 (defprotocol StormClusterState
   (assignments [this callback])
@@ -170,6 +65,10 @@
   (active-storms [this])
   (storm-base [this storm-id callback])
   (get-worker-heartbeat [this storm-id node port])
+  (get-worker-profile-requests [this storm-id nodeinfo thrift?])
+  (get-topology-profile-requests [this storm-id thrift?])
+  (set-worker-profile-request [this storm-id profile-request])
+  (delete-topology-profile-requests [this storm-id profile-request])
   (executor-beats [this storm-id executor->node+port])
   (supervisors [this callback])
   (supervisor-info [this supervisor-id]) ;; returns nil if doesn't exist
@@ -178,6 +77,8 @@
   (teardown-topology-errors! [this storm-id])
   (heartbeat-storms [this])
   (error-topologies [this])
+  (set-topology-log-config! [this storm-id log-config])
+  (topology-log-config [this storm-id cb])
   (worker-heartbeat! [this storm-id node port info])
   (remove-worker-heartbeat! [this storm-id node port])
   (supervisor-heartbeat! [this supervisor-id info])
@@ -209,7 +110,8 @@
 (def CODE-DISTRIBUTOR-ROOT "code-distributor")
 (def NIMBUSES-ROOT "nimbuses")
 (def CREDENTIALS-ROOT "credentials")
-
+(def LOGCONFIG-ROOT "logconfigs")
+(def PROFILERCONFIG-ROOT "profilerconfigs")
 
 (def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
 (def STORMS-SUBTREE (str "/" STORMS-ROOT))
@@ -220,6 +122,8 @@
 (def CODE-DISTRIBUTOR-SUBTREE (str "/" CODE-DISTRIBUTOR-ROOT))
 (def NIMBUSES-SUBTREE (str "/" NIMBUSES-ROOT))
 (def CREDENTIALS-SUBTREE (str "/" CREDENTIALS-ROOT))
+(def LOGCONFIG-SUBTREE (str "/" LOGCONFIG-ROOT))
+(def PROFILERCONFIG-SUBTREE (str "/" PROFILERCONFIG-ROOT))
 
 (defn supervisor-path
   [id]
@@ -279,6 +183,16 @@
   [storm-id]
   (str CREDENTIALS-SUBTREE "/" storm-id))
 
+(defn log-config-path
+  [storm-id]
+  (str LOGCONFIG-SUBTREE "/" storm-id))
+
+(defn profiler-config-path
+  ([storm-id]
+   (str PROFILERCONFIG-SUBTREE "/" storm-id))
+  ([storm-id host port request-type]
+   (str (profiler-config-path storm-id) "/" host "_" port "_" request-type)))
+
 (defn- issue-callback!
   [cb-atom]
   (let [cb @cb-atom]
@@ -319,10 +233,10 @@
 
 ;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
 (defnk mk-storm-cluster-state
-  [cluster-state-spec :acls nil]
-  (let [[solo? cluster-state] (if (satisfies? ClusterState cluster-state-spec)
+  [cluster-state-spec :acls nil :context (ClusterStateContext.)]
+  (let [[solo? cluster-state] (if (instance? ClusterState cluster-state-spec)
                                 [false cluster-state-spec]
-                                [true (mk-distributed-cluster-state cluster-state-spec :auth-conf cluster-state-spec :acls acls)])
+                                [true (mk-distributed-cluster-state cluster-state-spec :auth-conf cluster-state-spec :acls acls :context context)])
         assignment-info-callback (atom {})
         assignment-info-with-version-callback (atom {})
         assignment-version-callback (atom {})
@@ -332,7 +246,8 @@
         storm-base-callback (atom {})
         code-distributor-callback (atom nil)
         credentials-callback (atom {})
-        state-id (register
+        log-config-callback (atom {})
+        state-id (.register
                   cluster-state
                   (fn [type path]
                     (let [[subtree & args] (tokenize-path path)]
@@ -347,11 +262,13 @@
                          CODE-DISTRIBUTOR-ROOT (issue-callback! code-distributor-callback)
                          STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
                          CREDENTIALS-ROOT (issue-map-callback! credentials-callback (first args))
+                         LOGCONFIG-ROOT (issue-map-callback! log-config-callback (first args))
                          BACKPRESSURE-ROOT (issue-map-callback! backpressure-callback (first args))
                          ;; this should never happen
                          (exit-process! 30 "Unknown callback for subtree " subtree args)))))]
-    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE CODE-DISTRIBUTOR-SUBTREE NIMBUSES-SUBTREE]]
-      (mkdirs cluster-state p acls))
+    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE CODE-DISTRIBUTOR-SUBTREE NIMBUSES-SUBTREE
+               LOGCONFIG-SUBTREE]]
+      (.mkdirs cluster-state p acls))
     (reify
       StormClusterState
 
@@ -359,20 +276,20 @@
         [this callback]
         (when callback
           (reset! assignments-callback callback))
-        (get-children cluster-state ASSIGNMENTS-SUBTREE (not-nil? callback)))
+        (.get_children cluster-state ASSIGNMENTS-SUBTREE (not-nil? callback)))
 
       (assignment-info
         [this storm-id callback]
         (when callback
           (swap! assignment-info-callback assoc storm-id callback))
-        (clojurify-assignment (maybe-deserialize (get-data cluster-state (assignment-path storm-id) (not-nil? callback)) Assignment)))
+        (clojurify-assignment (maybe-deserialize (.get_data cluster-state (assignment-path storm-id) (not-nil? callback)) Assignment)))
 
       (assignment-info-with-version 
         [this storm-id callback]
         (when callback
           (swap! assignment-info-with-version-callback assoc storm-id callback))
         (let [{data :data version :version} 
-              (get-data-with-version cluster-state (assignment-path storm-id) (not-nil? callback))]
+              (.get_data_with_version cluster-state (assignment-path storm-id) (not-nil? callback))]
         {:data (clojurify-assignment (maybe-deserialize data Assignment))
          :version version}))
 
@@ -380,59 +297,59 @@
         [this storm-id callback]
         (when callback
           (swap! assignment-version-callback assoc storm-id callback))
-        (get-version cluster-state (assignment-path storm-id) (not-nil? callback)))
+        (.get_version cluster-state (assignment-path storm-id) (not-nil? callback)))
 
       (code-distributor
         [this callback]
         (when callback
           (reset! code-distributor-callback callback))
         (do
-          (sync-path cluster-state CODE-DISTRIBUTOR-SUBTREE)
-          (get-children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback))))
+          (.sync_path cluster-state CODE-DISTRIBUTOR-SUBTREE)
+          (.get_children cluster-state CODE-DISTRIBUTOR-SUBTREE (not-nil? callback))))
 
       (nimbuses
         [this]
-        (map #(maybe-deserialize (get-data cluster-state (nimbus-path %1) false) NimbusSummary)
-          (get-children cluster-state NIMBUSES-SUBTREE false)))
+        (map #(maybe-deserialize (.get_data cluster-state (nimbus-path %1) false) NimbusSummary)
+          (.get_children cluster-state NIMBUSES-SUBTREE false)))
 
       (add-nimbus-host!
         [this nimbus-id nimbus-summary]
         ;explicit delete for ephmeral node to ensure this session creates the entry.
-        (delete-node cluster-state (nimbus-path nimbus-id))
+        (.delete_node cluster-state (nimbus-path nimbus-id))
 
-        (add-listener cluster-state (reify ConnectionStateListener
-                        (^void stateChanged[this ^CuratorFramework client ^ConnectionState newState]
+        (.add_listener cluster-state (reify ClusterStateListener
+                        (^void stateChanged[this ^ConnectionState newState]
                           (log-message "Connection state listener invoked, zookeeper connection state has changed to " newState)
                           (if (.equals newState ConnectionState/RECONNECTED)
                             (do
                               (log-message "Connection state has changed to reconnected so setting nimbuses entry one more time")
-                              (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))))))
-
-        (set-ephemeral-node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
+                              (.set_ephemeral_node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))))))
+        
+        (.set_ephemeral_node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
 
       (code-distributor-info
         [this storm-id]
         (map (fn [nimbus-info] (NimbusInfo/parse nimbus-info))
           (let [path (code-distributor-path storm-id)]
             (do
-              (sync-path cluster-state path)
-              (get-children cluster-state path false)))))
+              (.sync_path cluster-state path)
+              (.get_children cluster-state path false)))))
 
       (active-storms
         [this]
-        (get-children cluster-state STORMS-SUBTREE false))
+        (.get_children cluster-state STORMS-SUBTREE false))
 
       (heartbeat-storms
         [this]
-        (get-children cluster-state WORKERBEATS-SUBTREE false))
+        (.get_worker_hb_children cluster-state WORKERBEATS-SUBTREE false))
 
       (error-topologies
         [this]
-        (get-children cluster-state ERRORS-SUBTREE false))
+        (.get_children cluster-state ERRORS-SUBTREE false))
 
       (get-worker-heartbeat
         [this storm-id node port]
-        (let [worker-hb (get-data cluster-state (workerbeat-path storm-id node port) false)]
+        (let [worker-hb (.get_worker_hb cluster-state (workerbeat-path storm-id node port) false)]
           (if worker-hb
             (-> worker-hb
               (maybe-deserialize ClusterWorkerHeartbeat)
@@ -455,30 +372,82 @@
         [this callback]
         (when callback
           (reset! supervisors-callback callback))
-        (get-children cluster-state SUPERVISORS-SUBTREE (not-nil? callback)))
+        (.get_children cluster-state SUPERVISORS-SUBTREE (not-nil? callback)))
 
       (supervisor-info
         [this supervisor-id]
-        (clojurify-supervisor-info (maybe-deserialize (get-data cluster-state (supervisor-path supervisor-id) false) SupervisorInfo)))
+        (clojurify-supervisor-info (maybe-deserialize (.get_data cluster-state (supervisor-path supervisor-id) false) SupervisorInfo)))
 
+      (topology-log-config
+        [this storm-id cb]
+        (when cb
+          (swap! log-config-callback assoc storm-id cb))
+        (maybe-deserialize (.get_data cluster-state (log-config-path storm-id) (not-nil? cb)) LogConfig))
+
+      (set-topology-log-config!
+        [this storm-id log-config]
+        (.set_data cluster-state (log-config-path storm-id) (Utils/serialize log-config) acls))
+
+      (set-worker-profile-request
+        [this storm-id profile-request]
+        (let [request-type (.get_action profile-request)
+              host (.get_node (.get_nodeInfo profile-request))
+              port (first (.get_port (.get_nodeInfo profile-request)))]
+          (.set_data cluster-state
+                     (profiler-config-path storm-id host port request-type)
+                     (Utils/serialize profile-request)
+                     acls)))
+
+      (get-topology-profile-requests
+        [this storm-id thrift?]
+        (let [path (profiler-config-path storm-id)
+              requests (if (.node_exists cluster-state path false)
+                         (dofor [c (.get_children cluster-state path false)]
+                                (let [raw (.get_data cluster-state (str path "/" c) false)
+                                      request (maybe-deserialize raw ProfileRequest)]
+                                      (if thrift?
+                                        request
+                                        (clojurify-profile-request request)))))]
+          requests))
+
+      (delete-topology-profile-requests
+        [this storm-id profile-request]
+        (let [profile-request-inst (thriftify-profile-request profile-request)
+              action (:action profile-request)
+              host (:host profile-request)
+              port (:port profile-request)]
+          (.delete_node cluster-state
+           (profiler-config-path storm-id host port action))))
+          
+      (get-worker-profile-requests
+        [this storm-id node-info thrift?]
+        (let [host (:host node-info)
+              port (:port node-info)
+              profile-requests (get-topology-profile-requests this storm-id thrift?)]
+          (if thrift?
+            (filter #(and (= host (.get_node (.get_nodeInfo %))) (= port (first (.get_port (.get_nodeInfo  %)))))
+                    profile-requests)
+            (filter #(and (= host (:host %)) (= port (:port %)))
+                    profile-requests))))
+      
       (worker-heartbeat!
         [this storm-id node port info]
         (let [thrift-worker-hb (thriftify-zk-worker-hb info)]
           (if thrift-worker-hb
-            (set-data cluster-state (workerbeat-path storm-id node port) (Utils/serialize thrift-worker-hb) acls))))
+            (.set_worker_hb cluster-state (workerbeat-path storm-id node port) (Utils/serialize thrift-worker-hb) acls))))
 
       (remove-worker-heartbeat!
         [this storm-id node port]
-        (delete-node cluster-state (workerbeat-path storm-id node port)))
+        (.delete_worker_hb cluster-state (workerbeat-path storm-id node port)))
 
       (setup-heartbeats!
         [this storm-id]
-        (mkdirs cluster-state (workerbeat-storm-root storm-id) acls))
+        (.mkdirs cluster-state (workerbeat-storm-root storm-id) acls))
 
       (teardown-heartbeats!
         [this storm-id]
         (try-cause
-          (delete-node cluster-state (workerbeat-storm-root storm-id))
+          (.delete_worker_hb cluster-state (workerbeat-storm-root storm-id))
           (catch KeeperException e
             (log-warn-error e "Could not teardown heartbeats for " storm-id))))
 
@@ -487,12 +456,12 @@
         "if znode exists and to be not on?, delete; if exists and on?, do nothing;
         if not exists and to be on?, create; if not exists and not on?, do nothing"
         (let [path (backpressure-path storm-id node port)
-              existed (exists-node? cluster-state path false)]
+              existed (.node_exists cluster-state path false)]
           (if existed
             (if (not on?)
-              (delete-node cluster-state path))   ;; delete the znode since the worker is not congested
+              (.delete_node cluster-state path))   ;; delete the znode since the worker is not congested
             (if on?
-              (set-ephemeral-node cluster-state path nil acls))))) ;; create the znode since worker is congested
+              (.set_ephemeral_node cluster-state path nil acls))))) ;; create the znode since worker is congested
     
       (topology-backpressure
         [this storm-id callback]
@@ -500,33 +469,33 @@
         (when callback
           (swap! backpressure-callback assoc storm-id callback))
         (let [path (backpressure-storm-root storm-id)
-              children (get-children cluster-state path (not-nil? callback))]
+              children (.get_children cluster-state path (not-nil? callback))]
               (> (count children) 0)))
       
       (setup-backpressure!
         [this storm-id]
-        (mkdirs cluster-state (backpressure-storm-root storm-id) acls))
+        (.mkdirs cluster-state (backpressure-storm-root storm-id) acls))
 
       (remove-worker-backpressure!
         [this storm-id node port]
-        (delete-node cluster-state (backpressure-path storm-id node port)))
+        (.delete_node cluster-state (backpressure-path storm-id node port)))
 
       (teardown-topology-errors!
         [this storm-id]
         (try-cause
-          (delete-node cluster-state (error-storm-root storm-id))
+          (.delete_node cluster-state (error-storm-root storm-id))
           (catch KeeperException e
             (log-warn-error e "Could not teardown errors for " storm-id))))
 
       (supervisor-heartbeat!
         [this supervisor-id info]
         (let [thrift-supervisor-info (thriftify-supervisor-info info)]
-          (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize thrift-supervisor-info) acls)))
+          (.set_ephemeral_node cluster-state (supervisor-path supervisor-id) (Utils/serialize thrift-supervisor-info) acls)))
 
       (activate-storm!
         [this storm-id storm-base]
         (let [thrift-storm-base (thriftify-storm-base storm-base)]
-          (set-data cluster-state (storm-path storm-id) (Utils/serialize thrift-storm-base) acls)))
+          (.set_data cluster-state (storm-path storm-id) (Utils/serialize thrift-storm-base) acls)))
 
       (update-storm!
         [this storm-id new-elems]
@@ -535,7 +504,7 @@
               component->debug (:component->debug base)
               new-elems (update new-elems :component->executors (partial merge executors))
               new-elems (update new-elems :component->debug (partial merge-with merge component->debug))]
-          (set-data cluster-state (storm-path storm-id)
+          (.set_data cluster-state (storm-path storm-id)
                     (-> base
                         (merge new-elems)
                         thriftify-storm-base
@@ -546,30 +515,32 @@
         [this storm-id callback]
         (when callback
           (swap! storm-base-callback assoc storm-id callback))
-        (clojurify-storm-base (maybe-deserialize (get-data cluster-state (storm-path storm-id) (not-nil? callback)) StormBase)))
+        (clojurify-storm-base (maybe-deserialize (.get_data cluster-state (storm-path storm-id) (not-nil? callback)) StormBase)))
 
       (remove-storm-base!
         [this storm-id]
-        (delete-node cluster-state (storm-path storm-id)))
+        (.delete_node cluster-state (storm-path storm-id)))
 
       (set-assignment!
         [this storm-id info]
         (let [thrift-assignment (thriftify-assignment info)]
-          (set-data cluster-state (assignment-path storm-id) (Utils/serialize thrift-assignment) acls)))
+          (.set_data cluster-state (assignment-path storm-id) (Utils/serialize thrift-assignment) acls)))
 
       (setup-code-distributor!
         [this storm-id nimbusInfo]
         (let [path (str (code-distributor-path storm-id) "/" (.toHostPortString nimbusInfo))]
-        (mkdirs cluster-state (code-distributor-path storm-id) acls)
+        (.mkdirs cluster-state (code-distributor-path storm-id) acls)
         ;we delete the node first to ensure the node gets created as part of this session only.
-        (delete-node cluster-state path)
-        (set-ephemeral-node cluster-state path nil acls)))
+        (.delete_node cluster-state path)
+        (.set_ephemeral_node cluster-state path nil acls)))
 
       (remove-storm!
         [this storm-id]
-        (delete-node cluster-state (assignment-path storm-id))
-        (delete-node cluster-state (code-distributor-path storm-id))
-        (delete-node cluster-state (credentials-path storm-id))
+        (.delete_node cluster-state (assignment-path storm-id))
+        (.delete_node cluster-state (code-distributor-path storm-id))
+        (.delete_node cluster-state (credentials-path storm-id))
+        (.delete_node cluster-state (log-config-path storm-id))
+        (.delete_node cluster-state (profiler-config-path storm-id))
         (remove-storm-base! this storm-id))
 
       (set-credentials!
@@ -577,37 +548,37 @@
          (let [topo-acls (mk-topo-only-acls topo-conf)
                path (credentials-path storm-id)
                thriftified-creds (thriftify-credentials creds)]
-           (set-data cluster-state path (Utils/serialize thriftified-creds) topo-acls)))
+           (.set_data cluster-state path (Utils/serialize thriftified-creds) topo-acls)))
 
       (credentials
         [this storm-id callback]
         (when callback
           (swap! credentials-callback assoc storm-id callback))
-        (clojurify-crdentials (maybe-deserialize (get-data cluster-state (credentials-path storm-id) (not-nil? callback)) Credentials)))
+        (clojurify-crdentials (maybe-deserialize (.get_data cluster-state (credentials-path storm-id) (not-nil? callback)) Credentials)))
 
       (report-error
          [this storm-id component-id node port error]
          (let [path (error-path storm-id component-id)
                last-error-path (last-error-path storm-id component-id)
                data (thriftify-error {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port})
-               _ (mkdirs cluster-state path acls)
+               _ (.mkdirs cluster-state path acls)
                ser-data (Utils/serialize data)
-               _ (mkdirs cluster-state path acls)
-               _ (create-sequential cluster-state (str path "/e") ser-data acls)
-               _ (set-data cluster-state last-error-path ser-data acls)
-               to-kill (->> (get-children cluster-state path false)
+               _ (.mkdirs cluster-state path acls)
+               _ (.create_sequential cluster-state (str path "/e") ser-data acls)
+               _ (.set_data cluster-state last-error-path ser-data acls)
+               to-kill (->> (.get_children cluster-state path false)
                             (sort-by parse-error-path)
                             reverse
                             (drop 10))]
            (doseq [k to-kill]
-             (delete-node cluster-state (str path "/" k)))))
+             (.delete_node cluster-state (str path "/" k)))))
 
       (errors
          [this storm-id component-id]
          (let [path (error-path storm-id component-id)
-               errors (if (exists-node? cluster-state path false)
-                        (dofor [c (get-children cluster-state path false)]
-                          (if-let [data (-> (get-data cluster-state
+               errors (if (.node_exists cluster-state path false)
+                        (dofor [c (.get_children cluster-state path false)]
+                          (if-let [data (-> (.get_data cluster-state
                                                       (str path "/" c)
                                                       false)
                                           (maybe-deserialize ErrorInfo)
@@ -620,17 +591,17 @@
       (last-error
         [this storm-id component-id]
         (let [path (last-error-path storm-id component-id)]
-          (if (exists-node? cluster-state path false)
-            (if-let [data (-> (get-data cluster-state path false)
+          (if (.node_exists cluster-state path false)
+            (if-let [data (-> (.get_data cluster-state path false)
                               (maybe-deserialize ErrorInfo)
                               clojurify-error)]
               (map->TaskError data)))))
       
       (disconnect
          [this]
-        (unregister cluster-state state-id)
+        (.unregister cluster-state state-id)
         (when solo?
-          (close cluster-state))))))
+          (.close cluster-state))))))
 
 ;; daemons have a single thread that will respond to events
 ;; start with initialize event
diff --git a/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj b/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj
new file mode 100644
index 0000000..ff942db
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/cluster_state/zookeeper_state_factory.clj
@@ -0,0 +1,157 @@
+;; 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.cluster-state.zookeeper-state-factory
+  (:import [org.apache.curator.framework.state ConnectionStateListener])
+  (:import [org.apache.zookeeper KeeperException$NoNodeException]
+           [backtype.storm.cluster ClusterState DaemonType])
+  (:use [backtype.storm cluster config log util])
+  (:require [backtype.storm [zookeeper :as zk]])
+  (:gen-class
+   :implements [backtype.storm.cluster.ClusterStateFactory]))
+
+(defn -mkState [this conf auth-conf acls context]
+  (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf auth-conf)]
+    (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT) acls)
+    (.close zk))
+  (let [callbacks (atom {})
+        active (atom true)
+        zk-writer (zk/mk-client conf
+                         (conf STORM-ZOOKEEPER-SERVERS)
+                         (conf STORM-ZOOKEEPER-PORT)
+                         :auth-conf auth-conf
+                         :root (conf STORM-ZOOKEEPER-ROOT)
+                         :watcher (fn [state type path]
+                                    (when @active
+                                      (when-not (= :connected state)
+                                        (log-warn "Received event " state ":" type ":" path " with disconnected Writer Zookeeper."))
+                                      (when-not (= :none type)
+                                        (doseq [callback (vals @callbacks)]
+                                          (callback type path))))))
+        is-nimbus? (= (.getDaemonType context) DaemonType/NIMBUS)
+        zk-reader (if is-nimbus?
+                    (zk/mk-client conf
+                         (conf STORM-ZOOKEEPER-SERVERS)
+                         (conf STORM-ZOOKEEPER-PORT)
+                         :auth-conf auth-conf
+                         :root (conf STORM-ZOOKEEPER-ROOT)
+                         :watcher (fn [state type path]
+                                    (when @active
+                                      (when-not (= :connected state)
+                                        (log-warn "Received event " state ":" type ":" path " with disconnected Reader Zookeeper."))
+                                      (when-not (= :none type)
+                                        (doseq [callback (vals @callbacks)]
+                                          (callback type path))))))
+                    zk-writer)]
+    (reify
+     ClusterState
+
+     (register
+       [this callback]
+       (let [id (uuid)]
+         (swap! callbacks assoc id callback)
+         id))
+
+     (unregister
+       [this id]
+       (swap! callbacks dissoc id))
+
+     (set-ephemeral-node
+       [this path data acls]
+       (zk/mkdirs zk-writer (parent-path path) acls)
+       (if (zk/exists zk-writer path false)
+         (try-cause
+           (zk/set-data zk-writer path data) ; should verify that it's ephemeral
+           (catch KeeperException$NoNodeException e
+             (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
+             (zk/create-node zk-writer path data :ephemeral acls)))
+         (zk/create-node zk-writer path data :ephemeral acls)))
+
+     (create-sequential
+       [this path data acls]
+       (zk/create-node zk-writer path data :sequential acls))
+
+     (set-data
+       [this path data acls]
+       ;; note: this does not turn off any existing watches
+       (if (zk/exists zk-writer path false)
+         (zk/set-data zk-writer path data)
+         (do
+           (zk/mkdirs zk-writer (parent-path path) acls)
+           (zk/create-node zk-writer path data :persistent acls))))
+
+     (set-worker-hb
+       [this path data acls]
+       (.set_data this path data acls))
+
+     (delete-node
+       [this path]
+       (zk/delete-node zk-writer path))
+
+     (delete-worker-hb
+       [this path]
+       (.delete_node this path))
+
+     (get-data
+       [this path watch?]
+       (zk/get-data zk-reader path watch?))
+
+     (get-data-with-version
+       [this path watch?]
+       (zk/get-data-with-version zk-reader path watch?))
+
+     (get-version
+       [this path watch?]
+       (zk/get-version zk-reader path watch?))
+
+     (get-worker-hb
+       [this path watch?]
+       (.get_data this path watch?))
+
+     (get-children
+       [this path watch?]
+       (zk/get-children zk-reader path watch?))
+
+     (get-worker-hb-children
+       [this path watch?]
+       (.get_children this path watch?))
+
+     (mkdirs
+       [this path acls]
+       (zk/mkdirs zk-writer path acls))
+
+     (node-exists
+       [this path watch?]
+       (zk/exists-node? zk-reader path watch?))
+
+     (add-listener
+       [this listener]
+       (let [curator-listener (reify ConnectionStateListener
+                                (stateChanged
+                                  [this client newState]
+                                  (.stateChanged listener client newState)))]
+         (zk/add-listener zk-reader curator-listener)))
+
+     (sync-path
+       [this path]
+       (zk/sync-path zk-writer path))
+
+     (close
+       [this]
+       (reset! active false)
+       (.close zk-writer)
+       (if is-nimbus?
+         (.close zk-reader))))))
diff --git a/storm-core/src/clj/backtype/storm/command/healthcheck.clj b/storm-core/src/clj/backtype/storm/command/healthcheck.clj
new file mode 100644
index 0000000..14af223
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/command/healthcheck.clj
@@ -0,0 +1,88 @@
+;; 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.command.healthcheck
+  (:require [backtype.storm
+             [config :refer :all]
+             [log :refer :all]]
+            [clojure.java [io :as io]]
+            [clojure [string :refer [split]]])
+  (:gen-class))
+
+(defn interrupter
+  "Interrupt a given thread after ms milliseconds."
+  [thread ms]
+  (let [interrupter (Thread.
+                     (fn []
+                       (try
+                         (Thread/sleep ms)
+                         (.interrupt thread)
+                         (catch InterruptedException e))))]
+    (.start interrupter)
+    interrupter))
+
+(defn check-output [lines]
+  (if (some #(.startsWith % "ERROR") lines)
+    :failed
+    :success))
+
+(defn process-script [conf script]
+  (let [script-proc (. (Runtime/getRuntime) (exec script))
+        curthread (Thread/currentThread)
+        interrupter-thread (interrupter curthread
+                                        (conf STORM-HEALTH-CHECK-TIMEOUT-MS))]
+    (try
+      (.waitFor script-proc)
+      (.interrupt interrupter-thread)
+      (if (not (= (.exitValue script-proc) 0))
+        :failed_with_exit_code
+        (check-output (split
+                       (slurp (.getInputStream script-proc))
+                       #"\n+")))
+      (catch InterruptedException e
+        (println "Script" script "timed out.")
+        :timeout)
+      (catch Exception e
+        (println "Script failed with exception: " e)
+        :failed_with_exception)
+      (finally (.interrupt interrupter-thread)))))
+
+(defn health-check [conf]
+  (let [health-dir (absolute-healthcheck-dir conf)
+        health-files (file-seq (io/file health-dir))
+        health-scripts (filter #(and (.canExecute %)
+                                     (not (.isDirectory %)))
+                               health-files)
+        results (->> health-scripts
+                     (map #(.getAbsolutePath %))
+                     (map (partial process-script conf)))]
+    (log-message
+     (pr-str (map #'vector
+                  (map #(.getAbsolutePath %) health-scripts)
+                  results)))
+    ; failed_with_exit_code is OK. We're mimicing Hadoop's health checks.
+    ; We treat non-zero exit codes as indicators that the scripts failed
+    ; to execute properly, not that the system is unhealthy, in which case
+    ; we don't want to start killing things.
+    (if (every? #(or (= % :failed_with_exit_code)
+                     (= % :success))
+                results)
+      0
+      1)))
+
+(defn -main [& args]
+  (let [conf (read-storm-config)]
+    (System/exit
+     (health-check conf))))
diff --git a/storm-core/src/clj/backtype/storm/command/heartbeats.clj b/storm-core/src/clj/backtype/storm/command/heartbeats.clj
new file mode 100644
index 0000000..99790aa
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/command/heartbeats.clj
@@ -0,0 +1,52 @@
+;; 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.command.heartbeats
+  (:require [backtype.storm
+             [config :refer :all]
+             [log :refer :all]
+             [cluster :refer :all]
+             [converter :refer :all]]
+        [clojure.string :refer :all])
+  (:import [backtype.storm.generated ClusterWorkerHeartbeat]
+           [backtype.storm.utils Utils])
+  (:gen-class))
+
+(defn -main [command path & args]
+  (let [conf (read-storm-config)
+        cluster (mk-distributed-cluster-state conf :auth-conf conf)]
+    (println "Command: [" command "]")
+    (condp = command
+      "list"
+      (let [message (join " \n" (.get_worker_hb_children cluster path false))]
+        (log-message "list " path ":\n"
+                     message "\n"))
+      "get"
+      (log-message 
+       (if-let [hb (.get_worker_hb cluster path false)]
+         (clojurify-zk-worker-hb
+          (Utils/deserialize
+           hb
+           ClusterWorkerHeartbeat))
+         "Nothing"))
+      
+      (log-message "Usage: heartbeats [list|get] path"))
+    
+    (try
+      (.close cluster)
+      (catch Exception e
+        (log-message "Caught exception: " e " on close."))))
+  (System/exit 0))
+         
diff --git a/storm-core/src/clj/backtype/storm/command/kill_workers.clj b/storm-core/src/clj/backtype/storm/command/kill_workers.clj
new file mode 100644
index 0000000..3866cc7
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/command/kill_workers.clj
@@ -0,0 +1,33 @@
+;; 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.command.kill-workers
+  (:import [java.io File])
+  (:use [backtype.storm.daemon common])
+  (:use [backtype.storm util config])
+  (:require [backtype.storm.daemon
+             [supervisor :as supervisor]])
+  (:gen-class))
+
+(defn -main 
+  "Construct the supervisor-data from scratch and kill the workers on this supervisor"
+  [& args]
+  (let [conf (read-storm-config)
+        conf (assoc conf STORM-LOCAL-DIR (. (File. (conf STORM-LOCAL-DIR)) getCanonicalPath))
+        isupervisor (supervisor/standalone-supervisor)
+        supervisor-data (supervisor/supervisor-data conf nil isupervisor)
+        ids (supervisor/my-worker-ids conf)]
+    (doseq [id ids]
+      (supervisor/shutdown-worker supervisor-data id))))
diff --git a/storm-core/src/clj/backtype/storm/command/set_log_level.clj b/storm-core/src/clj/backtype/storm/command/set_log_level.clj
new file mode 100644
index 0000000..88b297d
--- /dev/null
+++ b/storm-core/src/clj/backtype/storm/command/set_log_level.clj
@@ -0,0 +1,75 @@
+;; 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.command.set-log-level
+  (:use [clojure.tools.cli :only [cli]])
+  (:use [backtype.storm thrift log])
+  (:import [org.apache.logging.log4j Level])
+  (:import [backtype.storm.generated LogConfig LogLevel LogLevelAction])
+  (:gen-class))
+
+(defn- get-storm-id
+  "Get topology id for a running topology from the topology name."
+  [nimbus name]
+  (let [info (.getClusterInfo nimbus)
+        topologies (.get_topologies info)
+        topology (first (filter (fn [topo] (= name (.get_name topo))) topologies))]
+    (if topology 
+      (.get_id topology)
+      (throw (.IllegalArgumentException (str name " is not a running topology"))))))
+
+(defn- parse-named-log-levels [action]
+  "Parses [logger name]=[level string]:[optional timeout],[logger name2]...
+
+   e.g. ROOT=DEBUG:30
+        root logger, debug for 30 seconds
+
+        org.apache.foo=WARN
+        org.apache.foo set to WARN indefinitely"
+  (fn [^String s]
+    (let [log-args (re-find #"(.*)=([A-Z]+):?(\d*)" s)
+          name (if (= action LogLevelAction/REMOVE) s (nth log-args 1))
+          level (Level/toLevel (nth log-args 2))
+          timeout-str (nth log-args 3)
+          log-level (LogLevel.)]
+      (if (= action LogLevelAction/REMOVE)
+        (.set_action log-level action)
+        (do
+          (.set_action log-level action)
+          (.set_target_log_level log-level (.toString level))
+          (.set_reset_log_level_timeout_secs log-level
+            (Integer. (if (= timeout-str "") "0" timeout-str)))))
+      {name log-level})))
+
+(defn- merge-together [previous key val]
+   (assoc previous key
+      (if-let [oldval (get previous key)]
+         (merge oldval val)
+         val)))
+
+(defn -main [& args]
+  (let [[{log-setting :log-setting remove-log-setting :remove-log-setting} [name] _]
+        (cli args ["-l" "--log-setting"
+                   :parse-fn (parse-named-log-levels LogLevelAction/UPDATE)
+                   :assoc-fn merge-together]
+                  ["-r" "--remove-log-setting"
+                   :parse-fn (parse-named-log-levels LogLevelAction/REMOVE)
+                   :assoc-fn merge-together])
+        log-config (LogConfig.)]
+    (doseq [[log-name log-val] (merge log-setting remove-log-setting)]
+      (.put_to_named_logger_level log-config log-name log-val))
+    (log-message "Sent log config " log-config " for topology " name)
+    (with-configured-nimbus-connection nimbus
+      (.setLogConfig nimbus (get-storm-id nimbus name) log-config))))
diff --git a/storm-core/src/clj/backtype/storm/config.clj b/storm-core/src/clj/backtype/storm/config.clj
index f7fabc4..94b66c3 100644
--- a/storm-core/src/clj/backtype/storm/config.clj
+++ b/storm-core/src/clj/backtype/storm/config.clj
@@ -17,18 +17,20 @@
 (ns backtype.storm.config
   (:import [java.io FileReader File IOException]
            [backtype.storm.generated StormTopology])
-  (:import [backtype.storm Config ConfigValidation$FieldValidator])
+  (:import [backtype.storm Config])
   (:import [backtype.storm.utils Utils LocalState])
+  (:import [backtype.storm.validation ConfigValidation])
   (:import [org.apache.commons.io FileUtils])
   (:require [clojure [string :as str]])
   (:use [backtype.storm log util]))
 
 (def RESOURCES-SUBDIR "resources")
+(def NIMBUS-DO-NOT-REASSIGN "NIMBUS-DO-NOT-REASSIGN")
 
 (defn- clojure-config-name [name]
   (.replace (.toUpperCase name) "_" "-"))
 
-;; define clojure constants for every configuration parameter
+; define clojure constants for every configuration parameter
 (doseq [f (seq (.getFields Config))]
   (let [name (.getName f)
         new-name (clojure-config-name name)]
@@ -39,35 +41,6 @@
   (dofor [f (seq (.getFields Config))]
          (.get f nil)))
 
-(defmulti get-FieldValidator class-selector)
-
-(defmethod get-FieldValidator nil [_]
-  (throw (IllegalArgumentException. "Cannot validate a nil field.")))
-
-(defmethod get-FieldValidator
-  ConfigValidation$FieldValidator [validator] validator)
-
-(defmethod get-FieldValidator Object
-  [klass]
-  {:pre [(not (nil? klass))]}
-  (reify ConfigValidation$FieldValidator
-    (validateField [this name v]
-                   (if (and (not (nil? v))
-                            (not (instance? klass v)))
-                     (throw (IllegalArgumentException.
-                              (str "field " name " '" v "' must be a '" (.getName klass) "'")))))))
-
-;; Create a mapping of config-string -> validator
-;; Config fields must have a _SCHEMA field defined
-(def CONFIG-SCHEMA-MAP
-  (->> (.getFields Config)
-       (filter #(not (re-matches #".*_SCHEMA$" (.getName %))))
-       (map (fn [f] [(.get f nil)
-                     (get-FieldValidator
-                       (-> Config
-                           (.getField (str (.getName f) "_SCHEMA"))
-                           (.get nil)))]))
-       (into {})))
 
 (defn cluster-mode
   [conf & args]
@@ -92,30 +65,13 @@
   [conf]
   (even-sampler (sampling-rate conf)))
 
-; storm.zookeeper.servers:
-;     - "server1"
-;     - "server2"
-;     - "server3"
-; nimbus.host: "master"
-;
-; ########### These all have default values as shown
-;
-; ### storm.* configs are general configurations
-; # the local dir is where jars are kept
-; storm.local.dir: "/mnt/storm"
-; storm.zookeeper.port: 2181
-; storm.zookeeper.root: "/storm"
-
 (defn read-default-config
   []
   (clojurify-structure (Utils/readDefaultConfig)))
 
 (defn validate-configs-with-schemas
   [conf]
-  (doseq [[k v] conf
-          :let [schema (CONFIG-SCHEMA-MAP k)]]
-    (if (not (nil? schema))
-      (.validateField schema k v))))
+  (ConfigValidation/validateFields conf))
 
 (defn read-storm-config
   []
@@ -134,7 +90,16 @@
 (defn absolute-storm-local-dir [conf]
   (let [storm-home (System/getProperty "storm.home")
         path (conf STORM-LOCAL-DIR)]
-    (if (is-absolute-path? path) path (str storm-home file-path-separator path))))
+    (if path
+      (if (is-absolute-path? path) path (str storm-home file-path-separator path))
+      (str storm-home file-path-separator "storm-local"))))
+
+(defn absolute-healthcheck-dir [conf]
+  (let [storm-home (System/getProperty "storm.home")
+        path (conf STORM-HEALTH-CHECK-DIR)]
+    (if path
+      (if (is-absolute-path? path) path (str storm-home file-path-separator path))
+      (str storm-home file-path-separator "healthchecks"))))
 
 (defn master-local-dir
   [conf]
@@ -223,6 +188,10 @@
   [conf]
   (LocalState. (str (supervisor-local-dir conf) file-path-separator "localstate")))
 
+(defn ^LocalState nimbus-topo-history-state
+  [conf]
+  (LocalState. (str (master-local-dir conf) file-path-separator "history")))
+
 (defn read-supervisor-storm-conf
   [conf storm-id]
   (let [stormroot (supervisor-stormdist-root conf storm-id)
@@ -263,6 +232,29 @@
   (log-message "REMOVE worker-user " worker-id)
   (.delete (File. (worker-user-file conf worker-id))))
 
+(defn worker-artifacts-root
+  ([conf]
+   (str (absolute-storm-local-dir conf) file-path-separator "workers-artifacts"))
+  ([conf id]
+   (str (worker-artifacts-root conf) file-path-separator id))
+  ([conf id port]
+   (str (worker-artifacts-root conf id) file-path-separator port)))
+
+(defn worker-artifacts-pid-path
+  [conf id port]
+  (str (worker-artifacts-root conf id port) file-path-separator "worker.pid"))
+
+(defn get-log-metadata-file
+  ([fname]
+    (let [[id port & _] (str/split fname (re-pattern file-path-separator))]
+      (get-log-metadata-file (read-storm-config) id port)))
+  ([conf id port]
+    (clojure.java.io/file (str (worker-artifacts-root conf id) file-path-separator port file-path-separator) "worker.yaml")))
+
+(defn get-worker-dir-from-root
+  [log-root id port]
+  (clojure.java.io/file (str log-root file-path-separator id file-path-separator port)))
+
 (defn worker-root
   ([conf]
    (str (absolute-storm-local-dir conf) file-path-separator "workers"))
@@ -287,3 +279,23 @@
 (defn ^LocalState worker-state
   [conf id]
   (LocalState. (worker-heartbeats-root conf id)))
+
+(defn override-login-config-with-system-property [conf]
+  (if-let [login_conf_file (System/getProperty "java.security.auth.login.config")]
+    (assoc conf "java.security.auth.login.config" login_conf_file)
+    conf))
+
+(defn get-topo-logs-users
+  [topology-conf]
+  (sort (distinct (remove nil?
+                    (concat
+                      (topology-conf LOGS-USERS)
+                      (topology-conf TOPOLOGY-USERS))))))
+
+(defn get-topo-logs-groups
+  [topology-conf]
+  (sort (distinct (remove nil?
+                    (concat
+                      (topology-conf LOGS-GROUPS)
+                      (topology-conf TOPOLOGY-GROUPS))))))
+
diff --git a/storm-core/src/clj/backtype/storm/converter.clj b/storm-core/src/clj/backtype/storm/converter.clj
index c571fa1..52a1817 100644
--- a/storm-core/src/clj/backtype/storm/converter.clj
+++ b/storm-core/src/clj/backtype/storm/converter.clj
@@ -14,9 +14,9 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.converter
-  (:import [backtype.storm.generated SupervisorInfo NodeInfo Assignment
+  (:import [backtype.storm.generated SupervisorInfo NodeInfo Assignment WorkerResources
             StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
-            TopologyActionOptions DebugOptions])
+            TopologyActionOptions DebugOptions ProfileRequest])
   (:use [backtype.storm util stats log])
   (:require [backtype.storm.daemon [common :as common]]))
 
@@ -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,22 +43,33 @@
       (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.)
-    (.set_master_code_dir (:master-code-dir assignment))
-    (.set_node_host (:node->host assignment))
-    (.set_executor_node_port (map-val
-                               (fn [node+port]
-                                 (NodeInfo. (first node+port) (set (map long (rest node+port)))))
-                               (map-key #(map long %)
-                                 (:executor->node+port assignment))))
-    (.set_executor_start_time_secs
-      (map-val
-        long
-        (map-key #(map long %)
-          (:executor->start-time-secs assignment))))))
+  (let [thrift-assignment (doto (Assignment.)
+                            (.set_master_code_dir (:master-code-dir assignment))
+                            (.set_node_host (:node->host assignment))
+                            (.set_executor_node_port (into {}
+                                                           (map (fn [[k v]]
+                                                                  [(map long k)
+                                                                   (NodeInfo. (first v) (set (map long (rest v))))])
+                                                                (:executor->node+port assignment))))
+                            (.set_executor_start_time_secs
+                              (into {}
+                                    (map (fn [[k v]]
+                                           [(map long k) (long v)])
+                                         (:executor->start-time-secs assignment)))))]
+    (if (:worker->resources assignment)
+      (.set_worker_resources thrift-assignment (into {} (map
+                                                          (fn [[node+port resources]]
+                                                            [(NodeInfo. (first node+port) (set (map long (rest node+port))))
+                                                             (doto (WorkerResources.)
+                                                               (.set_mem_on_heap (first resources))
+                                                               (.set_mem_off_heap (second resources))
+                                                               (.set_cpu (last resources)))])
+                                                          (:worker->resources assignment)))))
+    thrift-assignment))
 
 (defn clojurify-executor->node_port [executor->node_port]
   (into {}
@@ -69,6 +81,15 @@
           (into [] list-of-executors)) ; list of executors must be coverted to clojure vector to ensure it is sortable.
         executor->node_port))))
 
+(defn clojurify-worker->resources [worker->resources]
+  "convert worker info to be [node, port]
+   convert resources to be [mem_on_heap mem_off_heap cpu]"
+  (into {} (map
+             (fn [[nodeInfo resources]]
+               [(concat [(.get_node nodeInfo)] (.get_port nodeInfo))
+                [(.get_mem_on_heap resources) (.get_mem_off_heap resources) (.get_cpu resources)]])
+             worker->resources)))
+
 (defn clojurify-assignment [^Assignment assignment]
   (if assignment
     (backtype.storm.daemon.common.Assignment.
@@ -76,7 +97,8 @@
       (into {} (.get_node_host assignment))
       (clojurify-executor->node_port (into {} (.get_executor_node_port assignment)))
       (map-key (fn [executor] (into [] executor))
-        (into {} (.get_executor_start_time_secs assignment))))))
+        (into {} (.get_executor_start_time_secs assignment)))
+      (clojurify-worker->resources (into {} (.get_worker_resources assignment))))))
 
 (defn convert-to-symbol-from-status [status]
   (condp = status
@@ -227,6 +249,23 @@
     (.set_host (:host error))
     (.set_port (:port error))))
 
+(defn clojurify-profile-request
+  [^ProfileRequest request]
+  (when request
+    {:host (.get_node (.get_nodeInfo request))
+     :port (first (.get_port (.get_nodeInfo request)))
+     :action     (.get_action request)
+     :timestamp  (.get_time_stamp request)}))
+
+(defn thriftify-profile-request
+  [profile-request]
+  (let [nodeinfo (doto (NodeInfo.)
+                   (.set_node (:host profile-request))
+                   (.set_port (set [(:port profile-request)])))
+        request (ProfileRequest. nodeinfo (:action profile-request))]
+    (.set_time_stamp request (:timestamp profile-request))
+    request))
+
 (defn thriftify-credentials [credentials]
     (doto (Credentials.)
       (.set_creds (if credentials credentials {}))))
diff --git a/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj b/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj
index 990800a..0caa0b9 100644
--- a/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj
@@ -14,41 +14,42 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.daemon.builtin-metrics
-  (:import [backtype.storm.metric.api MultiCountMetric MultiReducedMetric CountMetric MeanReducer StateMetric IMetric IStatefulObject])
+  (:import [backtype.storm.metric.api CountMetric StateMetric IMetric IStatefulObject])
+  (:import [backtype.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
   (:import [backtype.storm Config])
-  (:use [backtype.storm.stats :only [stats-rate]]))
+  (:use [backtype.storm.stats]))
 
-(defrecord BuiltinSpoutMetrics [^MultiCountMetric ack-count                                
-                                ^MultiReducedMetric complete-latency
-                                ^MultiCountMetric fail-count
-                                ^MultiCountMetric emit-count
-                                ^MultiCountMetric transfer-count])
-(defrecord BuiltinBoltMetrics [^MultiCountMetric ack-count
-                               ^MultiReducedMetric process-latency
-                               ^MultiCountMetric fail-count
-                               ^MultiCountMetric execute-count
-                               ^MultiReducedMetric execute-latency
-                               ^MultiCountMetric emit-count
-                               ^MultiCountMetric transfer-count])
+(defrecord BuiltinSpoutMetrics [^MultiCountStatAndMetric ack-count
+                                ^MultiLatencyStatAndMetric complete-latency
+                                ^MultiCountStatAndMetric fail-count
+                                ^MultiCountStatAndMetric emit-count
+                                ^MultiCountStatAndMetric transfer-count])
+(defrecord BuiltinBoltMetrics [^MultiCountStatAndMetric ack-count
+                               ^MultiLatencyStatAndMetric process-latency
+                               ^MultiCountStatAndMetric fail-count
+                               ^MultiCountStatAndMetric execute-count
+                               ^MultiLatencyStatAndMetric execute-latency
+                               ^MultiCountStatAndMetric emit-count
+                               ^MultiCountStatAndMetric transfer-count])
 (defrecord SpoutThrottlingMetrics [^CountMetric skipped-max-spout
                                    ^CountMetric skipped-throttle
                                    ^CountMetric skipped-inactive])
 
 
-(defn make-data [executor-type]
+(defn make-data [executor-type stats]
   (condp = executor-type
-    :spout (BuiltinSpoutMetrics. (MultiCountMetric.)
-                                 (MultiReducedMetric. (MeanReducer.))
-                                 (MultiCountMetric.)
-                                 (MultiCountMetric.)
-                                 (MultiCountMetric.))
-    :bolt (BuiltinBoltMetrics. (MultiCountMetric.)
-                               (MultiReducedMetric. (MeanReducer.))
-                               (MultiCountMetric.)
-                               (MultiCountMetric.)
-                               (MultiReducedMetric. (MeanReducer.))
-                               (MultiCountMetric.)
-                               (MultiCountMetric.))))
+    :spout (BuiltinSpoutMetrics. (stats-acked stats)
+                                 (stats-complete-latencies stats)
+                                 (stats-failed stats)
+                                 (stats-emitted stats)
+                                 (stats-transferred stats))
+    :bolt (BuiltinBoltMetrics. (stats-acked stats)
+                               (stats-process-latencies stats)
+                               (stats-failed stats)
+                               (stats-executed stats)
+                               (stats-execute-latencies stats)
+                               (stats-emitted stats)
+                               (stats-transferred stats))))
 
 (defn make-spout-throttling-data []
   (SpoutThrottlingMetrics. (CountMetric.)
@@ -87,33 +88,6 @@
     (.registerMetric topology-context (str "__" (name qname)) (StateMetric. q)
                      (int (get storm-conf Config/TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)))))
 
-(defn spout-acked-tuple! [^BuiltinSpoutMetrics m stats stream latency-ms]  
-  (-> m .ack-count (.scope stream) (.incrBy (stats-rate stats)))
-  (-> m .complete-latency (.scope stream) (.update latency-ms)))
-
-(defn spout-failed-tuple! [^BuiltinSpoutMetrics m stats stream]  
-  (-> m .fail-count (.scope stream) (.incrBy (stats-rate stats))))
-
-(defn bolt-execute-tuple! [^BuiltinBoltMetrics m stats comp-id stream latency-ms]
-  (let [scope (str comp-id ":" stream)]    
-    (-> m .execute-count (.scope scope) (.incrBy (stats-rate stats)))
-    (-> m .execute-latency (.scope scope) (.update latency-ms))))
-
-(defn bolt-acked-tuple! [^BuiltinBoltMetrics m stats comp-id stream latency-ms]
-  (let [scope (str comp-id ":" stream)]
-    (-> m .ack-count (.scope scope) (.incrBy (stats-rate stats)))
-    (-> m .process-latency (.scope scope) (.update latency-ms))))
-
-(defn bolt-failed-tuple! [^BuiltinBoltMetrics m stats comp-id stream]
-  (let [scope (str comp-id ":" stream)]    
-    (-> m .fail-count (.scope scope) (.incrBy (stats-rate stats)))))
-
-(defn emitted-tuple! [m stats stream]
-  (-> m :emit-count (.scope stream) (.incrBy (stats-rate stats))))
-
-(defn transferred-tuple! [m stats stream num-out-tasks]
-  (-> m :transfer-count (.scope stream) (.incrBy (* num-out-tasks (stats-rate stats)))))
-
 (defn skipped-max-spout! [^SpoutThrottlingMetrics m stats]
   (-> m .skipped-max-spout (.incrBy (stats-rate stats))))
 
diff --git a/storm-core/src/clj/backtype/storm/daemon/common.clj b/storm-core/src/clj/backtype/storm/daemon/common.clj
index 9da0e4f..9b3aab3 100644
--- a/storm-core/src/clj/backtype/storm/daemon/common.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/common.clj
@@ -16,7 +16,8 @@
 (ns backtype.storm.daemon.common
   (:use [backtype.storm log config util])
   (:import [backtype.storm.generated StormTopology
-            InvalidTopologyException GlobalStreamId])
+            InvalidTopologyException GlobalStreamId]
+           [backtype.storm.utils ThriftTopologyUtils])
   (:import [backtype.storm.utils Utils])
   (:import [backtype.storm.task WorkerTopologyContext])
   (:import [backtype.storm Constants])
@@ -27,10 +28,10 @@
   (:require [clojure.set :as set])  
   (:require [backtype.storm.daemon.acker :as acker])
   (:require [backtype.storm.thrift :as thrift])
-  )
+  (:require [metrics.reporters.jmx :as jmx]))
 
-(defn system-id? [id]
-  (Utils/isSystemId id))
+(defn start-metrics-reporters []
+  (jmx/start (jmx/reporter {})))
 
 (def ACKER-COMPONENT-ID acker/ACKER-COMPONENT-ID)
 (def ACKER-INIT-STREAM-ID acker/ACKER-INIT-STREAM-ID)
@@ -51,13 +52,13 @@
 ;; the task id is the virtual port
 ;; node->host is here so that tasks know who to talk to just from assignment
 ;; this avoid situation where node goes down and task doesn't know what to do information-wise
-(defrecord Assignment [master-code-dir node->host executor->node+port executor->start-time-secs])
+(defrecord Assignment [master-code-dir node->host executor->node+port executor->start-time-secs worker->resources])
 
 
 ;; 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]))
@@ -113,22 +114,23 @@
               (str "Duplicate component ids: " offending))))
     (doseq [f thrift/STORM-TOPOLOGY-FIELDS
             :let [obj-map (.getFieldValue topology f)]]
-      (doseq [id (keys obj-map)]
-        (if (system-id? id)
-          (throw (InvalidTopologyException.
-                  (str id " is not a valid component id")))))
-      (doseq [obj (vals obj-map)
-              id (-> obj .get_common .get_streams keys)]
-        (if (system-id? id)
-          (throw (InvalidTopologyException.
-                  (str id " is not a valid stream id"))))))
-    ))
+      (if-not (ThriftTopologyUtils/isWorkerHook f)
+        (do
+          (doseq [id (keys obj-map)]
+            (if (Utils/isSystemId id)
+              (throw (InvalidTopologyException.
+                       (str id " is not a valid component id")))))
+          (doseq [obj (vals obj-map)
+                  id (-> obj .get_common .get_streams keys)]
+            (if (Utils/isSystemId id)
+              (throw (InvalidTopologyException.
+                       (str id " is not a valid stream id"))))))))))
 
 (defn all-components [^StormTopology topology]
   (apply merge {}
-         (for [f thrift/STORM-TOPOLOGY-FIELDS]
-           (.getFieldValue topology f)
-           )))
+    (for [f thrift/STORM-TOPOLOGY-FIELDS]
+      (if-not (ThriftTopologyUtils/isWorkerHook f)
+        (.getFieldValue topology f)))))
 
 (defn component-conf [component]
   (->> component
@@ -310,7 +312,7 @@
     (doseq [[_ component] (all-components ret)
             :let [common (.get_common component)]]
       (.put_to_streams common EVENTLOGGER-STREAM-ID (thrift/output-fields (eventlogger-bolt-fields))))
-    (.put_to_bolts ret "__eventlogger" eventlogger-bolt)
+    (.put_to_bolts ret EVENTLOGGER-COMPONENT-ID eventlogger-bolt)
     ))
 
 (defn add-metric-components! [storm-conf ^StormTopology topology]  
diff --git a/storm-core/src/clj/backtype/storm/daemon/drpc.clj b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
index bbc9611..40744fb 100644
--- a/storm-core/src/clj/backtype/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/drpc.clj
@@ -31,8 +31,16 @@
   (:use compojure.core)
   (:use ring.middleware.reload)
   (:require [compojure.handler :as handler])
+  (:require [metrics.meters :refer [defmeter mark!]])
   (:gen-class))
 
+(defmeter drpc:num-execute-http-requests)
+(defmeter drpc:num-execute-calls)
+(defmeter drpc:num-result-calls)
+(defmeter drpc:num-failRequest-calls)
+(defmeter drpc:num-fetchRequest-calls)
+(defmeter drpc:num-shutdown-calls)
+
 (defn timeout-check-secs [] 5)
 
 (defn acquire-queue [queues-atom function]
@@ -45,6 +53,8 @@
 
 (defn check-authorization
   ([aclHandler mapping operation context]
+    (if (not-nil? context)
+      (log-thrift-access (.requestID context) (.remoteAddress context) (.principal context) operation))
     (if aclHandler
       (let [context (or context (ReqContext/context))]
         (if-not (.permit aclHandler context operation mapping)
@@ -85,6 +95,7 @@
     (reify DistributedRPC$Iface
       (^String execute
         [this ^String function ^String args]
+        (mark! drpc:num-execute-calls)
         (log-debug "Received DRPC request for " function " (" args ") at " (System/currentTimeMillis))
         (check-authorization drpc-acl-handler
                              {DRPCAuthorizerBase/FUNCTION_NAME function}
@@ -114,6 +125,7 @@
 
       (^void result
         [this ^String id ^String result]
+        (mark! drpc:num-result-calls)
         (when-let [func (@id->function id)]
           (check-authorization drpc-acl-handler
                                {DRPCAuthorizerBase/FUNCTION_NAME func}
@@ -127,6 +139,7 @@
 
       (^void failRequest
         [this ^String id]
+        (mark! drpc:num-failRequest-calls)
         (when-let [func (@id->function id)]
           (check-authorization drpc-acl-handler
                                {DRPCAuthorizerBase/FUNCTION_NAME func}
@@ -138,6 +151,7 @@
 
       (^DRPCRequest fetchRequest
         [this ^String func]
+        (mark! drpc:num-fetchRequest-calls)
         (check-authorization drpc-acl-handler
                              {DRPCAuthorizerBase/FUNCTION_NAME func}
                              "fetchRequest")
@@ -152,41 +166,39 @@
 
       (shutdown
         [this]
+        (mark! drpc:num-shutdown-calls)
         (.interrupt clear-thread)))))
 
 (defn handle-request [handler]
   (fn [request]
     (handler request)))
 
+(defn populate-context!
+  "Populate the Storm RequestContext from an servlet-request. This should be called in each handler"
+  [http-creds-handler servlet-request]
+    (when http-creds-handler
+      (.populateContext http-creds-handler (ReqContext/context) servlet-request)))
+
 (defn webapp [handler http-creds-handler]
+  (mark! drpc:num-execute-http-requests)
   (->
     (routes
       (POST "/drpc/:func" [:as {:keys [body servlet-request]} func & m]
         (let [args (slurp body)]
-          (if http-creds-handler
-            (.populateContext http-creds-handler (ReqContext/context)
-                              servlet-request))
+          (populate-context! http-creds-handler servlet-request)
           (.execute handler func args)))
       (POST "/drpc/:func/" [:as {:keys [body servlet-request]} func & m]
         (let [args (slurp body)]
-          (if http-creds-handler
-            (.populateContext http-creds-handler (ReqContext/context)
-                              servlet-request))
+          (populate-context! http-creds-handler servlet-request)
           (.execute handler func args)))
       (GET "/drpc/:func/:args" [:as {:keys [servlet-request]} func args & m]
-          (if http-creds-handler
-            (.populateContext http-creds-handler (ReqContext/context)
-                              servlet-request))
+          (populate-context! http-creds-handler servlet-request)
           (.execute handler func args))
       (GET "/drpc/:func/" [:as {:keys [servlet-request]} func & m]
-          (if http-creds-handler
-            (.populateContext http-creds-handler (ReqContext/context)
-                              servlet-request))
+          (populate-context! http-creds-handler servlet-request)
           (.execute handler func ""))
       (GET "/drpc/:func" [:as {:keys [servlet-request]} func & m]
-          (if http-creds-handler
-            (.populateContext http-creds-handler (ReqContext/context)
-                              servlet-request))
+          (populate-context! http-creds-handler servlet-request)
           (.execute handler func "")))
     (wrap-reload '[backtype.storm.daemon.drpc])
     handle-request))
@@ -217,7 +229,8 @@
       (log-message "Starting Distributed RPC servers...")
       (future (.serve invoke-server))
       (when (> drpc-http-port 0)
-        (let [app (webapp drpc-service-handler http-creds-handler)
+        (let [app (-> (webapp drpc-service-handler http-creds-handler)
+                    requests-middleware)
               filter-class (conf DRPC-HTTP-FILTER)
               filter-params (conf DRPC-HTTP-FILTER-PARAMS)
               filters-confs [{:filter-class filter-class
@@ -248,6 +261,7 @@
                                         https-need-client-auth
                                         https-want-client-auth)
                             (config-filter server app filters-confs))})))
+      (start-metrics-reporters)
       (when handler-server
         (.serve handler-server)))))
 
diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj
index d7a68be..2aeb5e7 100644
--- a/storm-core/src/clj/backtype/storm/daemon/executor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj
@@ -21,7 +21,7 @@
   (:import [java.util List Random HashMap ArrayList LinkedList Map])
   (:import [backtype.storm ICredentialsListener])
   (:import [backtype.storm.hooks ITaskHook])
-  (:import [backtype.storm.tuple Tuple Fields TupleImpl MessageId])
+  (:import [backtype.storm.tuple AddressedTuple Tuple Fields TupleImpl MessageId])
   (:import [backtype.storm.spout ISpoutWaitStrategy ISpout SpoutOutputCollector ISpoutOutputCollector])
   (:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo
             EmitInfo BoltFailInfo BoltAckInfo BoltExecuteInfo])
@@ -30,10 +30,11 @@
   (:import [backtype.storm.generated GlobalStreamId])
   (:import [backtype.storm.utils Utils MutableObject RotatingMap RotatingMap$ExpiredCallback MutableLong Time DisruptorQueue WorkerBackpressureThread])
   (:import [com.lmax.disruptor InsufficientCapacityException])
-  (:import [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer])
+  (:import [backtype.storm.serialization KryoTupleSerializer])
   (:import [backtype.storm.daemon Shutdownable])
   (:import [backtype.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
   (:import [backtype.storm Config Constants])
+  (:import [backtype.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
   (:import [java.util.concurrent ConcurrentLinkedQueue])
   (:require [backtype.storm [tuple :as tuple] [thrift :as thrift]
              [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
@@ -41,54 +42,59 @@
   (:require [backtype.storm.daemon.builtin-metrics :as builtin-metrics])
   (:require [clojure.set :as set]))
 
-(defn- mk-fields-grouper [^Fields out-fields ^Fields group-fields ^List target-tasks]
+(defn- mk-fields-grouper
+  [^Fields out-fields ^Fields group-fields ^List target-tasks]
   (let [num-tasks (count target-tasks)
         task-getter (fn [i] (.get target-tasks i))]
-    (fn [task-id ^List values]
+    (fn [task-id ^List values load]
       (-> (.select out-fields group-fields values)
           tuple/list-hash-code
           (mod num-tasks)
           task-getter))))
 
-(defn- mk-shuffle-grouper [^List target-tasks]
-  (let [choices (rotating-random-range target-tasks)]
-    (fn [task-id tuple]
-      (acquire-random-range-id choices))))
-
-(defn- mk-custom-grouper [^CustomStreamGrouping grouping ^WorkerTopologyContext context ^String component-id ^String stream-id target-tasks]
+(defn- mk-custom-grouper
+  [^CustomStreamGrouping grouping ^WorkerTopologyContext context ^String component-id ^String stream-id target-tasks]
   (.prepare grouping context (GlobalStreamId. component-id stream-id) target-tasks)
-  (fn [task-id ^List values]
-    (.chooseTasks grouping task-id values)
-    ))
+  (if (instance? LoadAwareCustomStreamGrouping grouping)
+    (fn [task-id ^List values load]
+      (.chooseTasks grouping task-id values load))
+    (fn [task-id ^List values load]
+      (.chooseTasks grouping task-id values))))
+
+(defn mk-shuffle-grouper
+  [^List target-tasks topo-conf ^WorkerTopologyContext context ^String component-id ^String stream-id]
+  (if (.get topo-conf TOPOLOGY-DISABLE-LOADAWARE-MESSAGING)
+    (mk-custom-grouper (ShuffleGrouping.) context component-id stream-id target-tasks)
+    (mk-custom-grouper (LoadAwareShuffleGrouping.) context component-id stream-id target-tasks)))
 
 (defn- mk-grouper
   "Returns a function that returns a vector of which task indices to send tuple to, or just a single task index."
-  [^WorkerTopologyContext context component-id stream-id ^Fields out-fields thrift-grouping ^List target-tasks]
+  [^WorkerTopologyContext context component-id stream-id ^Fields out-fields thrift-grouping ^List target-tasks topo-conf]
   (let [num-tasks (count target-tasks)
         random (Random.)
         target-tasks (vec (sort target-tasks))]
     (condp = (thrift/grouping-type thrift-grouping)
       :fields
         (if (thrift/global-grouping? thrift-grouping)
-          (fn [task-id tuple]
+          (fn [task-id tuple load]
             ;; It's possible for target to have multiple tasks if it reads multiple sources
             (first target-tasks))
           (let [group-fields (Fields. (thrift/field-grouping thrift-grouping))]
             (mk-fields-grouper out-fields group-fields target-tasks)
             ))
       :all
-        (fn [task-id tuple] target-tasks)
+        (fn [task-id tuple load] target-tasks)
       :shuffle
-        (mk-shuffle-grouper target-tasks)
+        (mk-shuffle-grouper target-tasks topo-conf context component-id stream-id)
       :local-or-shuffle
         (let [same-tasks (set/intersection
                            (set target-tasks)
                            (set (.getThisWorkerTasks context)))]
           (if-not (empty? same-tasks)
-            (mk-shuffle-grouper (vec same-tasks))
-            (mk-shuffle-grouper target-tasks)))
+            (mk-shuffle-grouper (vec same-tasks) topo-conf context component-id stream-id)
+            (mk-shuffle-grouper target-tasks topo-conf context component-id stream-id)))
       :none
-        (fn [task-id tuple]
+        (fn [task-id tuple load]
           (let [i (mod (.nextInt random) num-tasks)]
             (get target-tasks i)
             ))
@@ -102,7 +108,8 @@
         :direct
       )))
 
-(defn- outbound-groupings [^WorkerTopologyContext worker-context this-component-id stream-id out-fields component->grouping]
+(defn- outbound-groupings
+  [^WorkerTopologyContext worker-context this-component-id stream-id out-fields component->grouping topo-conf]
   (->> component->grouping
        (filter-key #(-> worker-context
                         (.getComponentTasks %)
@@ -116,13 +123,13 @@
                             out-fields
                             tgrouping
                             (.getComponentTasks worker-context component)
-                            )]))
+                            topo-conf)]))
        (into {})
        (HashMap.)))
 
 (defn outbound-components
   "Returns map of stream id to component id to grouper"
-  [^WorkerTopologyContext worker-context component-id]
+  [^WorkerTopologyContext worker-context component-id topo-conf]
   (->> (.getTargets worker-context component-id)
         clojurify-structure
         (map (fn [[stream-id component->grouping]]
@@ -132,7 +139,8 @@
                   component-id
                   stream-id
                   (.getComponentOutputFields worker-context component-id stream-id)
-                  component->grouping)]))
+                  component->grouping
+                  topo-conf)]))
          (into {})
          (HashMap.)))
 
@@ -159,6 +167,10 @@
                         TOPOLOGY-TICK-TUPLE-FREQ-SECS
                         TOPOLOGY-SLEEP-SPOUT-WAIT-STRATEGY-TIME-MS
                         TOPOLOGY-SPOUT-WAIT-STRATEGY
+                        TOPOLOGY-BOLTS-WINDOW-LENGTH-COUNT
+                        TOPOLOGY-BOLTS-WINDOW-LENGTH-DURATION-MS
+                        TOPOLOGY-BOLTS-SLIDING-INTERVAL-COUNT
+                        TOPOLOGY-BOLTS-SLIDING-INTERVAL-DURATION-MS
                         )
         spec-conf (-> general-context
                       (.getComponentCommon component-id)
@@ -197,23 +209,11 @@
 ;; in its own function so that it can be mocked out by tracked topologies
 (defn mk-executor-transfer-fn [batch-transfer->worker storm-conf]
   (fn this
-    ([task tuple block? ^ConcurrentLinkedQueue overflow-buffer]
+    [task tuple]
+    (let [val (AddressedTuple. task tuple)]
       (when (= true (storm-conf TOPOLOGY-DEBUG))
-        (log-message "TRANSFERING tuple TASK: " task " TUPLE: " tuple))
-      (if (and overflow-buffer (not (.isEmpty overflow-buffer)))
-        (.add overflow-buffer [task tuple])
-        (try-cause
-          (disruptor/publish batch-transfer->worker [task tuple] block?)
-        (catch InsufficientCapacityException e
-          (if overflow-buffer
-            (.add overflow-buffer [task tuple])
-            (throw e))
-          ))))
-    ([task tuple overflow-buffer]
-      (this task tuple (nil? overflow-buffer) overflow-buffer))
-    ([task tuple]
-      (this task tuple nil)
-      )))
+        (log-message "TRANSFERING tuple " val))
+      (disruptor/publish batch-transfer->worker val))))
 
 (defn mk-executor-data [worker executor-id]
   (let [worker-context (worker-context worker)
@@ -225,8 +225,9 @@
                                   (str "executor"  executor-id "-send-queue")
                                   (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                  :claim-strategy :single-threaded
-                                  :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
+                                  :producer-type :single-threaded
+                                  :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
+                                  :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
         ]
     (recursive-map
      :worker worker
@@ -252,7 +253,7 @@
      :stats (mk-executor-stats <> (sampling-rate storm-conf))
      :interval->task->metric-registry (HashMap.)
      :task->component (:task->component worker)
-     :stream->component->grouper (outbound-components worker-context component-id)
+     :stream->component->grouper (outbound-components worker-context component-id storm-conf)
      :report-error (throttled-report-error-fn <>)
      :report-error-and-die (fn [error]
                              ((:report-error <>) error)
@@ -261,7 +262,6 @@
                                     (exception-cause? java.io.InterruptedIOException error))
                                (log-message "Got interrupted excpetion shutting thread down...")
                                ((:suicide-fn <>))))
-     :deserializer (KryoTupleDeserializer. storm-conf worker-context)
      :sampler (mk-stats-sampler storm-conf)
      :backpressure (atom false)
      :spout-throttling-metrics (if (= executor-type :spout) 
@@ -284,6 +284,7 @@
       "When receive queue is below lowWaterMark"
       (if @(:backpressure executor-data)
         (do (reset! (:backpressure executor-data) false)
+            (log-debug "executor " (:executor-id executor-data) " is not-congested, set backpressure flag false")
             (WorkerBackpressureThread/notifyBackpressureChecker (:backpressure-trigger (:worker executor-data))))))))
 
 (defn start-batch-transfer->worker-handler! [worker executor-data]
@@ -299,8 +300,7 @@
           (.add alist o)
           (when batch-end?
             (worker-transfer-fn serializer alist)
-            (.setObject cached-emit (ArrayList.))
-            )))
+            (.setObject cached-emit (ArrayList.)))))
       :kill-fn (:report-error-and-die executor-data))))
 
 (defn setup-metrics! [executor-data]
@@ -312,12 +312,11 @@
        interval
        interval
        (fn []
-         (disruptor/publish
-          receive-queue
-          [[nil (TupleImpl. worker-context [interval] Constants/SYSTEM_TASK_ID Constants/METRICS_TICK_STREAM_ID)]]))))))
+         (let [val [(AddressedTuple. AddressedTuple/BROADCAST_DEST (TupleImpl. worker-context [interval] Constants/SYSTEM_TASK_ID Constants/METRICS_TICK_STREAM_ID))]]
+           (disruptor/publish receive-queue val)))))))
 
 (defn metrics-tick
-  ([executor-data task-data ^TupleImpl tuple overflow-buffer]
+  [executor-data task-data ^TupleImpl tuple]
    (let [{:keys [interval->task->metric-registry ^WorkerTopologyContext worker-context]} executor-data
          interval (.getInteger tuple 0)
          task-id (:task-id task-data)
@@ -336,12 +335,8 @@
                                      (IMetricsConsumer$DataPoint. name value)))))
                           (filter identity)
                           (into []))]
-     (if (seq data-points)
-       (task/send-unanchored task-data Constants/METRICS_STREAM_ID [task-info data-points] overflow-buffer))))
-  ([executor-data task-data ^TupleImpl tuple]
-    (metrics-tick executor-data task-data tuple nil)
-    ))
-
+     (when (seq data-points)
+       (task/send-unanchored task-data Constants/METRICS_STREAM_ID [task-info data-points]))))
 
 (defn setup-ticks! [worker executor-data]
   (let [storm-conf (:storm-conf executor-data)
@@ -349,7 +344,7 @@
         receive-queue (:receive-queue executor-data)
         context (:worker-context executor-data)]
     (when tick-time-secs
-      (if (or (system-id? (:component-id executor-data))
+      (if (or (Utils/isSystemId (:component-id executor-data))
               (and (= false (storm-conf TOPOLOGY-ENABLE-MESSAGE-TIMEOUTS))
                    (= :spout (:type executor-data))))
         (log-message "Timeouts disabled for executor " (:component-id executor-data) ":" (:executor-id executor-data))
@@ -358,10 +353,8 @@
           tick-time-secs
           tick-time-secs
           (fn []
-            (disruptor/publish
-              receive-queue
-              [[nil (TupleImpl. context [tick-time-secs] Constants/SYSTEM_TASK_ID Constants/SYSTEM_TICK_STREAM_ID)]]
-              )))))))
+            (let [val [(AddressedTuple. AddressedTuple/BROADCAST_DEST (TupleImpl. context [tick-time-secs] Constants/SYSTEM_TASK_ID Constants/SYSTEM_TICK_STREAM_ID))]]
+              (disruptor/publish receive-queue val))))))))
 
 (defn mk-executor [worker executor-id initial-credentials]
   (let [executor-data (mk-executor-data worker executor-id)
@@ -401,11 +394,9 @@
         executor-id)
       (credentials-changed [this creds]
         (let [receive-queue (:receive-queue executor-data)
-              context (:worker-context executor-data)]
-          (disruptor/publish
-            receive-queue
-            [[nil (TupleImpl. context [creds] Constants/SYSTEM_TASK_ID Constants/CREDENTIALS_CHANGED_STREAM_ID)]]
-              )))
+              context (:worker-context executor-data)
+              val [(AddressedTuple. AddressedTuple/BROADCAST_DEST (TupleImpl. context [creds] Constants/SYSTEM_TASK_ID Constants/CREDENTIALS_CHANGED_STREAM_ID))]]
+          (disruptor/publish receive-queue val)))
       (get-backpressure-flag [this]
         @(:backpressure executor-data))
       Shutdownable
@@ -438,7 +429,6 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (builtin-metrics/spout-failed-tuple! (:builtin-metrics task-data) (:stats executor-data) (:stream tuple-info))      
       (stats/spout-failed-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id]
@@ -450,20 +440,19 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (builtin-metrics/spout-acked-tuple! (:builtin-metrics task-data) (:stats executor-data) (:stream tuple-info) time-delta)
       (stats/spout-acked-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
-  (let [^KryoTupleDeserializer deserializer (:deserializer executor-data)
-        task-ids (:task-ids executor-data)
+  (let [task-ids (:task-ids executor-data)
         debug? (= true (-> executor-data :storm-conf (get TOPOLOGY-DEBUG)))
         ]
     (disruptor/clojure-handler
       (fn [tuple-batch sequence-id end-of-batch?]
-        (fast-list-iter [[task-id msg] tuple-batch]
-          (let [^TupleImpl tuple (if (instance? Tuple msg) msg (.deserialize deserializer msg))]
+        (fast-list-iter [^AddressedTuple addressed-tuple tuple-batch]
+          (let [^TupleImpl tuple (.getTuple addressed-tuple)
+                task-id (.getDest addressed-tuple)]
             (when debug? (log-message "Processing received message FOR " task-id " TUPLE: " tuple))
-            (if task-id
+            (if (not= task-id AddressedTuple/BROADCAST_DEST)
               (tuple-action-fn task-id tuple)
               ;; null task ids are broadcast tuples
               (fast-list-iter [task-id task-ids]
@@ -483,18 +472,17 @@
 
 ;; Send sampled data to the eventlogger if the global or component level
 ;; debug flag is set (via nimbus api).
-(defn send-to-eventlogger [executor-data task-data values overflow-buffer component-id message-id random]
+(defn send-to-eventlogger [executor-data task-data values component-id message-id random]
     (let [c->d @(:storm-component->debug-atom executor-data)
           options (get c->d component-id (get c->d (:storm-id executor-data)))
           spct    (if (and (not-nil? options) (:enable options)) (:samplingpct options) 0)]
       ;; the thread's initialized random number generator is used to generate
       ;; uniformily distributed random numbers.
-      (if (and (> spct 0) (< (* 100 (.nextDouble random)) spct))
+      (when (and (> spct 0) (< (* 100 (.nextDouble random)) spct))
         (task/send-unanchored
           task-data
           EVENTLOGGER-STREAM-ID
-          [component-id message-id (System/currentTimeMillis) values]
-          overflow-buffer))))
+          [component-id message-id (System/currentTimeMillis) values]))))
 
 (defmethod mk-threads :spout [executor-data task-datas initial-credentials]
   (let [{:keys [storm-conf component-id worker-context transfer-fn report-error sampler open-or-prepare-was-called?]} executor-data
@@ -504,15 +492,7 @@
         last-active (atom false)        
         spouts (ArrayList. (map :object (vals task-datas)))
         rand (Random. (Utils/secureRandomLong))
-
-        ;; the overflow buffer is used to ensure that spouts never block when emitting
-        ;; this ensures that the spout can always clear the incoming buffer (acks and fails), which
-        ;; prevents deadlock from occuring across the topology (e.g. Spout -> Bolt -> Acker -> Spout, and all
-        ;; buffers filled up)
-        ;; when the overflow buffer is full, spouts stop calling nextTuple until it's able to clear the overflow buffer
-        ;; this limits the size of the overflow buffer to however many tuples a spout emits in one call of nextTuple, 
-        ;; preventing memory issues
-        overflow-buffer (ConcurrentLinkedQueue.)
+        ^DisruptorQueue transfer-queue (executor-data :batch-transfer-queue)
 
         pending (RotatingMap.
                  2 ;; microoptimize for performance of .size method
@@ -525,7 +505,7 @@
                           (let [stream-id (.getSourceStreamId tuple)]
                             (condp = stream-id
                               Constants/SYSTEM_TICK_STREAM_ID (.rotate pending)
-                              Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple overflow-buffer)
+                              Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple)
                               Constants/CREDENTIALS_CHANGED_STREAM_ID 
                                 (let [task-data (get task-datas task-id)
                                       spout-obj (:object task-data)]
@@ -570,7 +550,7 @@
                                                          (tasks-fn out-stream-id values))
                                              rooted? (and message-id has-ackers?)
                                              root-id (if rooted? (MessageId/generateId rand))
-                                             out-ids (fast-list-for [t out-tasks] (if rooted? (MessageId/generateId rand)))]
+                                             ^List out-ids (fast-list-for [t out-tasks] (if rooted? (MessageId/generateId rand)))]
                                          (fast-list-iter [out-task out-tasks id out-ids]
                                                          (let [tuple-id (if rooted?
                                                                           (MessageId/makeRootId root-id id)
@@ -580,12 +560,9 @@
                                                                                      task-id
                                                                                      out-stream-id
                                                                                      tuple-id)]
-                                                           (transfer-fn out-task
-                                                                        out-tuple
-                                                                        overflow-buffer)
-                                                           ))
+                                                           (transfer-fn out-task out-tuple)))
                                          (if has-eventloggers?
-                                           (send-to-eventlogger executor-data task-data values overflow-buffer component-id message-id rand))
+                                           (send-to-eventlogger executor-data task-data values component-id message-id rand))
                                          (if (and rooted?
                                                   (not (.isEmpty out-ids)))
                                            (do
@@ -595,8 +572,7 @@
                                                                     (if (sampler) (System/currentTimeMillis))])
                                              (task/send-unanchored task-data
                                                                    ACKER-INIT-STREAM-ID
-                                                                   [root-id (bit-xor-vals out-ids) task-id]
-                                                                   overflow-buffer))
+                                                                   [root-id (bit-xor-vals out-ids) task-id]))
                                            (when message-id
                                              (ack-spout-msg executor-data task-data message-id
                                                             {:stream out-stream-id :values values}
@@ -631,48 +607,40 @@
         (log-message "Opened spout " component-id ":" (keys task-datas))
         (setup-metrics! executor-data)
         
-        (disruptor/consumer-started! (:receive-queue executor-data))
         (fn []
           ;; This design requires that spouts be non-blocking
           (disruptor/consume-batch receive-queue event-handler)
           
-          ;; try to clear the overflow-buffer
-          (try-cause
-            (while (not (.isEmpty overflow-buffer))
-              (let [[out-task out-tuple] (.peek overflow-buffer)]
-                (transfer-fn out-task out-tuple false nil)
-                (.poll overflow-buffer)))
-          (catch InsufficientCapacityException e
-            ))
-          
           (let [active? @(:storm-active-atom executor-data)
                 curr-count (.get emitted-count)
-                ;; suspend-time ((:storm-conf executor-data) BACKPRESSURE-SPOUT-SUSPEND-TIME-MS)
                 backpressure-enabled ((:storm-conf executor-data) TOPOLOGY-BACKPRESSURE-ENABLE)
                 throttle-on (and backpressure-enabled
                               @(:throttle-on (:worker executor-data)))
                 reached-max-spout-pending (and max-spout-pending
                                                (>= (.size pending) max-spout-pending))
                 ]
-            (if (and (.isEmpty overflow-buffer)
-                     (not throttle-on)
-                     (not reached-max-spout-pending))
-              (if active?
-                (do
-                  (when-not @last-active
-                    (reset! last-active true)
-                    (log-message "Activating spout " component-id ":" (keys task-datas))
-                    (fast-list-iter [^ISpout spout spouts] (.activate spout)))
-               
-                  (fast-list-iter [^ISpout spout spouts] (.nextTuple spout)))
-                (do
-                  (when @last-active
-                    (reset! last-active false)
-                    (log-message "Deactivating spout " component-id ":" (keys task-datas))
-                    (fast-list-iter [^ISpout spout spouts] (.deactivate spout)))
-                  ;; TODO: log that it's getting throttled
-                  (Time/sleep 100)
-                  (builtin-metrics/skipped-inactive! (:spout-throttling-metrics executor-data) (:stats executor-data)))))
+            (if active?
+              ; activated
+              (do
+                (when-not @last-active
+                  (reset! last-active true)
+                  (log-message "Activating spout " component-id ":" (keys task-datas))
+                  (fast-list-iter [^ISpout spout spouts] (.activate spout)))
+
+                (if (and (not (.isFull transfer-queue))
+                      (not throttle-on)
+                      (not reached-max-spout-pending))
+                  (fast-list-iter [^ISpout spout spouts] (.nextTuple spout))))
+              ; deactivated
+              (do
+                (when @last-active
+                  (reset! last-active false)
+                  (log-message "Deactivating spout " component-id ":" (keys task-datas))
+                  (fast-list-iter [^ISpout spout spouts] (.deactivate spout)))
+                ;; TODO: log that it's getting throttled
+                (Time/sleep 100)
+                (builtin-metrics/skipped-inactive! (:spout-throttling-metrics executor-data) (:stats executor-data))))
+
             (if (and (= curr-count (.get emitted-count)) active?)
               (do (.increment empty-emit-streak)
                   (.emptyEmit spout-wait-strategy (.get empty-emit-streak))
@@ -682,11 +650,11 @@
                     (if reached-max-spout-pending
                       (builtin-metrics/skipped-max-spout! (:spout-throttling-metrics executor-data) (:stats executor-data)))))
               (.set empty-emit-streak 0)
-              ))           
+              ))
           0))
       :kill-fn (:report-error-and-die executor-data)
       :factory? true
-      :thread-name component-id)]))
+      :thread-name (str component-id "-executor" (:executor-id executor-data)))]))
 
 (defn- tuple-time-delta! [^TupleImpl tuple]
   (let [ms (.getProcessSampleStartTime tuple)]
@@ -710,14 +678,6 @@
                 open-or-prepare-was-called?]} executor-data
         rand (Random. (Utils/secureRandomLong))
 
-        ;; the overflow buffer is used to ensure that bolts do not block when emitting
-        ;; this ensures that the bolt can always clear the incoming messages, which
-        ;; prevents deadlock from occurs across the topology
-        ;; (e.g. Spout -> BoltA -> Splitter -> BoltB -> BoltA, and all
-        ;; buffers filled up)
-        ;; the overflow buffer is might gradually fill degrading the performance gradually
-        ;; eventually running out of memory, but at least prevent live-locks/deadlocks.
-        overflow-buffer (if (storm-conf TOPOLOGY-BOLTS-OUTGOING-OVERFLOW-BUFFER-ENABLE) (ConcurrentLinkedQueue.) nil)
         tuple-action-fn (fn [task-id ^TupleImpl tuple]
                           ;; synchronization needs to be done with a key provided by this bolt, otherwise:
                           ;; spout 1 sends synchronization (s1), dies, same spout restarts somewhere else, sends synchronization (s2) and incremental update. s2 and update finish before s1 -> lose the incremental update
@@ -742,7 +702,7 @@
                                       bolt-obj (:object task-data)]
                                   (when (instance? ICredentialsListener bolt-obj)
                                     (.setCredentials bolt-obj (.getValue tuple 0))))
-                              Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple overflow-buffer)
+                              Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple)
                               (let [task-data (get task-datas task-id)
                                     ^IBolt bolt-obj (:object task-data)
                                     user-context (:user-context task-data)
@@ -761,11 +721,6 @@
  
                                   (task/apply-hooks user-context .boltExecute (BoltExecuteInfo. tuple task-id delta))
                                   (when delta
-                                    (builtin-metrics/bolt-execute-tuple! (:builtin-metrics task-data)
-                                                                         executor-stats
-                                                                         (.getSourceComponent tuple)                                                      
-                                                                         (.getSourceStreamId tuple)
-                                                                         delta)
                                     (stats/bolt-execute-tuple! executor-stats
                                                                (.getSourceComponent tuple)
                                                                (.getSourceStreamId tuple)
@@ -799,15 +754,14 @@
                                                                             (fast-list-iter [root-id root-ids]
                                                                                             (put-xor! anchors-to-ids root-id edge-id))
                                                                             ))))
-                                                        (transfer-fn t
-                                                                   (TupleImpl. worker-context
+                                                        (let [tuple (TupleImpl. worker-context
                                                                                values
                                                                                task-id
                                                                                stream
-                                                                               (MessageId/makeId anchors-to-ids))
-                                                                   overflow-buffer)))
+                                                                               (MessageId/makeId anchors-to-ids))]
+                                                          (transfer-fn t tuple))))
                                     (if has-eventloggers?
-                                      (send-to-eventlogger executor-data task-data values overflow-buffer component-id nil rand))
+                                      (send-to-eventlogger executor-data task-data values component-id nil rand))
                                     (or out-tasks [])))]]
           (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
           (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials)) 
@@ -839,19 +793,13 @@
                            (fast-map-iter [[root id] (.. tuple getMessageId getAnchorsToIds)]
                                           (task/send-unanchored task-data
                                                                 ACKER-ACK-STREAM-ID
-                                                                [root (bit-xor id ack-val)] overflow-buffer)
-                                          ))
+                                                                [root (bit-xor id ack-val)])))
                          (let [delta (tuple-time-delta! tuple)
                                debug? (= true (storm-conf TOPOLOGY-DEBUG))]
                            (when debug? 
                              (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when delta
-                             (builtin-metrics/bolt-acked-tuple! (:builtin-metrics task-data)
-                                                                executor-stats
-                                                                (.getSourceComponent tuple)                                                      
-                                                                (.getSourceStreamId tuple)
-                                                                delta)
                              (stats/bolt-acked-tuple! executor-stats
                                                       (.getSourceComponent tuple)
                                                       (.getSourceStreamId tuple)
@@ -860,17 +808,13 @@
                          (fast-list-iter [root (.. tuple getMessageId getAnchors)]
                                          (task/send-unanchored task-data
                                                                ACKER-FAIL-STREAM-ID
-                                                               [root] overflow-buffer))
+                                                               [root]))
                          (let [delta (tuple-time-delta! tuple)
                                debug? (= true (storm-conf TOPOLOGY-DEBUG))]
                            (when debug? 
                              (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when delta
-                             (builtin-metrics/bolt-failed-tuple! (:builtin-metrics task-data)
-                                                                 executor-stats
-                                                                 (.getSourceComponent tuple)                                                      
-                                                                 (.getSourceStreamId tuple))
                              (stats/bolt-failed-tuple! executor-stats
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)
@@ -884,23 +828,12 @@
 
         (let [receive-queue (:receive-queue executor-data)
               event-handler (mk-task-receiver executor-data tuple-action-fn)]
-          (disruptor/consumer-started! receive-queue)
           (fn []            
             (disruptor/consume-batch-when-available receive-queue event-handler)
-            ;; try to clear the overflow-buffer
-            (try-cause
-              (while (and overflow-buffer (not (.isEmpty overflow-buffer)))
-                (let [[out-task out-tuple] (.peek overflow-buffer)]
-                  (transfer-fn out-task out-tuple false nil)
-                  (.poll overflow-buffer)))
-              (catch InsufficientCapacityException e
-                (when (= true (storm-conf TOPOLOGY-DEBUG))
-                  (log-message "Insufficient Capacity on queue to emit by bolt " component-id ":" (keys task-datas) ))
-                ))
             0)))
       :kill-fn (:report-error-and-die executor-data)
       :factory? true
-      :thread-name component-id)]))
+      :thread-name (str component-id "-executor" (:executor-id executor-data)))]))
 
 (defmethod close-component :spout [executor-data spout]
   (.close spout))
diff --git a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
index 2a8a40c..80ee01a 100644
--- a/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/logviewer.clj
@@ -16,48 +16,74 @@
 (ns backtype.storm.daemon.logviewer
   (:use compojure.core)
   (:use [clojure.set :only [difference intersection]])
-  (:use [clojure.string :only [blank?]])
-  (:use [hiccup core page-helpers])
+  (:use [clojure.string :only [blank? split]])
+  (:use [hiccup core page-helpers form-helpers])
   (:use [backtype.storm config util log timer])
   (:use [backtype.storm.ui helpers])
+  (:import [backtype.storm.utils Utils])
   (:import [org.slf4j LoggerFactory])
-  (:import [java.io File FileFilter FileInputStream])
+  (:import [java.util Arrays])
+  (:import [java.util.zip GZIPInputStream])
   (:import [org.apache.logging.log4j LogManager])
   (:import [org.apache.logging.log4j.core Appender LoggerContext])
   (:import [org.apache.logging.log4j.core.appender RollingFileAppender])
+  (:import [java.io BufferedInputStream File FileFilter FileInputStream
+            InputStream InputStreamReader])
+  (:import [java.nio ByteBuffer])
   (:import [org.yaml.snakeyaml Yaml]
            [org.yaml.snakeyaml.constructor SafeConstructor])
   (:import [backtype.storm.ui InvalidRequestException]
            [backtype.storm.security.auth AuthUtils])
+  (:require [backtype.storm.daemon common [supervisor :as supervisor]])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]
             [ring.middleware.keyword-params]
-            [ring.util.response :as resp])
-  (:require [backtype.storm.daemon common [supervisor :as supervisor]])
-  (:import [java.io File FileFilter])
-  (:require [compojure.route :as route]
-            [compojure.handler :as handler]
+            [ring.util.codec :as codec]
             [ring.util.response :as resp]
             [clojure.string :as string])
+  (:require [metrics.meters :refer [defmeter mark!]])
+  (:use [backtype.storm.daemon.common :only [start-metrics-reporters]])
   (:gen-class))
 
 (def ^:dynamic *STORM-CONF* (read-storm-config))
 
+(defmeter logviewer:num-log-page-http-requests)
+(defmeter logviewer:num-daemonlog-page-http-requests)
+(defmeter logviewer:num-download-log-file-http-requests)
+(defmeter logviewer:num-download-log-daemon-file-http-requests)
+(defmeter logviewer:num-list-logs-http-requests)
+
 (defn cleanup-cutoff-age-millis [conf now-millis]
   (- now-millis (* (conf LOGVIEWER-CLEANUP-AGE-MINS) 60 1000)))
 
-;TODO: handle cleanup of old event log files
+(defn- last-modifiedtime-worker-logdir
+  "Return the last modified time for all log files in a worker's log dir"
+  [log-dir]
+  (apply max
+         (.lastModified log-dir)
+         (for [^File file (.listFiles log-dir)]
+           (.lastModified file))))
+
 (defn mk-FileFilter-for-log-cleanup [conf now-millis]
   (let [cutoff-age-millis (cleanup-cutoff-age-millis conf now-millis)]
     (reify FileFilter (^boolean accept [this ^File file]
                         (boolean (and
-                          (.isFile file)
-                          (re-find worker-log-filename-pattern (.getName file))
-                          (<= (.lastModified file) cutoff-age-millis)))))))
+                                   (not (.isFile file))
+                                   (<= (last-modifiedtime-worker-logdir file) cutoff-age-millis)))))))
 
-(defn select-files-for-cleanup [conf now-millis root-dir]
+(defn select-dirs-for-cleanup [conf now-millis root-dir]
   (let [file-filter (mk-FileFilter-for-log-cleanup conf now-millis)]
-    (.listFiles (File. root-dir) file-filter)))
+    (reduce clojure.set/union
+            (sorted-set)
+            (for [^File topo-dir (.listFiles (File. root-dir))]
+              (into [] (.listFiles topo-dir file-filter))))))
+
+(defn get-topo-port-workerlog
+  "Return the path of the worker log with the format of topoId/port/worker.log.*"
+  [^File file]
+  (clojure.string/join file-path-separator
+                       (take-last 3
+                                  (split (.getCanonicalPath file) (re-pattern file-path-separator)))))
 
 (defn get-metadata-file-for-log-root-name [root-name root-dir]
   (let [metaFile (clojure.java.io/file root-dir "metadata"
@@ -69,116 +95,202 @@
                   " to clean up for " root-name)
         nil))))
 
+(defn get-metadata-file-for-wroker-logdir [logdir]
+  (let [metaFile (clojure.java.io/file logdir "worker.yaml")]
+    (if (.exists metaFile)
+      metaFile
+      (do
+        (log-warn "Could not find " (.getCanonicalPath metaFile)
+                  " to clean up for " logdir)
+        nil))))
+
 (defn get-worker-id-from-metadata-file [metaFile]
   (get (clojure-from-yaml-file metaFile) "worker-id"))
 
 (defn get-topo-owner-from-metadata-file [metaFile]
   (get (clojure-from-yaml-file metaFile) TOPOLOGY-SUBMITTER-USER))
 
-(defn get-log-root->files-map [log-files]
-  "Returns a map of \"root name\" to a the set of files in log-files having the
-  root name.  The \"root name\" of a log file is the part of the name preceding
-  the extension."
-  (reduce #(assoc %1                                      ;; The accumulated map so far
-                  (first %2)                              ;; key: The root name of the log file
-                  (conj (%1 (first %2) #{}) (second %2))) ;; val: The set of log files with the root name
-          {}                                              ;; initial (empty) map
-          (map #(list
-                  (second (re-find worker-log-filename-pattern (.getName %))) ;; The root name of the log file
-                  %)                                                          ;; The log file
-               log-files)))
-
-(defn identify-worker-log-files [log-files root-dir]
-  (into {} (for [log-root-entry (get-log-root->files-map log-files)
-                 :let [metaFile (get-metadata-file-for-log-root-name
-                                  (key log-root-entry) root-dir)
-                       log-root (key log-root-entry)
-                       files (val log-root-entry)]
+(defn identify-worker-log-dirs [log-dirs]
+  "return the workerid to worker-log-dir map"
+  (into {} (for [logdir log-dirs
+                 :let [metaFile (get-metadata-file-for-wroker-logdir logdir)]
                  :when metaFile]
-             {(get-worker-id-from-metadata-file metaFile)
-              {:owner (get-topo-owner-from-metadata-file metaFile)
-               :files
-                 ;; If each log for this root name is to be deleted, then
-                 ;; include the metadata file also.
-                 (if (empty? (difference
-                                  (set (filter #(re-find (re-pattern log-root) %)
-                                               (read-dir-contents root-dir)))
-                                  (set (map #(.getName %) files))))
-                  (conj files metaFile)
-                  ;; Otherwise, keep the list of files as it is.
-                  files)}})))
+             {(get-worker-id-from-metadata-file metaFile) logdir})))
 
-(defn get-dead-worker-files-and-owners [conf now-secs log-files root-dir]
-  (if (empty? log-files)
-    {}
-    (let [id->heartbeat (supervisor/read-worker-heartbeats conf)
-          alive-ids (keys (remove
-                            #(or (not (val %))
-                                 (supervisor/is-worker-hb-timed-out? now-secs (val %) conf))
-                            id->heartbeat))
-          id->entries (identify-worker-log-files log-files root-dir)]
-      (for [[id {:keys [owner files]}] id->entries
-            :when (not (contains? (set alive-ids) id))]
-        {:owner owner
-         :files files}))))
+(defn get-alive-ids
+  [conf now-secs]
+  (->>
+    (supervisor/read-worker-heartbeats conf)
+    (remove
+      #(or (not (val %))
+           (supervisor/is-worker-hb-timed-out? now-secs
+                                               (val %)
+                                               conf)))
+    keys
+    set))
 
-(defn cleanup-fn! [log-root-dir]
+(defn get-dead-worker-dirs
+  "Return a sorted set of java.io.Files that were written by workers that are
+  now dead"
+  [conf now-secs log-dirs]
+  (if (empty? log-dirs)
+    (sorted-set)
+    (let [alive-ids (get-alive-ids conf now-secs)
+          id->dir (identify-worker-log-dirs log-dirs)]
+      (apply sorted-set
+             (for [[id dir] id->dir
+                   :when (not (contains? alive-ids id))]
+               dir)))))
+
+(defn get-all-worker-dirs [^File root-dir]
+  (reduce clojure.set/union
+          (sorted-set)
+          (for [^File topo-dir (.listFiles root-dir)]
+            (into [] (.listFiles topo-dir)))))
+
+(defn get-alive-worker-dirs
+  "Return a sorted set of java.io.Files that were written by workers that are
+  now active"
+  [conf root-dir]
+  (let [alive-ids (get-alive-ids conf (current-time-secs))
+        log-dirs (get-all-worker-dirs root-dir)
+        id->dir (identify-worker-log-dirs log-dirs)]
+    (apply sorted-set
+           (for [[id dir] id->dir
+                 :when (contains? alive-ids id)]
+             (.getCanonicalPath dir)))))
+
+(defn get-all-logs-for-rootdir [^File log-dir]
+  (reduce concat
+          (for [port-dir (get-all-worker-dirs log-dir)]
+            (into [] (.listFiles port-dir)))))
+
+(defn is-active-log [^File file]
+  (re-find #"\.(log|err|out|current|yaml|pid)$" (.getName file)))
+
+(defn filter-candidate-files
+  "Filter candidate files for global cleanup"
+  [logs log-dir]
+  (let [alive-worker-dirs (get-alive-worker-dirs *STORM-CONF* log-dir)]
+    (filter #(and (not= (.getName %) "worker.yaml")  ; exclude metadata file
+                  (not (and (contains? alive-worker-dirs (.getCanonicalPath (.getParentFile %)))
+                            (is-active-log %)))) ; exclude active workers' active logs
+            logs)))
+
+(defn sorted-worker-logs
+  "Collect the wroker log files recursively, sorted by decreasing age."
+  [^File root-dir]
+  (let [files (get-all-logs-for-rootdir root-dir)
+        logs (filter-candidate-files files root-dir)]
+    (sort-by #(.lastModified %) logs)))
+
+(defn sum-file-size
+  "Given a sequence of Files, sum their sizes."
+  [files]
+  (reduce #(+ %1 (.length %2)) 0 files))
+
+(defn delete-oldest-while-logs-too-large [logs_ size]
+  (loop [logs logs_]
+    (if (> (sum-file-size logs) size)
+      (do
+        (log-message "Log sizes too high. Going to delete: " (.getName (first logs)))
+        (try (rmr (.getCanonicalPath (first logs)))
+             (catch Exception ex (log-error ex)))
+        (recur (rest logs)))
+      logs)))
+
+(defn per-workerdir-cleanup
+  "Delete the oldest files in each overloaded worker log dir"
+  [^File root-dir size]
+  (dofor [worker-dir (get-all-worker-dirs root-dir)]
+    (let [filtered-logs (filter #(not (is-active-log %)) (.listFiles worker-dir))
+          sorted-logs (sort-by #(.lastModified %) filtered-logs)]
+      (delete-oldest-while-logs-too-large sorted-logs size))))
+
+(defn cleanup-empty-topodir
+  "Delete the topo dir if it contains zero port dirs"
+  [^File dir]
+  (let [topodir (.getParentFile dir)]
+    (if (empty? (.listFiles topodir))
+      (rmr (.getCanonicalPath topodir)))))
+
+(defn cleanup-fn!
+  "Delete old log dirs for which the workers are no longer alive"
+  [log-root-dir]
   (let [now-secs (current-time-secs)
-        old-log-files (select-files-for-cleanup *STORM-CONF* (* now-secs 1000) log-root-dir)
-        dead-worker-files (get-dead-worker-files-and-owners *STORM-CONF* now-secs old-log-files log-root-dir)]
+        old-log-dirs (select-dirs-for-cleanup *STORM-CONF*
+                                              (* now-secs 1000)
+                                              log-root-dir)
+        total-size (*STORM-CONF* LOGVIEWER-MAX-SUM-WORKER-LOGS-SIZE-MB)
+        per-dir-size (*STORM-CONF* LOGVIEWER-MAX-PER-WORKER-LOGS-SIZE-MB)
+        per-dir-size (min per-dir-size (* total-size 0.5))
+        dead-worker-dirs (get-dead-worker-dirs *STORM-CONF*
+                                               now-secs
+                                               old-log-dirs)]
     (log-debug "log cleanup: now=" now-secs
-               " old log files " (pr-str (map #(.getName %) old-log-files))
-               " dead worker files " (->> dead-worker-files
-                                          (mapcat (fn [{l :files}] l))
-                                          (map #(.getName %))
-                                          (pr-str)))
-    (dofor [{:keys [owner files]} dead-worker-files
-            file files]
-      (let [path (.getCanonicalPath file)]
-        (log-message "Cleaning up: Removing " path)
-        (try
-          (if (or (blank? owner) (re-matches #".*\.yaml$" path))
-            (rmr path)
-            ;; worker-launcher does not actually launch a worker process.  It
-            ;; merely executes one of a prescribed set of commands.  In this case, we ask it
-            ;; to delete a file as the owner of that file.
-            (supervisor/worker-launcher *STORM-CONF* owner (str "rmr " path)))
-          (catch Exception ex
-            (log-error ex)))))))
+               " old log dirs " (pr-str (map #(.getName %) old-log-dirs))
+               " dead worker dirs " (pr-str
+                                       (map #(.getName %) dead-worker-dirs)))
+    (dofor [dir dead-worker-dirs]
+           (let [path (.getCanonicalPath dir)]
+             (log-message "Cleaning up: Removing " path)
+             (try (rmr path)
+                  (cleanup-empty-topodir dir)
+                  (catch Exception ex (log-error ex)))))
+    (per-workerdir-cleanup (File. log-root-dir) (* per-dir-size (* 1024 1024)))
+    (let [all-logs (sorted-worker-logs (File. log-root-dir))
+          size (* total-size (*  1024 1024))]
+      (delete-oldest-while-logs-too-large all-logs size))))
 
 (defn start-log-cleaner! [conf log-root-dir]
   (let [interval-secs (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)]
     (when interval-secs
       (log-debug "starting log cleanup thread at interval: " interval-secs)
-      (schedule-recurring (mk-timer :thread-name "logviewer-cleanup")
+      (schedule-recurring (mk-timer :thread-name "logviewer-cleanup"
+                                    :kill-fn (fn [t]
+                                               (log-error t "Error when doing logs cleanup")
+                                               (exit-process! 20 "Error when doing log cleanup")))
                           0 ;; Start immediately.
                           interval-secs
                           (fn [] (cleanup-fn! log-root-dir))))))
 
+(defn- skip-bytes
+  "FileInputStream#skip may not work the first time, so ensure it successfully
+  skips the given number of bytes."
+  [^InputStream stream n]
+  (loop [skipped 0]
+    (let [skipped (+ skipped (.skip stream (- n skipped)))]
+      (if (< skipped n) (recur skipped)))))
+
+(defn logfile-matches-filter?
+  [log-file-name]
+  (let [regex-string (str "worker.log.*")
+        regex-pattern (re-pattern regex-string)]
+    (not= (re-seq regex-pattern (.toString log-file-name)) nil)))
+
 (defn page-file
   ([path tail]
-    (let [flen (.length (clojure.java.io/file path))
+    (let [zip-file? (.endsWith path ".gz")
+          flen (if zip-file? (Utils/zipFileSize (clojure.java.io/file path)) (.length (clojure.java.io/file path)))
           skip (- flen tail)]
       (page-file path skip tail)))
   ([path start length]
-    (with-open [input (FileInputStream. path)
-                output (java.io.ByteArrayOutputStream.)]
-      (if (>= start (.length (clojure.java.io/file path)))
-        (throw
-          (InvalidRequestException. "Cannot start past the end of the file")))
-      (if (> start 0)
-        ;; FileInputStream#skip may not work the first time.
-        (loop [skipped 0]
-          (let [skipped (+ skipped (.skip input (- start skipped)))]
-            (if (< skipped start) (recur skipped)))))
-      (let [buffer (make-array Byte/TYPE 1024)]
-        (loop []
-          (when (< (.size output) length)
-            (let [size (.read input buffer 0 (min 1024 (- length (.size output))))]
-              (when (pos? size)
-                (.write output buffer 0 size)
-                (recur)))))
-      (.toString output)))))
+    (let [zip-file? (.endsWith path ".gz")
+          flen (if zip-file? (Utils/zipFileSize (clojure.java.io/file path)) (.length (clojure.java.io/file path)))]
+      (with-open [input (if zip-file? (GZIPInputStream. (FileInputStream. path)) (FileInputStream. path))
+                  output (java.io.ByteArrayOutputStream.)]
+        (if (>= start flen)
+          (throw
+            (InvalidRequestException. "Cannot start past the end of the file")))
+        (if (> start 0) (skip-bytes input start))
+        (let [buffer (make-array Byte/TYPE 1024)]
+          (loop []
+            (when (< (.size output) length)
+              (let [size (.read input buffer 0 (min 1024 (- length (.size output))))]
+                (when (pos? size)
+                  (.write output buffer 0 size)
+                  (recur)))))
+        (.toString output))))))
 
 (defn get-log-user-group-whitelist [fname]
   (let [wl-file (get-log-metadata-file fname)
@@ -212,14 +324,13 @@
 
 (defn log-root-dir
   "Given an appender name, as configured, get the parent directory of the appender's log file.
-
-Note that if anything goes wrong, this will throw an Error and exit."
+   Note that if anything goes wrong, this will throw an Error and exit."
   [appender-name]
   (let [appender (.getAppender (.getConfiguration (LogManager/getContext)) appender-name)]
     (if (and appender-name appender (instance? RollingFileAppender appender))
       (.getParent (File. (.getFileName appender)))
       (throw
-       (RuntimeException. "Log viewer could not find configured appender, or the appender is not a FileAppender. Please check that the appender name configured in storm and log4j2 agree.")))))
+       (RuntimeException. "Log viewer could not find configured appender, or the appender is not a FileAppender. Please check that the appender name configured in storm and log4j agree.")))))
 
 (defnk to-btn-link
   "Create a link that is formatted like a button"
@@ -227,6 +338,18 @@
   [:a {:href (java.net.URI. url)
        :class (str "btn btn-default " (if enabled "enabled" "disabled"))} text])
 
+(defn search-file-form [fname]
+  [[:form {:action "logviewer_search.html" :id "search-box"}
+    "Search this file:"
+    [:input {:type "text" :name "search"}]
+    [:input {:type "hidden" :name "file" :value fname}]
+    [:input {:type "submit" :value "Search"}]]])
+
+(defn log-file-selection-form [log-files type]
+  [[:form {:action type :id "list-of-files"}
+    (drop-down "file" log-files )
+    [:input {:type "submit" :value "Switch file"}]]])
+
 (defn pager-links [fname start length file-size]
   (let [prev-start (max 0 (- start length))
         next-start (if (> file-size 0)
@@ -255,25 +378,43 @@
                         "Next" :enabled (> next-start start))])]]))
 
 (defn- download-link [fname]
-  [[:p (link-to (url-format "/download/%s" fname) "Download Full Log")]])
+  [[:p (link-to (url-format "/download/%s" fname) "Download Full File")]])
+
+(defn- daemon-download-link [fname]
+  [[:p (link-to (url-format "/daemondownload/%s" fname) "Download Full File")]])
+
+(defn- is-txt-file [fname]
+  (re-find #"\.(log.*|txt|yaml|pid)$" fname))
+
+(def default-bytes-per-page 51200)
 
 (defn log-page [fname start length grep user root-dir]
   (if (or (blank? (*STORM-CONF* UI-FILTER))
           (authorized-log-user? user fname *STORM-CONF*))
     (let [file (.getCanonicalFile (File. root-dir fname))
-          file-length (.length file)
-          path (.getCanonicalPath file)]
-      (if (and (= (.getCanonicalFile (File. root-dir))
-                  (.getParentFile file))
-               (.exists file))
-        (let [default-length 51200
-              length (if length
+          path (.getCanonicalPath file)
+          zip-file? (.endsWith path ".gz")
+          topo-dir (.getParentFile (.getParentFile file))]
+      (if (and (.exists file)
+               (= (.getCanonicalFile (File. root-dir))
+                  (.getParentFile topo-dir)))
+        (let [file-length (if zip-file? (Utils/zipFileSize (clojure.java.io/file path)) (.length (clojure.java.io/file path)))
+              log-files (reduce clojure.set/union
+                          (sorted-set)
+                          (for [^File port-dir (.listFiles topo-dir)]
+                            (into [] (filter #(.isFile %) (.listFiles port-dir))))) ;all types of files included
+              files-str (for [file log-files]
+                          (get-topo-port-workerlog file))
+              reordered-files-str (conj (filter #(not= fname %) files-str) fname)
+               length (if length
                        (min 10485760 length)
-                     default-length)
+                       default-bytes-per-page)
               log-string (escape-html
-                           (if start
-                             (page-file path start length)
-                             (page-file path length)))
+                           (if (is-txt-file fname)
+                             (if start
+                               (page-file path start length)
+                               (page-file path length))
+                             "This is a binary file and cannot display! You may download the full file."))
               start (or start (- file-length length))]
           (if grep
             (html [:pre#logContent
@@ -282,8 +423,10 @@
                           (filter #(.contains % grep))
                           (string/join "\n"))
                      log-string)])
-            (let [pager-data (pager-links fname start length file-length)]
-              (html (concat pager-data
+            (let [pager-data (if (is-txt-file fname) (pager-links fname start length file-length) nil)]
+              (html (concat (search-file-form fname)
+                            (log-file-selection-form reordered-files-str "log") ; list all files for this topology
+                            pager-data
                             (download-link fname)
                             [[:pre#logContent log-string]]
                             pager-data)))))
@@ -294,9 +437,54 @@
         (resp/status 404))
       (unauthorized-user-html user))))
 
+(defn daemonlog-page [fname start length grep user root-dir]
+  (if (or (blank? (*STORM-CONF* UI-FILTER))
+        (authorized-log-user? user fname *STORM-CONF*))
+    (let [file (.getCanonicalFile (File. root-dir fname))
+          file-length (.length file)
+          path (.getCanonicalPath file)
+          zip-file? (.endsWith path ".gz")]
+      (if (and (= (.getCanonicalFile (File. root-dir))
+                 (.getParentFile file))
+            (.exists file))
+        (let [file-length (if zip-file? (Utils/zipFileSize (clojure.java.io/file path)) (.length (clojure.java.io/file path)))
+              length (if length
+                       (min 10485760 length)
+                       default-bytes-per-page)
+              log-files (into [] (filter #(.isFile %) (.listFiles (File. root-dir)))) ;all types of files included
+              files-str (for [file log-files]
+                          (.getName file))
+              reordered-files-str (conj (filter #(not= fname %) files-str) fname)
+              log-string (escape-html
+                           (if (is-txt-file fname)
+                             (if start
+                               (page-file path start length)
+                               (page-file path length))
+                             "This is a binary file and cannot display! You may download the full file."))
+              start (or start (- file-length length))]
+          (if grep
+            (html [:pre#logContent
+                   (if grep
+                     (->> (.split log-string "\n")
+                       (filter #(.contains % grep))
+                       (string/join "\n"))
+                     log-string)])
+            (let [pager-data (if (is-txt-file fname) (pager-links fname start length file-length) nil)]
+              (html (concat (log-file-selection-form reordered-files-str "daemonlog") ; list all daemon logs
+                      pager-data
+                      (daemon-download-link fname)
+                      [[:pre#logContent log-string]]
+                      pager-data)))))
+        (-> (resp/response "Page not found")
+          (resp/status 404))))
+    (if (nil? (get-log-user-group-whitelist fname))
+      (-> (resp/response "Page not found")
+        (resp/status 404))
+      (unauthorized-user-html user))))
+
 (defn download-log-file [fname req resp user ^String root-dir]
   (let [file (.getCanonicalFile (File. root-dir fname))]
-    (if (= (File. root-dir) (.getParentFile file))
+    (if (.exists file)
       (if (or (blank? (*STORM-CONF* UI-FILTER))
               (authorized-log-user? user fname *STORM-CONF*))
         (-> (resp/response file)
@@ -305,6 +493,405 @@
       (-> (resp/response "Page not found")
           (resp/status 404)))))
 
+(def grep-max-search-size 1024)
+(def grep-buf-size 2048)
+(def grep-context-size 128)
+
+(defn logviewer-port
+  []
+  (int (*STORM-CONF* LOGVIEWER-PORT)))
+
+(defn url-to-match-centered-in-log-page
+  [needle fname offset port]
+  (let [host (local-hostname)
+        port (logviewer-port)
+        fname (clojure.string/join file-path-separator (take-last 3 (split fname (re-pattern file-path-separator))))]
+    (url (str "http://" host ":" port "/log")
+      {:file fname
+       :start (max 0
+                (- offset
+                  (int (/ default-bytes-per-page 2))
+                  (int (/ (alength needle) -2)))) ;; Addition
+       :length default-bytes-per-page})))
+
+(defnk mk-match-data
+  [^bytes needle ^ByteBuffer haystack haystack-offset file-offset fname
+   :before-bytes nil :after-bytes nil]
+  (let [url (url-to-match-centered-in-log-page needle
+              fname
+              file-offset
+              (*STORM-CONF* LOGVIEWER-PORT))
+        haystack-bytes (.array haystack)
+        before-string (if (>= haystack-offset grep-context-size)
+                        (String. haystack-bytes
+                          (- haystack-offset grep-context-size)
+                          grep-context-size
+                          "UTF-8")
+                        (let [num-desired (max 0 (- grep-context-size
+                                                   haystack-offset))
+                              before-size (if before-bytes
+                                            (alength before-bytes)
+                                            0)
+                              num-expected (min before-size num-desired)]
+                          (if (pos? num-expected)
+                            (str (String. before-bytes
+                                   (- before-size num-expected)
+                                   num-expected
+                                   "UTF-8")
+                              (String. haystack-bytes
+                                0
+                                haystack-offset
+                                "UTF-8"))
+                            (String. haystack-bytes
+                              0
+                              haystack-offset
+                              "UTF-8"))))
+        after-string (let [needle-size (alength needle)
+                           after-offset (+ haystack-offset needle-size)
+                           haystack-size (.limit haystack)]
+                       (if (< (+ after-offset grep-context-size) haystack-size)
+                         (String. haystack-bytes
+                           after-offset
+                           grep-context-size
+                           "UTF-8")
+                         (let [num-desired (- grep-context-size
+                                             (- haystack-size after-offset))
+                               after-size (if after-bytes
+                                            (alength after-bytes)
+                                            0)
+                               num-expected (min after-size num-desired)]
+                           (if (pos? num-expected)
+                             (str (String. haystack-bytes
+                                    after-offset
+                                    (- haystack-size after-offset)
+                                    "UTF-8")
+                               (String. after-bytes 0 num-expected "UTF-8"))
+                             (String. haystack-bytes
+                               after-offset
+                               (- haystack-size after-offset)
+                               "UTF-8")))))]
+    {"byteOffset" file-offset
+     "beforeString" before-string
+     "afterString" after-string
+     "matchString" (String. needle "UTF-8")
+     "logviewerURL" url}))
+
+(defn- try-read-ahead!
+  "Tries once to read ahead in the stream to fill the context and resets the
+  stream to its position before the call."
+  [^BufferedInputStream stream haystack offset file-len bytes-read]
+  (let [num-expected (min (- file-len bytes-read)
+                       grep-context-size)
+        after-bytes (byte-array num-expected)]
+    (.mark stream num-expected)
+    ;; Only try reading once.
+    (.read stream after-bytes 0 num-expected)
+    (.reset stream)
+    after-bytes))
+
+(defn offset-of-bytes
+  "Searches a given byte array for a match of a sub-array of bytes.  Returns
+  the offset to the byte that matches, or -1 if no match was found."
+  [^bytes buf ^bytes value init-offset]
+  {:pre [(> (alength value) 0)
+         (not (neg? init-offset))]}
+  (loop [offset init-offset
+         candidate-offset init-offset
+         val-offset 0]
+    (if-not (pos? (- (alength value) val-offset))
+      ;; Found
+      candidate-offset
+      (if (>= offset (alength buf))
+        ;; We ran out of buffer for the search.
+        -1
+        (if (not= (aget value val-offset) (aget buf offset))
+          ;; The match at this candidate offset failed, so start over with the
+          ;; next candidate byte from the buffer.
+          (let [new-offset (inc candidate-offset)]
+            (recur new-offset new-offset 0))
+          ;; So far it matches.  Keep going...
+          (recur (inc offset) candidate-offset (inc val-offset)))))))
+
+(defn- buffer-substring-search!
+  "As the file is read into a buffer, 1/2 the buffer's size at a time, we
+  search the buffer for matches of the substring and return a list of zero or
+  more matches."
+  [file file-len offset-to-buf init-buf-offset stream bytes-skipped
+   bytes-read ^ByteBuffer haystack ^bytes needle initial-matches num-matches
+   ^bytes before-bytes]
+  (loop [buf-offset init-buf-offset
+         matches initial-matches]
+    (let [offset (offset-of-bytes (.array haystack) needle buf-offset)]
+      (if (and (< (count matches) num-matches) (not (neg? offset)))
+        (let [file-offset (+ offset-to-buf offset)
+              bytes-needed-after-match (- (.limit haystack)
+                                         grep-context-size
+                                         (alength needle))
+              before-arg (if (< offset grep-context-size) before-bytes)
+              after-arg (if (> offset bytes-needed-after-match)
+                          (try-read-ahead! stream
+                            haystack
+                            offset
+                            file-len
+                            bytes-read))]
+          (recur (+ offset (alength needle))
+            (conj matches
+              (mk-match-data needle
+                haystack
+                offset
+                file-offset
+                (.getCanonicalPath file)
+                :before-bytes before-arg
+                :after-bytes after-arg))))
+        (let [before-str-to-offset (min (.limit haystack)
+                                     grep-max-search-size)
+              before-str-from-offset (max 0 (- before-str-to-offset
+                                              grep-context-size))
+              new-before-bytes (Arrays/copyOfRange (.array haystack)
+                                 before-str-from-offset
+                                 before-str-to-offset)
+              ;; It's OK if new-byte-offset is negative.  This is normal if
+              ;; we are out of bytes to read from a small file.
+              new-byte-offset (if (>= (count matches) num-matches)
+                                (+ (get (last matches) "byteOffset")
+                                  (alength needle))
+                                (+ bytes-skipped
+                                  bytes-read
+                                  (- grep-max-search-size)))]
+          [matches new-byte-offset new-before-bytes])))))
+
+(defn- mk-grep-response
+  "This response data only includes a next byte offset if there is more of the
+  file to read."
+  [search-bytes offset matches next-byte-offset]
+  (merge {"searchString" (String. search-bytes "UTF-8")
+          "startByteOffset" offset
+          "matches" matches}
+    (and next-byte-offset {"nextByteOffset" next-byte-offset})))
+
+(defn rotate-grep-buffer!
+  [^ByteBuffer buf ^BufferedInputStream stream total-bytes-read file file-len]
+  (let [buf-arr (.array buf)]
+    ;; Copy the 2nd half of the buffer to the first half.
+    (System/arraycopy buf-arr
+      grep-max-search-size
+      buf-arr
+      0
+      grep-max-search-size)
+    ;; Zero-out the 2nd half to prevent accidental matches.
+    (Arrays/fill buf-arr
+      grep-max-search-size
+      (count buf-arr)
+      (byte 0))
+    ;; Fill the 2nd half with new bytes from the stream.
+    (let [bytes-read (.read stream
+                       buf-arr
+                       grep-max-search-size
+                       (min file-len grep-max-search-size))]
+      (.limit buf (+ grep-max-search-size bytes-read))
+      (swap! total-bytes-read + bytes-read))))
+
+(defnk substring-search
+  "Searches for a substring in a log file, starting at the given offset,
+  returning the given number of matches, surrounded by the given number of
+  context lines.  Other information is included to be useful for progressively
+  searching through a file for display in a UI. The search string must
+  grep-max-search-size bytes or fewer when decoded with UTF-8."
+  [file ^String search-string :num-matches 10 :start-byte-offset 0]
+  {:pre [(not (empty? search-string))
+         (<= (count (.getBytes search-string "UTF-8")) grep-max-search-size)]}
+  (let [zip-file? (.endsWith (.getName file) ".gz")
+        f-input-steam (FileInputStream. file)
+        gzipped-input-stream (if zip-file?
+                               (GZIPInputStream. f-input-steam)
+                               f-input-steam)
+        stream ^BufferedInputStream (BufferedInputStream.
+                                      gzipped-input-stream)
+        file-len (if zip-file? (Utils/zipFileSize file) (.length file))
+        buf ^ByteBuffer (ByteBuffer/allocate grep-buf-size)
+        buf-arr ^bytes (.array buf)
+        string nil
+        total-bytes-read (atom 0)
+        matches []
+        search-bytes ^bytes (.getBytes search-string "UTF-8")
+        num-matches (or num-matches 10)
+        start-byte-offset (or start-byte-offset 0)]
+    ;; Start at the part of the log file we are interested in.
+    ;; Allow searching when start-byte-offset == file-len so it doesn't blow up on 0-length files
+    (if (> start-byte-offset file-len)
+      (throw
+        (InvalidRequestException. "Cannot search past the end of the file")))
+    (when (> start-byte-offset 0)
+      (skip-bytes stream start-byte-offset))
+    (java.util.Arrays/fill buf-arr (byte 0))
+    (let [bytes-read (.read stream buf-arr 0 (min file-len grep-buf-size))]
+      (.limit buf bytes-read)
+      (swap! total-bytes-read + bytes-read))
+    (loop [initial-matches []
+           init-buf-offset 0
+           byte-offset start-byte-offset
+           before-bytes nil]
+      (let [[matches new-byte-offset new-before-bytes]
+            (buffer-substring-search! file
+              file-len
+              byte-offset
+              init-buf-offset
+              stream
+              start-byte-offset
+              @total-bytes-read
+              buf
+              search-bytes
+              initial-matches
+              num-matches
+              before-bytes)]
+        (if (and (< (count matches) num-matches)
+              (< (+ @total-bytes-read start-byte-offset) file-len))
+          (let [;; The start index is positioned to find any possible
+                ;; occurrence search string that did not quite fit in the
+                ;; buffer on the previous read.
+                new-buf-offset (- (min (.limit ^ByteBuffer buf)
+                                    grep-max-search-size)
+                                 (alength search-bytes))]
+            (rotate-grep-buffer! buf stream total-bytes-read file file-len)
+            (when (< @total-bytes-read 0)
+              (throw (InvalidRequestException. "Cannot search past the end of the file")))
+            (recur matches
+              new-buf-offset
+              new-byte-offset
+              new-before-bytes))
+          (mk-grep-response search-bytes
+            start-byte-offset
+            matches
+            (if-not (and (< (count matches) num-matches)
+                      (>= @total-bytes-read file-len))
+              (let [next-byte-offset (+ (get (last matches)
+                                          "byteOffset")
+                                       (alength search-bytes))]
+                (if (> file-len next-byte-offset)
+                  next-byte-offset)))))))))
+
+(defn- try-parse-int-param
+  [nam value]
+  (try
+    (Integer/parseInt value)
+    (catch java.lang.NumberFormatException e
+      (->
+        (str "Could not parse " nam " to an integer")
+        (InvalidRequestException. e)
+        throw))))
+
+(defn search-log-file
+  [fname user ^String root-dir search num-matches offset callback origin]
+  (let [file (.getCanonicalFile (File. root-dir fname))]
+    (if (.exists file)
+      (if (or (blank? (*STORM-CONF* UI-FILTER))
+            (authorized-log-user? user fname *STORM-CONF*))
+        (let [num-matches-int (if num-matches
+                                (try-parse-int-param "num-matches"
+                                  num-matches))
+              offset-int (if offset
+                           (try-parse-int-param "start-byte-offset" offset))]
+          (try
+            (if (and (not (empty? search))
+                  <= (count (.getBytes search "UTF-8")) grep-max-search-size)
+              (json-response
+                (substring-search file
+                  search
+                  :num-matches num-matches-int
+                  :start-byte-offset offset-int)
+                callback
+                :headers {"Access-Control-Allow-Origin" origin
+                          "Access-Control-Allow-Credentials" "true"})
+              (throw
+                (InvalidRequestException.
+                  (str "Search substring must be between 1 and 1024 UTF-8 "
+                    "bytes in size (inclusive)"))))
+            (catch Exception ex
+              (json-response (exception->json ex) callback :status 500))))
+        (json-response (unauthorized-user-json user) callback :status 401))
+      (json-response {"error" "Not Found"
+                      "errorMessage" "The file was not found on this node."}
+        callback
+        :status 404))))
+
+(defn find-n-matches [logs n file-offset offset search]
+  (let [logs (drop file-offset logs)
+        wrap-matches-fn (fn [matches]
+                          {"fileOffset" file-offset
+                           "searchString" search
+                           "matches" matches})]
+    (loop [matches []
+           logs logs
+           offset offset
+           file-offset file-offset
+           match-count 0]
+      (if (empty? logs)
+        (wrap-matches-fn matches)
+        (let [these-matches (try
+                              (log-debug "Looking through " (first logs))
+                              (substring-search (first logs)
+                                search
+                                :num-matches (- n match-count)
+                                :start-byte-offset offset)
+                              (catch InvalidRequestException e
+                                (log-error e "Can't search past end of file.")
+                                {}))
+              file-name (get-topo-port-workerlog (first logs))
+              new-matches (conj matches
+                            (merge these-matches
+                              { "fileName" file-name
+                                "port" (first (take-last 2 (split (.getCanonicalPath (first logs)) (re-pattern file-path-separator))))}))
+              new-count (+ match-count (count (these-matches "matches")))]
+          (if (empty? these-matches)
+            (recur matches (rest logs) 0 (+ file-offset 1) match-count)
+            (if (>= new-count n)
+              (wrap-matches-fn new-matches)
+              (recur new-matches (rest logs) 0 (+ file-offset 1) new-count))))))))
+
+(defn logs-for-port
+  "Get the filtered, authorized, sorted log files for a port."
+  [user port-dir]
+  (let [filter-authorized-fn (fn [user logs]
+                               (filter #(or
+                                          (blank? (*STORM-CONF* UI-FILTER))
+                                          (authorized-log-user? user (get-topo-port-workerlog %) *STORM-CONF*)) logs))]
+    (sort #(compare (.lastModified %2) (.lastModified %1))
+      (filter-authorized-fn
+        user
+        (filter #(re-find worker-log-filename-pattern (.getName %)) (.listFiles port-dir))))))
+
+(defn deep-search-logs-for-topology
+  [topology-id user ^String root-dir search num-matches port file-offset offset search-archived? callback origin]
+  (json-response
+    (if (or (not search) (not (.exists (File. (str root-dir file-path-separator topology-id)))))
+      []
+      (let [file-offset (if file-offset (Integer/parseInt file-offset) 0)
+            offset (if offset (Integer/parseInt offset) 0)
+            num-matches (or (Integer/parseInt num-matches) 1)
+            port-dirs (vec (.listFiles (File. (str root-dir file-path-separator topology-id))))
+            logs-for-port-fn (partial logs-for-port user)]
+        (if (or (not port) (= "*" port))
+          ;; Check for all ports
+          (let [filtered-logs (filter (comp not empty?) (map logs-for-port-fn port-dirs))]
+            (if search-archived?
+              (map #(find-n-matches % num-matches 0 0 search)
+                filtered-logs)
+              (map #(find-n-matches % num-matches 0 0 search)
+                (map (comp vector first) filtered-logs))))
+          ;; Check just the one port
+          (if (not (contains? (into #{} (map str (*STORM-CONF* SUPERVISOR-SLOTS-PORTS))) port))
+            []
+            (let [port-dir (File. (str root-dir file-path-separator topology-id file-path-separator port))]
+              (if (or (not (.exists port-dir)) (empty? (logs-for-port user port-dir)))
+                []
+                (let [filtered-logs (logs-for-port user port-dir)]
+                  (if search-archived?
+                    (find-n-matches filtered-logs num-matches file-offset offset search)
+                    (find-n-matches [(first filtered-logs)] num-matches 0 offset search)))))))))
+    callback
+    :headers {"Access-Control-Allow-Origin" origin
+              "Access-Control-Allow-Credentials" "true"}))
+
 (defn log-template
   ([body] (log-template body nil nil))
   ([body fname user]
@@ -333,42 +920,227 @@
                     (name k) "'")
                ex)))))
 
+(defn list-log-files
+  [user topoId port log-root callback origin]
+  (let [file-results
+        (if (nil? topoId)
+          (if (nil? port)
+            (get-all-logs-for-rootdir (File. log-root))
+            (reduce concat
+              (for [topo-dir (.listFiles (File. log-root))]
+                (reduce concat
+                  (for [port-dir (.listFiles topo-dir)]
+                    (if (= (str port) (.getName port-dir))
+                      (into [] (.listFiles port-dir))))))))
+          (if (nil? port)
+            (let [topo-dir (File. (str log-root file-path-separator topoId))]
+              (if (.exists topo-dir)
+                (reduce concat
+                  (for [port-dir (.listFiles topo-dir)]
+                    (into [] (.listFiles port-dir))))
+                []))
+            (let [port-dir (get-worker-dir-from-root log-root topoId port)]
+              (if (.exists port-dir)
+                (into [] (.listFiles port-dir))
+                []))))
+        file-strs (sort (for [file file-results]
+                          (get-topo-port-workerlog file)))]
+    (json-response file-strs
+      callback
+      :headers {"Access-Control-Allow-Origin" origin
+                "Access-Control-Allow-Credentials" "true"})))
+
+(defn get-profiler-dump-files
+  [dir]
+  (filter (comp not nil?)
+        (for [f (.listFiles dir)]
+          (let [name (.getName f)]
+            (if (or
+                  (.endsWith name ".txt")
+                  (.endsWith name ".jfr")
+                  (.endsWith name ".bin"))
+              (.getName f))))))
+
 (defroutes log-routes
   (GET "/log" [:as req & m]
-       (try
-         (let [servlet-request (:servlet-request req)
-               log-root (:log-root req)
-               user (.getUserName http-creds-handler servlet-request)
-               start (if (:start m) (parse-long-from-map m :start))
-               length (if (:length m) (parse-long-from-map m :length))]
-           (log-template (log-page (:file m) start length (:grep m) user log-root)
-                         (:file m) user))
-         (catch InvalidRequestException ex
-           (log-error ex)
-           (ring-response-from-exception ex))))
+    (try
+      (mark! logviewer:num-log-page-http-requests)
+      (let [servlet-request (:servlet-request req)
+            log-root (:log-root req)
+            user (.getUserName http-creds-handler servlet-request)
+            start (if (:start m) (parse-long-from-map m :start))
+            length (if (:length m) (parse-long-from-map m :length))
+            file (url-decode (:file m))]
+        (log-template (log-page file start length (:grep m) user log-root)
+          file user))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (ring-response-from-exception ex))))
+  (GET "/dumps/:topo-id/:host-port/:filename"
+       [:as {:keys [servlet-request servlet-response log-root]} topo-id host-port filename &m]
+     (let [user (.getUserName http-creds-handler servlet-request)
+           port (second (split host-port #":"))
+           dir (File. (str log-root
+                           file-path-separator
+                           topo-id
+                           file-path-separator
+                           port))
+           file (File. (str log-root
+                            file-path-separator
+                            topo-id
+                            file-path-separator
+                            port
+                            file-path-separator
+                            filename))]
+       (if (and (.exists dir) (.exists file))
+         (if (or (blank? (*STORM-CONF* UI-FILTER))
+               (authorized-log-user? user 
+                                     (str topo-id file-path-separator port file-path-separator "worker.log")
+                                     *STORM-CONF*))
+           (-> (resp/response file)
+               (resp/content-type "application/octet-stream"))
+           (unauthorized-user-html user))
+         (-> (resp/response "Page not found")
+           (resp/status 404)))))
+  (GET "/dumps/:topo-id/:host-port"
+       [:as {:keys [servlet-request servlet-response log-root]} topo-id host-port &m]
+     (let [user (.getUserName http-creds-handler servlet-request)
+           port (second (split host-port #":"))
+           dir (File. (str log-root
+                           file-path-separator
+                           topo-id
+                           file-path-separator
+                           port))]
+       (if (.exists dir)
+         (if (or (blank? (*STORM-CONF* UI-FILTER))
+               (authorized-log-user? user 
+                                     (str topo-id file-path-separator port file-path-separator "worker.log")
+                                     *STORM-CONF*))
+           (html4
+             [:head
+              [:title "File Dumps - Storm Log Viewer"]
+              (include-css "/css/bootstrap-3.3.1.min.css")
+              (include-css "/css/jquery.dataTables.1.10.4.min.css")
+              (include-css "/css/style.css")]
+             [:body
+              [:ul
+               (for [file (get-profiler-dump-files dir)]
+                 [:li
+                  [:a {:href (str "/dumps/" topo-id "/" host-port "/" file)} file ]])]])
+           (unauthorized-user-html user))
+         (-> (resp/response "Page not found")
+           (resp/status 404)))))
+  (GET "/daemonlog" [:as req & m]
+    (try
+      (mark! logviewer:num-daemonlog-page-http-requests)
+      (let [servlet-request (:servlet-request req)
+            daemonlog-root (:daemonlog-root req)
+            user (.getUserName http-creds-handler servlet-request)
+            start (if (:start m) (parse-long-from-map m :start))
+            length (if (:length m) (parse-long-from-map m :length))
+            file (url-decode (:file m))]
+        (log-template (daemonlog-page file start length (:grep m) user daemonlog-root)
+          file user))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (ring-response-from-exception ex))))
   (GET "/download/:file" [:as {:keys [servlet-request servlet-response log-root]} file & m]
-       (try
-         (let [user (.getUserName http-creds-handler servlet-request)]
-           (download-log-file file servlet-request servlet-response user log-root))
-         (catch InvalidRequestException ex
-           (log-error ex)
-           (ring-response-from-exception ex))))
+    (try
+      (mark! logviewer:num-download-log-file-http-requests)
+      (let [user (.getUserName http-creds-handler servlet-request)]
+        (download-log-file file servlet-request servlet-response user log-root))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (ring-response-from-exception ex))))
+  (GET "/daemondownload/:file" [:as {:keys [servlet-request servlet-response daemonlog-root]} file & m]
+    (try
+      (mark! logviewer:num-download-log-daemon-file-http-requests)
+      (let [user (.getUserName http-creds-handler servlet-request)]
+        (download-log-file file servlet-request servlet-response user daemonlog-root))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (ring-response-from-exception ex))))
+  (GET "/search/:file" [:as {:keys [servlet-request servlet-response log-root]} file & m]
+    ;; We do not use servlet-response here, but do not remove it from the
+    ;; :keys list, or this rule could stop working when an authentication
+    ;; filter is configured.
+    (try
+      (let [user (.getUserName http-creds-handler servlet-request)]
+        (search-log-file (url-decode file)
+          user
+          log-root
+          (:search-string m)
+          (:num-matches m)
+          (:start-byte-offset m)
+          (:callback m)
+          (.getHeader servlet-request "Origin")))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (json-response (exception->json ex) (:callback m) :status 400))))
+  (GET "/deepSearch/:topo-id" [:as {:keys [servlet-request servlet-response log-root]} topo-id & m]
+    ;; We do not use servlet-response here, but do not remove it from the
+    ;; :keys list, or this rule could stop working when an authentication
+    ;; filter is configured.
+    (try
+      (let [user (.getUserName http-creds-handler servlet-request)]
+        (deep-search-logs-for-topology topo-id
+          user
+          log-root
+          (:search-string m)
+          (:num-matches m)
+          (:port m)
+          (:start-file-offset m)
+          (:start-byte-offset m)
+          (:search-archived m)
+          (:callback m)
+          (.getHeader servlet-request "Origin")))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (json-response (exception->json ex) (:callback m) :status 400))))
+  (GET "/searchLogs" [:as req & m]
+    (try
+      (let [servlet-request (:servlet-request req)
+            user (.getUserName http-creds-handler servlet-request)]
+        (list-log-files user
+          (:topoId m)
+          (:port m)
+          (:log-root req)
+          (:callback m)
+          (.getHeader servlet-request "Origin")))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (json-response (exception->json ex) (:callback m) :status 400))))
+  (GET "/listLogs" [:as req & m]
+    (try
+      (mark! logviewer:num-list-logs-http-requests)
+      (let [servlet-request (:servlet-request req)
+            user (.getUserName http-creds-handler servlet-request)]
+        (list-log-files user
+          (:topoId m)
+          (:port m)
+          (:log-root req)
+          (:callback m)
+          (.getHeader servlet-request "Origin")))
+      (catch InvalidRequestException ex
+        (log-error ex)
+        (json-response (exception->json ex) (:callback m) :status 400))))
   (route/resources "/")
   (route/not-found "Page not found"))
 
 (defn conf-middleware
   "For passing the storm configuration with each request."
-  [app log-root]
+  [app log-root daemonlog-root]
   (fn [req]
-    (app (assoc req :log-root log-root))))
+    (app (assoc req :log-root log-root :daemonlog-root daemonlog-root))))
 
-(defn start-logviewer! [conf log-root-dir]
+(defn start-logviewer! [conf log-root-dir daemonlog-root-dir]
   (try
     (let [header-buffer-size (int (.get conf UI-HEADER-BUFFER-BYTES))
           filter-class (conf UI-FILTER)
           filter-params (conf UI-FILTER-PARAMS)
-          logapp (handler/api log-routes) ;; query params as map
-          middle (conf-middleware logapp log-root-dir)
+          logapp (handler/api (-> log-routes
+                                requests-middleware))  ;; query params as map
+          middle (conf-middleware logapp log-root-dir daemonlog-root-dir)
           filters-confs (if (conf UI-FILTER)
                           [{:filter-class filter-class
                             :filter-params (or (conf UI-FILTER-PARAMS) {})}]
@@ -406,7 +1178,9 @@
 
 (defn -main []
   (let [conf (read-storm-config)
-        log-root (log-root-dir (conf LOGVIEWER-APPENDER-NAME))]
+        log-root (worker-artifacts-root conf)
+        daemonlog-root (log-root-dir (conf LOGVIEWER-APPENDER-NAME))]
     (setup-default-uncaught-exception-handler)
     (start-log-cleaner! conf log-root)
-    (start-logviewer! conf log-root)))
+    (start-logviewer! conf log-root daemonlog-root)
+    (start-metrics-reporters)))
diff --git a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
index 7ea515b..71d4654 100644
--- a/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/nimbus.clj
@@ -20,7 +20,7 @@
   (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
   (:import [org.apache.commons.io FileUtils])
   (:import [java.nio ByteBuffer]
-           [java.util Collections HashMap]
+           [java.util Collections List HashMap]
            [backtype.storm.generated NimbusSummary])
   (:import [java.io FileNotFoundException File FileOutputStream])
   (:import [java.net InetAddress])
@@ -34,19 +34,57 @@
             BufferFileInputStream])
   (:import [backtype.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
             ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
-            KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo
-            ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice])
+            KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
+            ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice
+            ComponentPageInfo TopologyPageInfo LogConfig LogLevel LogLevelAction
+            ProfileRequest ProfileAction NodeInfo])
   (:import [backtype.storm.daemon Shutdownable])
-  (:use [backtype.storm util config log timer zookeeper])
-  (:require [backtype.storm [cluster :as cluster] [stats :as stats] [converter :as converter]])
+  (:use [backtype.storm util config log timer zookeeper local-state])
+  (:require [backtype.storm [cluster :as cluster]
+                            [converter :as converter]
+                            [stats :as stats]
+                            [tuple :as tuple]])
   (:require [clojure.set :as set])
   (:import [backtype.storm.daemon.common StormBase Assignment])
   (:use [backtype.storm.daemon common])
   (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
   (:import [backtype.storm.utils VersionInfo])
+  (:require [clj-time.core :as time])
+  (:require [clj-time.coerce :as coerce])
+  (:require [metrics.meters :refer [defmeter mark!]])
+  (:require [metrics.gauges :refer [defgauge]])
   (:gen-class
     :methods [^{:static true} [launch [backtype.storm.scheduler.INimbus] void]]))
 
+(defmeter nimbus:num-submitTopologyWithOpts-calls)
+(defmeter nimbus:num-submitTopology-calls)
+(defmeter nimbus:num-killTopologyWithOpts-calls)
+(defmeter nimbus:num-killTopology-calls)
+(defmeter nimbus:num-rebalance-calls)
+(defmeter nimbus:num-activate-calls)
+(defmeter nimbus:num-deactivate-calls)
+(defmeter nimbus:num-debug-calls)
+(defmeter nimbus:num-setWorkerProfiler-calls)
+(defmeter nimbus:num-getComponentPendingProfileActions-calls)
+(defmeter nimbus:num-setLogConfig-calls)
+(defmeter nimbus:num-uploadNewCredentials-calls)
+(defmeter nimbus:num-beginFileUpload-calls)
+(defmeter nimbus:num-uploadChunk-calls)
+(defmeter nimbus:num-finishFileUpload-calls)
+(defmeter nimbus:num-beginFileDownload-calls)
+(defmeter nimbus:num-downloadChunk-calls)
+(defmeter nimbus:num-getNimbusConf-calls)
+(defmeter nimbus:num-getLogConfig-calls)
+(defmeter nimbus:num-getTopologyConf-calls)
+(defmeter nimbus:num-getTopology-calls)
+(defmeter nimbus:num-getUserTopology-calls)
+(defmeter nimbus:num-getClusterInfo-calls)
+(defmeter nimbus:num-getTopologyInfoWithOpts-calls)
+(defmeter nimbus:num-getTopologyInfo-calls)
+(defmeter nimbus:num-getTopologyPageInfo-calls)
+(defmeter nimbus:num-getComponentPageInfo-calls)
+(defmeter nimbus:num-shutdown-calls)
+
 (defn file-cache-map [conf]
   (TimeCacheMap.
    (int (conf NIMBUS-FILE-COPY-EXPIRATION-SECS))
@@ -88,6 +126,15 @@
   [(first ZooDefs$Ids/CREATOR_ALL_ACL) 
    (ACL. (bit-or ZooDefs$Perms/READ ZooDefs$Perms/CREATE) ZooDefs$Ids/ANYONE_ID_UNSAFE)])
 
+(defn create-tology-action-notifier [conf]
+  (when-not (clojure.string/blank? (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN))
+    (let [instance (new-instance (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN))]
+      (try
+        (.prepare instance conf)
+        instance
+        (catch Exception e
+          (log-warn-error e "Ingoring exception, Could not initialize " (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN)))))))
+
 (defn nimbus-data [conf inimbus]
   (let [forced-scheduler (.getForcedScheduler inimbus)]
     {:conf conf
@@ -99,9 +146,10 @@
      :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
                                                                        (Utils/isZkAuthenticationConfiguredStormServer
                                                                          conf)
-                                                                       NIMBUS-ZK-ACLS))
+                                                                       NIMBUS-ZK-ACLS) :separate-zk-writer? true)
      :submit-lock (Object.)
      :cred-update-lock (Object.)
+     :log-update-lock (Object.)
      :heartbeats-cache (atom {})
      :downloaders (file-cache-map conf)
      :uploaders (file-cache-map conf)
@@ -115,8 +163,13 @@
      :leader-elector (zk-leader-elector conf)
      :code-distributor (mk-code-distributor conf)
      :id->sched-status (atom {})
+     :node-id->resources (atom {}) ;;resources of supervisors
+     :id->resources (atom {}) ;;resources of topologies
      :cred-renewers (AuthUtils/GetCredentialRenewers conf)
+     :topology-history-lock (Object.)
+     :topo-history-state (nimbus-topo-history-state conf)
      :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)
+     :nimbus-topology-action-notifier (create-tology-action-notifier conf)
      }))
 
 (defn inbox [nimbus]
@@ -326,7 +379,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
@@ -538,12 +591,21 @@
   (into {} (for [[tid assignment] existing-assignments
                  :let [alive-executors (topology->alive-executors tid)
                        executor->node+port (:executor->node+port assignment)
+                       worker->resources (:worker->resources assignment)
+                       ;; making a map from node+port to WorkerSlot with allocated resources
+                       node+port->slot (into {} (for [[[node port] [mem-on-heap mem-off-heap cpu]] worker->resources]
+                                                  {[node port]
+                                                   (doto (WorkerSlot. node port)
+                                                     (.allocateResource
+                                                       mem-on-heap
+                                                       mem-off-heap
+                                                       cpu))}))
                        executor->slot (into {} (for [[executor [node port]] executor->node+port]
                                                  ;; filter out the dead executors
                                                  (if (contains? alive-executors executor)
                                                    {(ExecutorDetails. (first executor)
                                                                       (second executor))
-                                                    (WorkerSlot. node port)}
+                                                    (get node+port->slot [node port])}
                                                    {})))]]
              {tid (SchedulerAssignmentImpl. tid executor->slot)})))
 
@@ -561,7 +623,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 {}
@@ -606,8 +669,42 @@
     (count (.getSlots scheduler-assignment))
     0 ))
 
+(defn convert-assignments-to-worker->resources [new-scheduler-assignments]
+  "convert {topology-id -> SchedulerAssignment} to
+           {topology-id -> {[node port] [mem-on-heap mem-off-heap cpu]}}
+   Make sure this can deal with other non-RAS schedulers
+   later we may further support map-for-any-resources"
+  (map-val (fn [^SchedulerAssignment assignment]
+             (->> assignment
+                  .getExecutorToSlot
+                  .values
+                  (#(into {} (for [^WorkerSlot slot %]
+                              {[(.getNodeId slot) (.getPort slot)]
+                               [(.getAllocatedMemOnHeap slot) (.getAllocatedMemOffHeap slot) (.getAllocatedCpu slot)]
+                               })))))
+           new-scheduler-assignments))
+
+(defn compute-new-topology->executor->node+port [new-scheduler-assignments existing-assignments]
+  (let [new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)]
+    ;; print some useful information.
+    (doseq [[topology-id executor->node+port] new-topology->executor->node+port
+            :let [old-executor->node+port (-> topology-id
+                                              existing-assignments
+                                              :executor->node+port)
+                  reassignment (filter (fn [[executor node+port]]
+                                         (and (contains? old-executor->node+port executor)
+                                              (not (= node+port (old-executor->node+port executor)))))
+                                       executor->node+port)]]
+      (when-not (empty? reassignment)
+        (let [new-slots-cnt (count (set (vals executor->node+port)))
+              reassign-executors (keys reassignment)]
+          (log-message "Reassigning " topology-id " to " new-slots-cnt " slots")
+          (log-message "Reassign executors: " (vec reassign-executors)))))
+
+    new-topology->executor->node+port))
+
 ;; public so it can be mocked out
-(defn compute-new-topology->executor->node+port [nimbus existing-assignments topologies scratch-topology-id]
+(defn compute-new-scheduler-assignments [nimbus existing-assignments topologies scratch-topology-id]
   (let [conf (:conf nimbus)
         storm-cluster-state (:storm-cluster-state nimbus)
         topology->executors (compute-topology->executors nimbus (keys existing-assignments))
@@ -630,46 +727,30 @@
                                            .getTopologies
                                            (map (memfn getId))
                                            (filter (fn [t]
-                                                      (let [alle (get topology->executors t)
-                                                            alivee (get topology->alive-executors t)]
-                                                            (or (empty? alle)
-                                                                (not= alle alivee)
-                                                                (< (-> topology->scheduler-assignment
-                                                                       (get t)
-                                                                       num-used-workers )
-                                                                   (-> topologies (.getById t) .getNumWorkers)
-                                                                   ))
-                                                            ))))
+                                                     (let [alle (get topology->executors t)
+                                                           alivee (get topology->alive-executors t)]
+                                                       (or (empty? alle)
+                                                           (not= alle alivee)
+                                                           (< (-> topology->scheduler-assignment
+                                                                  (get t)
+                                                                  num-used-workers )
+                                                              (-> topologies (.getById t) .getNumWorkers)))))))
         all-scheduling-slots (->> (all-scheduling-slots nimbus topologies missing-assignment-topologies)
                                   (map (fn [[node-id port]] {node-id #{port}}))
                                   (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.
         _ (.schedule (:scheduler nimbus) topologies cluster)
-        new-scheduler-assignments (.getAssignments cluster)
-        ;; add more information to convert SchedulerAssignment to Assignment
-        new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)]
-    (reset! (:id->sched-status nimbus) (.getStatusMap cluster))
-    ;; print some useful information.
-    (doseq [[topology-id executor->node+port] new-topology->executor->node+port
-            :let [old-executor->node+port (-> topology-id
-                                          existing-assignments
-                                          :executor->node+port)
-                  reassignment (filter (fn [[executor node+port]]
-                                         (and (contains? old-executor->node+port executor)
-                                              (not (= node+port (old-executor->node+port executor)))))
-                                       executor->node+port)]]
-      (when-not (empty? reassignment)
-        (let [new-slots-cnt (count (set (vals executor->node+port)))
-              reassign-executors (keys reassignment)]
-          (log-message "Reassigning " topology-id " to " new-slots-cnt " slots")
-          (log-message "Reassign executors: " (vec reassign-executors)))))
-
-    new-topology->executor->node+port))
+        _ (.setResourcesMap cluster @(:id->resources nimbus))
+        _ (if-not (conf SCHEDULER-DISPLAY-RESOURCE) (.updateAssignedMemoryForTopologyAndSupervisor cluster topologies))
+        _ (reset! (:id->sched-status nimbus) (.getStatusMap cluster))
+        _ (reset! (:node-id->resources nimbus) (.getSupervisorsResourcesMap cluster))
+        _ (reset! (:id->resources nimbus) (.getResourcesMap cluster))]
+    (.getAssignments cluster)))
 
 (defn changed-executors [executor->node+port new-executor->node+port]
   (let [executor->node+port (if executor->node+port (sort executor->node+port) nil)
@@ -694,7 +775,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]]
@@ -724,14 +805,16 @@
                                         (when (or (nil? scratch-topology-id) (not= tid scratch-topology-id))
                                           {tid (.assignment-info storm-cluster-state tid nil)})))
         ;; make the new assignments for topologies
-        topology->executor->node+port (compute-new-topology->executor->node+port
+        new-scheduler-assignments (compute-new-scheduler-assignments
                                        nimbus
                                        existing-assignments
                                        topologies
                                        scratch-topology-id)
 
-        topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
+        topology->executor->node+port (compute-new-topology->executor->node+port new-scheduler-assignments existing-assignments)
 
+        topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
+        new-assigned-worker->resources (convert-assignments-to-worker->resources new-scheduler-assignments)
         now-secs (current-time-secs)
 
         basic-supervisor-details-map (basic-supervisor-details-map storm-cluster-state)
@@ -752,12 +835,14 @@
                                                                 (into {}
                                                                       (for [id reassign-executors]
                                                                         [id now-secs]
-                                                                        )))]]
+                                                                        )))
+                                              worker->resources (get new-assigned-worker->resources topology-id)]]
                                    {topology-id (Assignment.
                                                  (master-stormdist-root conf topology-id)
                                                  (select-keys all-node->host all-nodes)
                                                  executor->node+port
-                                                 start-times)}))]
+                                                 start-times
+                                                 worker->resources)}))]
 
     ;; tasks figure out what tasks to talk to by looking at topology at runtime
     ;; only log/set when there's been a change to the assignment
@@ -779,6 +864,13 @@
           (.assignSlots inimbus topologies)))
     (log-message "not a leader, skipping assignments")))
 
+(defn notify-topology-action-listener [nimbus storm-id action]
+  (let [topology-action-notifier (:nimbus-topology-action-notifier nimbus)]
+    (when (not-nil? topology-action-notifier)
+      (try (.notify topology-action-notifier storm-id action)
+        (catch Exception e
+        (log-warn-error e "Ignoring exception from Topology action notifier for storm-Id " storm-id))))))
+
 (defn- start-storm [nimbus storm-name storm-id topology-initial-status]
   {:pre [(#{:active :inactive} topology-initial-status)]}
   (let [storm-cluster-state (:storm-cluster-state nimbus)
@@ -797,7 +889,8 @@
                                   (storm-conf TOPOLOGY-SUBMITTER-USER)
                                   nil
                                   nil
-                                  {}))))
+                                  {}))
+    (notify-topology-action-listener nimbus storm-name "activate")))
 
 ;; Master:
 ;; job submit:
@@ -823,8 +916,7 @@
            impersonation-authorizer (:impersonation-authorization-handler nimbus)
            ctx (or context (ReqContext/context))
            check-conf (if storm-conf storm-conf (if storm-name {TOPOLOGY-NAME storm-name}))]
-       (log-message "[req " (.requestID ctx) "] Access from: " (.remoteAddress ctx) " principal:" (.principal ctx) " op:" operation)
-
+       (log-thrift-access (.requestID ctx) (.remoteAddress ctx) (.principal ctx) operation)
        (if (.isImpersonating ctx)
          (do
           (log-warn "principal: " (.realPrincipal ctx) " is trying to impersonate principal: " (.principal ctx))
@@ -947,6 +1039,17 @@
         (log-error "Cleaning inbox ... error deleting: " (.getName f))
         ))))
 
+(defn clean-topology-history
+  "Deletes topologies from history older than minutes."
+  [mins nimbus]
+  (locking (:topology-history-lock nimbus)
+    (let [cutoff-age (- (current-time-secs) (* mins 60))
+          topo-history-state (:topo-history-state nimbus)
+          curr-history (vec (ls-topo-hist topo-history-state))
+          new-history (vec (filter (fn [line]
+                                     (> (line :timestamp) cutoff-age)) curr-history))]
+      (ls-topo-hist! topo-history-state new-history))))
+
 (defn cleanup-corrupt-topologies! [nimbus]
   (if (is-leader nimbus :throw-exception false)
     (let [storm-cluster-state (:storm-cluster-state nimbus)
@@ -974,13 +1077,6 @@
                    (.set_host (:host %))
                    (.set_port (:port %))))))
 
-(defn- get-last-error
-  [storm-cluster-state storm-id component-id]
-  (if-let [e (.last-error storm-cluster-state storm-id component-id)]
-    (doto (ErrorInfo. (:error e) (:time-secs e))
-                      (.set_host (:host e))
-                      (.set_port (:port e)))))
-
 (defn- thriftify-executor-id [[first-task-id last-task-id]]
   (ExecutorInfo. (int first-task-id) (int last-task-id)))
 
@@ -1027,6 +1123,46 @@
   )
 )
 
+(defn add-topology-to-history-log
+  [storm-id nimbus topology-conf]
+  (log-message "Adding topo to history log: " storm-id)
+  (locking (:topology-history-lock nimbus)
+    (let [topo-history-state (:topo-history-state nimbus)
+          users (get-topo-logs-users topology-conf)
+          groups (get-topo-logs-groups topology-conf)
+          curr-history (vec (ls-topo-hist topo-history-state))
+          new-history (conj curr-history {:topoid storm-id :timestamp (current-time-secs)
+                                          :users users :groups groups})]
+      (ls-topo-hist! topo-history-state new-history))))
+
+(defn igroup-mapper
+  [storm-conf]
+  (AuthUtils/GetGroupMappingServiceProviderPlugin storm-conf))
+
+(defn user-groups
+  [user storm-conf]
+  (if (clojure.string/blank? user) [] (.getGroups (igroup-mapper storm-conf) user)))
+
+(defn does-users-group-intersect?
+  "Check to see if any of the users groups intersect with the list of groups passed in"
+  [user groups-to-check storm-conf]
+  (let [groups (user-groups user storm-conf)]
+    (> (.size (set/intersection (set groups) (set groups-to-check))) 0)))
+
+(defn read-topology-history
+  [nimbus user admin-users]
+  (let [topo-history-state (:topo-history-state nimbus)
+        curr-history (vec (ls-topo-hist topo-history-state))
+        topo-user-can-access (fn [line user storm-conf]
+                               (if (nil? user)
+                                 (line :topoid)
+                                 (if (or (some #(= % user) admin-users)
+                                       (does-users-group-intersect? user (line :groups) storm-conf)
+                                       (some #(= % user) (line :users)))
+                                   (line :topoid)
+                                   nil)))]
+    (remove nil? (map #(topo-user-can-access % user (:conf nimbus)) curr-history))))
+
 (defn renew-credentials [nimbus]
   (if (is-leader nimbus :throw-exception false)
     (let [storm-cluster-state (:storm-cluster-state nimbus)
@@ -1067,11 +1203,54 @@
        (InvalidTopologyException. 
         (str "Failed to submit topology. Topology requests more than " workers-allowed " workers."))))))
 
+(defn- set-logger-timeouts [log-config]
+  (let [timeout-secs (.get_reset_log_level_timeout_secs log-config)
+       timeout (time/plus (time/now) (time/secs timeout-secs))]
+   (if (time/after? timeout (time/now))
+     (.set_reset_log_level_timeout_epoch log-config (coerce/to-long timeout))
+     (.unset_reset_log_level_timeout_epoch log-config))))
+
 (defserverfn service-handler [conf inimbus]
   (.prepare inimbus conf (master-inimbus-dir conf))
   (log-message "Starting Nimbus with conf " conf)
   (let [nimbus (nimbus-data conf inimbus)
-       principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)]
+        principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf)
+        admin-users (or (.get conf NIMBUS-ADMINS) [])
+        get-common-topo-info
+          (fn [^String storm-id operation]
+            (let [storm-cluster-state (:storm-cluster-state nimbus)
+                  topology-conf (try-read-storm-conf conf storm-id)
+                  storm-name (topology-conf TOPOLOGY-NAME)
+                  _ (check-authorization! nimbus
+                                          storm-name
+                                          topology-conf
+                                          operation)
+                  topology (try-read-storm-topology conf storm-id)
+                  task->component (storm-task-info topology topology-conf)
+                  base (.storm-base storm-cluster-state storm-id nil)
+                  launch-time-secs (if base (:launch-time-secs base)
+                                     (throw
+                                       (NotAliveException. (str storm-id))))
+                  assignment (.assignment-info storm-cluster-state storm-id nil)
+                  beats (map-val :heartbeat (get @(:heartbeats-cache nimbus)
+                                                 storm-id))
+                  all-components (set (vals task->component))]
+              {:storm-name storm-name
+               :storm-cluster-state storm-cluster-state
+               :all-components all-components
+               :launch-time-secs launch-time-secs
+               :assignment assignment
+               :beats beats
+               :topology topology
+               :task->component task->component
+               :base base}))
+        get-last-error (fn [storm-cluster-state storm-id component-id]
+                         (if-let [e (.last-error storm-cluster-state
+                                                 storm-id
+                                                 component-id)]
+                           (doto (ErrorInfo. (:error e) (:time-secs e))
+                             (.set_host (:host e))
+                             (.set_port (:port e)))))]
     (.prepare ^backtype.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
 
     ;add to nimbuses
@@ -1096,7 +1275,7 @@
                         0
                         (conf NIMBUS-MONITOR-FREQ-SECS)
                         (fn []
-                          (when (conf NIMBUS-REASSIGN)
+                          (when-not (conf NIMBUS-DO-NOT-REASSIGN)
                             (locking (:submit-lock nimbus)
                               (mk-assignments nimbus)))
                           (do-cleanup nimbus)
@@ -1108,6 +1287,14 @@
                         (fn []
                           (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))
                           ))
+    ;; Schedule topology history cleaner
+    (when-let [interval (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)]
+      (schedule-recurring (:timer nimbus)
+        0
+        (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)
+        (fn []
+          (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus)
+          )))
     ;;schedule nimbus code sync thread to sync code from other nimbuses.
     (schedule-recurring (:timer nimbus)
       0
@@ -1121,11 +1308,18 @@
                         (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
                         (fn []
                           (renew-credentials nimbus)))
+
+    (defgauge nimbus:num-supervisors
+      (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))
+
+    (start-metrics-reporters)
+
     (reify Nimbus$Iface
       (^void submitTopologyWithOpts
         [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology
          ^SubmitOptions submitOptions]
         (try
+          (mark! nimbus:num-submitTopologyWithOpts-calls)
           (is-leader nimbus)
           (assert (not-nil? submitOptions))
           (validate-topology-name! storm-name)
@@ -1191,62 +1385,75 @@
               (wait-for-desired-code-replication nimbus total-storm-conf storm-id)
               (.setup-heartbeats! storm-cluster-state storm-id)
               (.setup-backpressure! storm-cluster-state storm-id)
+              (notify-topology-action-listener nimbus storm-name "submitTopology")
               (let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive
                                               TopologyInitialStatus/ACTIVE :active}]
-                (start-storm nimbus storm-name storm-id (thrift-status->kw-status (.get_initial_status submitOptions))))
-              (mk-assignments nimbus)))
+                (start-storm nimbus storm-name storm-id (thrift-status->kw-status (.get_initial_status submitOptions))))))
           (catch Throwable e
             (log-warn-error e "Topology submission exception. (topology name='" storm-name "')")
             (throw e))))
       
       (^void submitTopology
         [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology]
+        (mark! nimbus:num-submitTopology-calls)
         (.submitTopologyWithOpts this storm-name uploadedJarLocation serializedConf topology
                                  (SubmitOptions. TopologyInitialStatus/ACTIVE)))
       
       (^void killTopology [this ^String name]
-         (.killTopologyWithOpts this name (KillOptions.)))
+        (mark! nimbus:num-killTopology-calls)
+        (.killTopologyWithOpts this name (KillOptions.)))
 
       (^void killTopologyWithOpts [this ^String storm-name ^KillOptions options]
+        (mark! nimbus:num-killTopologyWithOpts-calls)
         (check-storm-active! nimbus storm-name true)
-        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
-          (check-authorization! nimbus storm-name topology-conf "killTopology"))
-        (let [wait-amt (if (.is_set_wait_secs options)
-                         (.get_wait_secs options)                         
-                         )]
-          (transition-name! nimbus storm-name [:kill wait-amt] true)
-          ))
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)
+              operation "killTopology"]
+          (check-authorization! nimbus storm-name topology-conf operation)
+          (let [wait-amt (if (.is_set_wait_secs options)
+                           (.get_wait_secs options)
+                           )]
+            (transition-name! nimbus storm-name [:kill wait-amt] true)
+            (notify-topology-action-listener nimbus storm-name operation))))
 
       (^void rebalance [this ^String storm-name ^RebalanceOptions options]
+        (mark! nimbus:num-rebalance-calls)
         (check-storm-active! nimbus storm-name true)
-        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
-          (check-authorization! nimbus storm-name topology-conf "rebalance"))
-        (let [wait-amt (if (.is_set_wait_secs options)
-                         (.get_wait_secs options))
-              num-workers (if (.is_set_num_workers options)
-                            (.get_num_workers options))
-              executor-overrides (if (.is_set_num_executors options)
-                                   (.get_num_executors options)
-                                   {})]
-          (doseq [[c num-executors] executor-overrides]
-            (when (<= num-executors 0)
-              (throw (InvalidTopologyException. "Number of executors must be greater than 0"))
-              ))
-          (transition-name! nimbus storm-name [:rebalance wait-amt num-workers executor-overrides] true)
-          ))
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)
+              operation "rebalance"]
+          (check-authorization! nimbus storm-name topology-conf operation)
+          (let [wait-amt (if (.is_set_wait_secs options)
+                           (.get_wait_secs options))
+                num-workers (if (.is_set_num_workers options)
+                              (.get_num_workers options))
+                executor-overrides (if (.is_set_num_executors options)
+                                     (.get_num_executors options)
+                                     {})]
+            (doseq [[c num-executors] executor-overrides]
+              (when (<= num-executors 0)
+                (throw (InvalidTopologyException. "Number of executors must be greater than 0"))
+                ))
+            (transition-name! nimbus storm-name [:rebalance wait-amt num-workers executor-overrides] true)
+
+            (notify-topology-action-listener nimbus storm-name operation))))
 
       (activate [this storm-name]
-        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
-          (check-authorization! nimbus storm-name topology-conf "activate"))
-        (transition-name! nimbus storm-name :activate true)
-        )
+        (mark! nimbus:num-activate-calls)
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)
+              operation "activate"]
+          (check-authorization! nimbus storm-name topology-conf operation)
+          (transition-name! nimbus storm-name :activate true)
+          (notify-topology-action-listener nimbus storm-name operation)))
 
       (deactivate [this storm-name]
-        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)]
-          (check-authorization! nimbus storm-name topology-conf "deactivate"))
-        (transition-name! nimbus storm-name :inactivate true))
+        (mark! nimbus:num-deactivate-calls)
+        (let [topology-conf (try-read-storm-conf-from-name conf storm-name nimbus)
+              operation "deactivate"]
+          (check-authorization! nimbus storm-name topology-conf operation)
+          (transition-name! nimbus storm-name :inactivate true)
+          (notify-topology-action-listener nimbus storm-name operation)))
 
       (debug [this storm-name component-id enable? samplingPct]
+        (mark! nimbus:num-debug-calls)
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
               topology-conf (try-read-storm-conf conf storm-id)
@@ -1265,7 +1472,66 @@
           (locking (:submit-lock nimbus)
             (.update-storm! storm-cluster-state storm-id storm-base-updates))))
 
+      (^void setWorkerProfiler
+        [this ^String id ^ProfileRequest profileRequest]
+        (mark! nimbus:num-setWorkerProfiler-calls)
+        (let [topology-conf (try-read-storm-conf conf id)
+              storm-name (topology-conf TOPOLOGY-NAME)
+              _ (check-authorization! nimbus storm-name topology-conf "setWorkerProfiler")
+              storm-cluster-state (:storm-cluster-state nimbus)]
+          (.set-worker-profile-request storm-cluster-state id profileRequest)))
+
+      (^List getComponentPendingProfileActions
+        [this ^String id ^String component_id ^ProfileAction action]
+        (mark! nimbus:num-getComponentPendingProfileActions-calls)
+        (let [info (get-common-topo-info id "getComponentPendingProfileActions")
+              storm-cluster-state (:storm-cluster-state info)
+              task->component (:task->component info)
+              {:keys [executor->node+port node->host]} (:assignment info)
+              executor->host+port (map-val (fn [[node port]]
+                                             [(node->host node) port])
+                                    executor->node+port)
+              nodeinfos (stats/extract-nodeinfos-from-hb-for-comp executor->host+port task->component false component_id)
+              all-pending-actions-for-topology (.get-topology-profile-requests storm-cluster-state id true)
+              latest-profile-actions (remove nil? (map (fn [nodeInfo]
+                                                         (->> all-pending-actions-for-topology
+                                                              (filter #(and (= (:host nodeInfo) (.get_node (.get_nodeInfo %)))
+                                                                         (= (:port nodeInfo) (first (.get_port (.get_nodeInfo  %))))))
+                                                              (filter #(= action (.get_action %)))
+                                                              (sort-by #(.get_time_stamp %) >)
+                                                              first))
+                                                    nodeinfos))]
+          (log-message "Latest profile actions for topology " id " component " component_id " " (pr-str latest-profile-actions))
+          latest-profile-actions))
+
+      (^void setLogConfig [this ^String id ^LogConfig log-config-msg]
+        (mark! nimbus:num-setLogConfig-calls)
+        (let [topology-conf (try-read-storm-conf conf id)
+              storm-name (topology-conf TOPOLOGY-NAME)
+              _ (check-authorization! nimbus storm-name topology-conf "setLogConfig")
+              storm-cluster-state (:storm-cluster-state nimbus)
+              merged-log-config (or (.topology-log-config storm-cluster-state id nil) (LogConfig.))
+              named-loggers (.get_named_logger_level merged-log-config)]
+            (doseq [[_ level] named-loggers]
+              (.set_action level LogLevelAction/UNCHANGED))
+            (doseq [[logger-name log-config] (.get_named_logger_level log-config-msg)]
+              (let [action (.get_action log-config)]
+                (if (clojure.string/blank? logger-name)
+                  (throw (RuntimeException. "Named loggers need a valid name. Use ROOT for the root logger")))
+                (condp = action
+                  LogLevelAction/UPDATE
+                    (do (set-logger-timeouts log-config)
+                          (.put_to_named_logger_level merged-log-config logger-name log-config))
+                  LogLevelAction/REMOVE
+                    (let [named-loggers (.get_named_logger_level merged-log-config)]
+                      (if (and (not (nil? named-loggers))
+                               (.containsKey named-loggers logger-name))
+                        (.remove named-loggers logger-name))))))
+            (log-message "Setting log config for " storm-name ":" merged-log-config)
+            (.set-topology-log-config! storm-cluster-state id merged-log-config)))
+
       (uploadNewCredentials [this storm-name credentials]
+        (mark! nimbus:num-uploadNewCredentials-calls)
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               storm-id (get-storm-id storm-cluster-state storm-name)
               topology-conf (try-read-storm-conf conf storm-id)
@@ -1274,6 +1540,7 @@
           (locking (:cred-update-lock nimbus) (.set-credentials! storm-cluster-state storm-id creds topology-conf))))
 
       (beginFileUpload [this]
+        (mark! nimbus:num-beginFileUpload-calls)
         (check-authorization! nimbus nil nil "fileUpload")
         (let [fileloc (str (inbox nimbus) "/stormjar-" (uuid) ".jar")]
           (.put (:uploaders nimbus)
@@ -1284,6 +1551,7 @@
           ))
 
       (^void uploadChunk [this ^String location ^ByteBuffer chunk]
+        (mark! nimbus:num-uploadChunk-calls)
         (check-authorization! nimbus nil nil "fileUpload")
         (let [uploaders (:uploaders nimbus)
               ^WritableByteChannel channel (.get uploaders location)]
@@ -1295,6 +1563,7 @@
           ))
 
       (^void finishFileUpload [this ^String location]
+        (mark! nimbus:num-finishFileUpload-calls)
         (check-authorization! nimbus nil nil "fileUpload")
         (let [uploaders (:uploaders nimbus)
               ^WritableByteChannel channel (.get uploaders location)]
@@ -1307,6 +1576,7 @@
           ))
 
       (^String beginFileDownload [this ^String file]
+        (mark! nimbus:num-beginFileDownload-calls)
         (check-authorization! nimbus nil nil "fileDownload")
         (check-file-access (:conf nimbus) file)
         (let [is (BufferFileInputStream. file)
@@ -1316,6 +1586,7 @@
           ))
 
       (^ByteBuffer downloadChunk [this ^String id]
+        (mark! nimbus:num-downloadChunk-calls)
         (check-authorization! nimbus nil nil "fileDownload")
         (let [downloaders (:downloaders nimbus)
               ^BufferFileInputStream is (.get downloaders id)]
@@ -1330,44 +1601,61 @@
             )))
 
       (^String getNimbusConf [this]
+        (mark! nimbus:num-getNimbusConf-calls)
         (check-authorization! nimbus nil nil "getNimbusConf")
         (to-json (:conf nimbus)))
 
+      (^LogConfig getLogConfig [this ^String id]
+        (mark! nimbus:num-getLogConfig-calls)
+        (let [topology-conf (try-read-storm-conf conf id)
+              storm-name (topology-conf TOPOLOGY-NAME)
+              _ (check-authorization! nimbus storm-name topology-conf "getLogConfig")
+             storm-cluster-state (:storm-cluster-state nimbus)
+             log-config (.topology-log-config storm-cluster-state id nil)]
+           (if log-config log-config (LogConfig.))))
+
       (^String getTopologyConf [this ^String id]
+        (mark! nimbus:num-getTopologyConf-calls)
         (let [topology-conf (try-read-storm-conf conf id)
               storm-name (topology-conf TOPOLOGY-NAME)]
               (check-authorization! nimbus storm-name topology-conf "getTopologyConf")
               (to-json topology-conf)))
 
       (^StormTopology getTopology [this ^String id]
+        (mark! nimbus:num-getTopology-calls)
         (let [topology-conf (try-read-storm-conf conf id)
               storm-name (topology-conf TOPOLOGY-NAME)]
               (check-authorization! nimbus storm-name topology-conf "getTopology")
               (system-topology! topology-conf (try-read-storm-topology conf id))))
 
       (^StormTopology getUserTopology [this ^String id]
+        (mark! nimbus:num-getUserTopology-calls)
         (let [topology-conf (try-read-storm-conf conf id)
               storm-name (topology-conf TOPOLOGY-NAME)]
               (check-authorization! nimbus storm-name topology-conf "getUserTopology")
               (try-read-storm-topology topology-conf id)))
 
       (^ClusterSummary getClusterInfo [this]
+        (mark! nimbus:num-getClusterInfo-calls)
         (check-authorization! nimbus nil nil "getClusterInfo")
         (let [storm-cluster-state (:storm-cluster-state nimbus)
               supervisor-infos (all-supervisor-info storm-cluster-state)
               ;; 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 [[total-mem total-cpu used-mem used-cpu] (.get @(:node-id->resources nimbus) id)]
+                                         (.set_used_mem sup-sum used-mem)
+                                         (.set_used_cpu sup-sum used-cpu))
+                                       (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)
 
@@ -1398,27 +1686,34 @@
                                                             (extract-status-str base))]
                                                (when-let [owner (:owner base)] (.set_owner topo-summ owner))
                                                (when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
+                                               (when-let [resources (.get @(:id->resources nimbus) id)]
+                                                 (.set_requested_memonheap topo-summ (get resources 0))
+                                                 (.set_requested_memoffheap topo-summ (get resources 1))
+                                                 (.set_requested_cpu topo-summ (get resources 2))
+                                                 (.set_assigned_memonheap topo-summ (get resources 3))
+                                                 (.set_assigned_memoffheap topo-summ (get resources 4))
+                                                 (.set_assigned_cpu topo-summ (get resources 5)))
                                                (.set_replication_count topo-summ (if (:code-distributor nimbus)
                                                                                    (.getReplicationCount (:code-distributor nimbus) id)
                                                                                    1))
                                                topo-summ
-                                          ))]
-          (ClusterSummary. supervisor-summaries
-                           topology-summaries
-                           nimbuses)
-          ))
+                                          ))
+              ret (ClusterSummary. supervisor-summaries
+                                   topology-summaries
+                                   nimbuses)
+              _ (.set_nimbus_uptime_secs ret nimbus-uptime)]
+              ret))
       
       (^TopologyInfo getTopologyInfoWithOpts [this ^String storm-id ^GetInfoOptions options]
-        (let [storm-cluster-state (:storm-cluster-state nimbus)
-              topology-conf (try-read-storm-conf conf storm-id)
-              storm-name (topology-conf TOPOLOGY-NAME)
-              _ (check-authorization! nimbus storm-name topology-conf "getTopologyInfo")
-              task->component (storm-task-info (try-read-storm-topology conf storm-id) topology-conf)
-              base (.storm-base storm-cluster-state storm-id nil)
-              launch-time-secs (if base (:launch-time-secs base) (throw (NotAliveException. (str storm-id))))
-              assignment (.assignment-info storm-cluster-state storm-id nil)
-              beats (map-val :heartbeat (get @(:heartbeats-cache nimbus) storm-id))
-              all-components (-> task->component reverse-map keys)
+        (mark! nimbus:num-getTopologyInfoWithOpts-calls)
+        (let [{:keys [storm-name
+                      storm-cluster-state
+                      all-components
+                      launch-time-secs
+                      assignment
+                      beats
+                      task->component
+                      base]} (get-common-topo-info storm-id "getTopologyInfo")
               num-err-choice (or (.get_num_err_choice options)
                                  NumErrorsChoice/ALL)
               errors-fn (condp = num-err-choice
@@ -1459,19 +1754,136 @@
                            )]
             (when-let [owner (:owner base)] (.set_owner topo-info owner))
             (when-let [sched-status (.get @(:id->sched-status nimbus) storm-id)] (.set_sched_status topo-info sched-status))
+            (when-let [resources (.get @(:id->resources nimbus) storm-id)]
+              (.set_requested_memonheap topo-info (get resources 0))
+              (.set_requested_memoffheap topo-info (get resources 1))
+              (.set_requested_cpu topo-info (get resources 2))
+              (.set_assigned_memonheap topo-info (get resources 3))
+              (.set_assigned_memoffheap topo-info (get resources 4))
+              (.set_assigned_cpu topo-info (get resources 5)))
             (when-let [component->debug (:component->debug base)]
               (.set_component_debug topo-info (map-val converter/thriftify-debugoptions component->debug)))
-            (.set_replication_count topo-info (.getReplicationCount (:code-distributor nimbus) storm-id))
+            (.set_replication_count topo-info (if (:code-distributor nimbus) (.getReplicationCount (:code-distributor nimbus) storm-id) 1))
+
             topo-info
           ))
 
-      (^TopologyInfo getTopologyInfo [this ^String storm-id]
+      (^TopologyInfo getTopologyInfo [this ^String topology-id]
+        (mark! nimbus:num-getTopologyInfo-calls)
         (.getTopologyInfoWithOpts this
-                                  storm-id
+                                  topology-id
                                   (doto (GetInfoOptions.) (.set_num_err_choice NumErrorsChoice/ALL))))
 
+      (^TopologyPageInfo getTopologyPageInfo
+        [this ^String topo-id ^String window ^boolean include-sys?]
+        (mark! nimbus:num-getTopologyPageInfo-calls)
+        (let [info (get-common-topo-info topo-id "getTopologyPageInfo")
+
+              exec->node+port (:executor->node+port (:assignment info))
+              last-err-fn (partial get-last-error
+                                   (:storm-cluster-state info)
+                                   topo-id)
+              topo-page-info (stats/agg-topo-execs-stats topo-id
+                                                         exec->node+port
+                                                         (:task->component info)
+                                                         (:beats info)
+                                                         (:topology info)
+                                                         window
+                                                         include-sys?
+                                                         last-err-fn)]
+          (when-let [owner (:owner (:base info))]
+            (.set_owner topo-page-info owner))
+          (when-let [sched-status (.get @(:id->sched-status nimbus) topo-id)]
+            (.set_sched_status topo-page-info sched-status))
+          (when-let [resources (.get @(:id->resources nimbus) topo-id)]
+            (.set_requested_memonheap topo-page-info (get resources 0))
+            (.set_requested_memoffheap topo-page-info (get resources 1))
+            (.set_requested_cpu topo-page-info (get resources 2))
+            (.set_assigned_memonheap topo-page-info (get resources 3))
+            (.set_assigned_memoffheap topo-page-info (get resources 4))
+            (.set_assigned_cpu topo-page-info (get resources 5)))
+          (doto topo-page-info
+            (.set_name (:storm-name info))
+            (.set_status (extract-status-str (:base info)))
+            (.set_uptime_secs (time-delta (:launch-time-secs info)))
+            (.set_topology_conf (to-json (try-read-storm-conf conf
+                                                              topo-id)))
+            (.set_replication_count
+              (.getReplicationCount (:code-distributor nimbus) topo-id)))
+          (when-let [debug-options
+                     (get-in info [:base :component->debug topo-id])]
+            (.set_debug_options
+              topo-page-info
+              (converter/thriftify-debugoptions debug-options)))
+          topo-page-info))
+
+      (^ComponentPageInfo getComponentPageInfo
+        [this
+         ^String topo-id
+         ^String component-id
+         ^String window
+         ^boolean include-sys?]
+        (mark! nimbus:num-getComponentPageInfo-calls)
+        (let [info (get-common-topo-info topo-id "getComponentPageInfo")
+              {:keys [executor->node+port node->host]} (:assignment info)
+              executor->host+port (map-val (fn [[node port]]
+                                             [(node->host node) port])
+                                           executor->node+port)
+              comp-page-info (stats/agg-comp-execs-stats executor->host+port
+                                                         (:task->component info)
+                                                         (:beats info)
+                                                         window
+                                                         include-sys?
+                                                         topo-id
+                                                         (:topology info)
+                                                         component-id)]
+          (doto comp-page-info
+            (.set_topology_name (:storm-name info))
+            (.set_errors (get-errors (:storm-cluster-state info)
+                                     topo-id
+                                     component-id))
+            (.set_topology_status (extract-status-str (:base info))))
+          (when-let [debug-options
+                     (get-in info [:base :component->debug component-id])]
+            (.set_debug_options
+              comp-page-info
+              (converter/thriftify-debugoptions debug-options)))
+          ;; Add the event logger details.
+          (let [component->tasks (reverse-map (:task->component info))
+                eventlogger-tasks (sort (get component->tasks
+                                             EVENTLOGGER-COMPONENT-ID))
+                ;; Find the task the events from this component route to.
+                task-index (mod (tuple/list-hash-code [component-id])
+                                (count eventlogger-tasks))
+                task-id (nth eventlogger-tasks task-index)
+                eventlogger-exec (first (filter (fn [[start stop]]
+                                                  (between? task-id start stop))
+                                                (keys executor->host+port)))
+                [host port] (get executor->host+port eventlogger-exec)]
+            (if (and host port)
+              (doto comp-page-info
+                (.set_eventlog_host host)
+                (.set_eventlog_port port))))
+          comp-page-info))
+
+      (^TopologyHistoryInfo getTopologyHistory [this ^String user]
+        (let [storm-cluster-state (:storm-cluster-state nimbus)
+              bases (topology-bases storm-cluster-state)
+              assigned-topology-ids (.assignments storm-cluster-state nil)
+              user-group-match-fn (fn [topo-id user conf]
+                                    (let [topology-conf (try-read-storm-conf conf topo-id)
+                                          groups (get-topo-logs-groups topology-conf)]
+                                      (or (nil? user)
+                                          (some #(= % user) admin-users)
+                                          (does-users-group-intersect? user groups conf)
+                                          (some #(= % user) (get-topo-logs-users topology-conf)))))
+              active-ids-for-user (filter #(user-group-match-fn % user (:conf nimbus)) assigned-topology-ids)
+              topo-history-list (read-topology-history nimbus user admin-users)]
+          (TopologyHistoryInfo. (distinct (concat active-ids-for-user topo-history-list)))))
+
       Shutdownable
       (shutdown [this]
+        (mark! nimbus:num-shutdown-calls)
         (log-message "Shutting down master")
         (cancel-timer (:timer nimbus))
         (.disconnect (:storm-cluster-state nimbus))
@@ -1479,6 +1891,7 @@
         (.cleanup (:uploaders nimbus))
         (.close (:leader-elector nimbus))
         (if (:code-distributor nimbus) (.close (:code-distributor nimbus) (:conf nimbus)))
+        (when (:nimbus-topology-action-notifier nimbus) (.cleanup (:nimbus-topology-action-notifier nimbus)))
         (log-message "Shut down master")
         )
       DaemonCommon
@@ -1509,9 +1922,8 @@
 
 (defmethod sync-code :distributed [conf nimbus]
   (let [storm-cluster-state (:storm-cluster-state nimbus)
-        code-ids (set (code-ids (:conf nimbus)))
         active-topologies (set (.code-distributor storm-cluster-state (fn [] (sync-code conf nimbus))))
-        missing-topologies (set/difference active-topologies code-ids)]
+        missing-topologies (set/difference active-topologies (set (code-ids (:conf nimbus))))]
     (if (not (empty? missing-topologies))
       (do
         (.removeFromLeaderLockQueue (:leader-elector nimbus))
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 2a7613d..d659d57 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -25,16 +25,23 @@
            [java.io File])
   (:use [backtype.storm config util log timer local-state])
   (:import [backtype.storm.utils VersionInfo])
+  (:import [backtype.storm Config])
+  (:import [backtype.storm.generated WorkerResources ProfileAction])
   (:use [backtype.storm.daemon common])
+  (:require [backtype.storm.command [healthcheck :as healthcheck]])
   (:require [backtype.storm.daemon [worker :as worker]]
             [backtype.storm [process-simulator :as psim] [cluster :as cluster] [event :as event]]
             [clojure.set :as set])
   (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
   (:import [org.yaml.snakeyaml Yaml]
            [org.yaml.snakeyaml.constructor SafeConstructor])
+  (:require [metrics.gauges :refer [defgauge]])
+  (:require [metrics.meters :refer [defmeter mark!]])
   (:gen-class
     :methods [^{:static true} [launch [backtype.storm.scheduler.ISupervisor] void]]))
 
+(defmeter supervisor:num-workers-launched)
+
 (defmulti download-storm-code cluster-mode)
 (defmulti launch-worker (fn [supervisor & _] (cluster-mode (:conf supervisor))))
 (defmulti mk-code-distributor cluster-mode)
@@ -57,28 +64,38 @@
                         {sid (.assignment-info-with-version storm-cluster-state sid callback)})
                       {sid nil})))
            (apply merge)
-           (filter-val not-nil?))]
-
+           (filter-val not-nil?))
+          new-profiler-actions
+          (->>
+            (dofor [sid (distinct storm-ids)]
+                   (if-let [topo-profile-actions (.get-topology-profile-requests storm-cluster-state sid false)]
+                      {sid topo-profile-actions}))
+           (apply merge))]
+         
       {:assignments (into {} (for [[k v] new-assignments] [k (:data v)]))
+       :profiler-actions new-profiler-actions
        :versions new-assignments})))
 
 (defn- read-my-executors [assignments-snapshot storm-id assignment-id]
   (let [assignment (get assignments-snapshot storm-id)
+        my-slots-resources (into {}
+                                 (filter (fn [[[node _] _]] (= node assignment-id))
+                                         (:worker->resources assignment)))
         my-executors (filter (fn [[_ [node _]]] (= node assignment-id))
-                           (:executor->node+port assignment))
+                             (:executor->node+port assignment))
         port-executors (apply merge-with
-                          concat
-                          (for [[executor [_ port]] my-executors]
-                            {port [executor]}
-                            ))]
+                              concat
+                              (for [[executor [_ port]] my-executors]
+                                {port [executor]}
+                                ))]
     (into {} (for [[port executors] port-executors]
                ;; need to cast to int b/c it might be a long (due to how yaml parses things)
                ;; doall is to avoid serialization/deserialization problems with lazy seqs
-               [(Integer. port) (mk-local-assignment storm-id (doall executors))]
+               [(Integer. port) (mk-local-assignment storm-id (doall executors) (get my-slots-resources [assignment-id port]))]
                ))))
 
 (defn- read-assignments
-  "Returns map from port to struct containing :storm-id and :executors"
+  "Returns map from port to struct containing :storm-id, :executors and :resources"
   ([assignments-snapshot assignment-id]
      (->> (dofor [sid (keys assignments-snapshot)] (read-my-executors assignments-snapshot sid assignment-id))
           (apply merge-with (fn [& ignored] (throw-runtime "Should not have multiple topologies assigned to one port")))))
@@ -196,7 +213,7 @@
 (defn generate-supervisor-id []
   (uuid))
 
-(defnk worker-launcher [conf user args :environment {} :log-prefix nil :exit-code-callback nil]
+(defnk worker-launcher [conf user args :environment {} :log-prefix nil :exit-code-callback nil :directory nil]
   (let [_ (when (clojure.string/blank? user)
             (throw (java.lang.IllegalArgumentException.
                      "User cannot be blank when calling worker-launcher.")))
@@ -205,7 +222,7 @@
         wl (if wl-initial wl-initial (str storm-home "/bin/worker-launcher"))
         command (concat [wl user] args)]
     (log-message "Running as user:" user " command:" (pr-str command))
-    (launch-process command :environment environment :log-prefix log-prefix :exit-code-callback exit-code-callback)
+    (launch-process command :environment environment :log-prefix log-prefix :exit-code-callback exit-code-callback :directory directory)
   ))
 
 (defnk worker-launcher-and-wait [conf user args :environment {} :log-prefix nil]
@@ -238,8 +255,8 @@
           (do
             (rmr (worker-heartbeats-root conf id))
             ;; this avoids a race condition with worker or subprocess writing pid around same time
-            (rmpath (worker-pids-root conf id))
-            (rmpath (worker-root conf id))))
+            (rmr (worker-pids-root conf id))
+            (rmr (worker-root conf id))))
         (remove-worker-user! conf id)
         (remove-dead-worker id)
       ))
@@ -313,6 +330,7 @@
    :sync-retry (atom 0)
    :code-distributor (mk-code-distributor conf)
    :download-lock (Object.)
+   :stormid->profiler-actions (atom {})
    })
 
 (defn sync-processes [supervisor]
@@ -386,7 +404,10 @@
     (wait-for-workers-launch
      conf
      (dofor [[port assignment] reassign-executors]
-            (let [id (new-worker-ids port)]
+            (let [id (new-worker-ids port)
+                  storm-id (:storm-id assignment)
+                  ^WorkerResources resources (:resources assignment)
+                  mem-onheap (.get_mem_on_heap resources)]
               (try
                 (log-message "Launching worker with assignment "
                              (pr-str assignment)
@@ -400,7 +421,9 @@
                 (launch-worker supervisor
                                (:storm-id assignment)
                                port
-                               id)
+                               id
+                               mem-onheap)
+                (mark! supervisor:num-workers-launched)
                 (catch java.io.FileNotFoundException e
                   (log-message "Unable to launch worker due to "
                                (.getMessage e)))
@@ -440,9 +463,10 @@
           ^LocalState local-state (:local-state supervisor)
           sync-callback (fn [& ignored] (.add event-manager this))
           assignment-versions @(:assignment-versions supervisor)
-          {assignments-snapshot :assignments versions :versions}  (assignments-snapshot
-                                                                   storm-cluster-state sync-callback
-                                                                  assignment-versions)
+          {assignments-snapshot :assignments
+           storm-id->profiler-actions :profiler-actions
+           versions :versions}
+          (assignments-snapshot storm-cluster-state sync-callback assignment-versions)
           storm-code-map (read-storm-code-locations assignments-snapshot)
           downloaded-storm-ids (set (read-downloaded-storm-ids conf))
           existing-assignment (ls-local-assignments local-state)
@@ -452,13 +476,13 @@
                                            (:sync-retry supervisor))
           new-assignment (->> all-assignment
                               (filter-key #(.confirmAssigned isupervisor %)))
-          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)
-          ]
+          assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)]
       (log-debug "Synchronizing supervisor")
       (log-debug "Storm code map: " storm-code-map)
       (log-debug "Downloaded storm ids: " downloaded-storm-ids)
       (log-debug "All assignment: " all-assignment)
       (log-debug "New assignment: " new-assignment)
+      (log-debug "Storm Ids Profiler Actions" storm-id->profiler-actions)
 
       ;; download code first
       ;; This might take awhile
@@ -478,6 +502,8 @@
       (ls-local-assignments! local-state
             new-assignment)
       (reset! (:assignment-versions supervisor) versions)
+      (reset! (:stormid->profiler-actions supervisor) storm-id->profiler-actions)
+
       (reset! (:curr-assignment supervisor) new-assignment)
       ;; remove any downloaded code that's no longer assigned or active
       ;; important that this happens after setting the local assignment so that
@@ -495,6 +521,125 @@
       (.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))})
+
+(defn jvm-cmd [cmd]
+  (let [java-home (.get (System/getenv) "JAVA_HOME")]
+    (if (nil? java-home)
+      cmd
+      (str java-home file-path-separator "bin" file-path-separator cmd))))
+
+(defn java-cmd []
+  (jvm-cmd "java"))
+
+(defn jmap-dump-cmd [profile-cmd pid target-dir]
+  [profile-cmd pid "jmap" target-dir])
+
+(defn jstack-dump-cmd [profile-cmd pid target-dir]
+  [profile-cmd pid "jstack" target-dir])
+
+(defn jprofile-start [profile-cmd pid]
+  [profile-cmd pid "start"])
+
+(defn jprofile-stop [profile-cmd pid target-dir]
+  [profile-cmd pid "stop" target-dir])
+
+(defn jprofile-dump [profile-cmd pid workers-artifacts-directory]
+  [profile-cmd pid "dump" workers-artifacts-directory])
+
+(defn jprofile-jvm-restart [profile-cmd pid]
+  [profile-cmd pid "kill"])
+
+(defn- delete-topology-profiler-action [storm-cluster-state storm-id profile-action]
+  (log-message "Deleting profiler action.." profile-action)
+  (.delete-topology-profile-requests storm-cluster-state storm-id profile-action))
+
+(defnk launch-profiler-action-for-worker
+  "Launch profiler action for a worker"
+  [conf user target-dir command :environment {} :exit-code-on-profile-action nil :log-prefix nil]
+  (if-let [run-worker-as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)]
+    (let [container-file (container-file-path target-dir)
+          script-file (script-file-path target-dir)]
+      (log-message "Running as user:" user " command:" (shell-cmd command))
+      (if (exists-file? container-file) (rmr-as-user conf container-file container-file))
+      (if (exists-file? script-file) (rmr-as-user conf script-file script-file))
+      (worker-launcher
+        conf
+        user
+        ["profiler" target-dir (write-script target-dir command :environment environment)]
+        :log-prefix log-prefix
+        :exit-code-callback exit-code-on-profile-action
+        :directory (File. target-dir)))
+    (launch-process
+      command
+      :environment environment
+      :log-prefix log-prefix
+      :exit-code-callback exit-code-on-profile-action
+      :directory (File. target-dir))))
+
+(defn mk-run-profiler-actions-for-all-topologies
+  "Returns a function that downloads all profile-actions listed for all topologies assigned
+  to this supervisor, executes those actions as user and deletes them from zookeeper."
+  [supervisor]
+  (fn []
+    (try
+      (let [conf (:conf supervisor)
+            stormid->profiler-actions @(:stormid->profiler-actions supervisor)
+            storm-cluster-state (:storm-cluster-state supervisor)
+            hostname (:my-hostname supervisor)
+            profile-cmd (conf WORKER-PROFILER-COMMAND)
+            new-assignment @(:curr-assignment supervisor)
+            assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)]
+        (doseq [[storm-id profiler-actions] stormid->profiler-actions]
+          (when (not (empty? profiler-actions))
+            (doseq [pro-action profiler-actions]
+              (if (= hostname (:host pro-action))
+                (let [port (:port pro-action)
+                      action ^ProfileAction (:action pro-action)
+                      stop? (> (System/currentTimeMillis) (:timestamp pro-action))
+                      target-dir (worker-artifacts-root conf storm-id port)
+                      storm-conf (read-supervisor-storm-conf conf storm-id)
+                      user (storm-conf TOPOLOGY-SUBMITTER-USER)
+                      environment (if-let [env (storm-conf TOPOLOGY-ENVIRONMENT)] env {})
+                      worker-pid (slurp (worker-artifacts-pid-path conf storm-id port))
+                      log-prefix (str "ProfilerAction process " storm-id ":" port " PROFILER_ACTION: " action " ")
+                      ;; Until PROFILER_STOP action is invalid, keep launching profiler start in case worker restarted
+                      ;; The profiler plugin script validates if JVM is recording before starting another recording.
+                      command (cond
+                                (= action ProfileAction/JMAP_DUMP) (jmap-dump-cmd profile-cmd worker-pid target-dir)
+                                (= action ProfileAction/JSTACK_DUMP) (jstack-dump-cmd profile-cmd worker-pid target-dir)
+                                (= action ProfileAction/JPROFILE_DUMP) (jprofile-dump profile-cmd worker-pid target-dir)
+                                (= action ProfileAction/JVM_RESTART) (jprofile-jvm-restart profile-cmd worker-pid)
+                                (and (not stop?)
+                                     (= action ProfileAction/JPROFILE_STOP))
+                                  (jprofile-start profile-cmd worker-pid) ;; Ensure the profiler is still running
+                                (and stop? (= action ProfileAction/JPROFILE_STOP)) (jprofile-stop profile-cmd worker-pid target-dir))
+                      action-on-exit (fn [exit-code]
+                                       (log-message log-prefix " profile-action exited for code: " exit-code)
+                                       (if (and (= exit-code 0) stop?)
+                                         (delete-topology-profiler-action storm-cluster-state storm-id pro-action)))
+                      command (->> command (map str) (filter (complement empty?)))]
+
+                  (try
+                    (launch-profiler-action-for-worker conf
+                      user
+                      target-dir
+                      command
+                      :environment environment
+                      :exit-code-on-profile-action action-on-exit
+                      :log-prefix log-prefix)
+                    (catch IOException ioe
+                      (log-error ioe
+                        (str "Error in processing ProfilerAction '" action "' for " storm-id ":" port ", will retry later.")))
+                    (catch RuntimeException rte
+                      (log-error rte
+                        (str "Error in processing ProfilerAction '" action "' for " storm-id ":" port ", will retry later."))))))))))
+      (catch Exception e
+        (log-error e "Error running profiler actions, will retry again later")))))
+
 ;; 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]
@@ -505,6 +650,7 @@
         [event-manager processes-event-manager :as managers] [(event/event-manager false) (event/event-manager false)]
         sync-processes (partial sync-processes supervisor)
         synchronize-supervisor (mk-synchronize-supervisor supervisor sync-processes event-manager processes-event-manager)
+        run-profiler-actions-fn (mk-run-profiler-actions-for-all-topologies supervisor)
         heartbeat-fn (fn [] (.supervisor-heartbeat!
                                (:storm-cluster-state supervisor)
                                (:supervisor-id supervisor)
@@ -516,7 +662,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)
@@ -531,7 +678,22 @@
       (schedule-recurring (:event-timer supervisor)
                           0
                           (conf SUPERVISOR-MONITOR-FREQUENCY-SECS)
-                          (fn [] (.add processes-event-manager sync-processes))))
+                          (fn [] (.add processes-event-manager sync-processes)))
+      (schedule-recurring (:event-timer supervisor)
+                          (* 60 5)
+                          (* 60 5)
+                          (fn [] (let [health-code (healthcheck/health-check conf)
+                                       ids (my-worker-ids conf)]
+                                   (if (not (= health-code 0))
+                                     (do
+                                       (doseq [id ids]
+                                         (shutdown-worker supervisor id))
+                                       (throw (RuntimeException. "Supervisor failed health check. Exiting.")))))))
+      ;; Launch a thread that Runs profiler commands . Starts with 30 seconds delay, every 30 seconds
+      (schedule-recurring (:event-timer supervisor)
+                          30
+                          30
+                          (fn [] (.add event-manager run-profiler-actions-fn))))
     (log-message "Starting supervisor with id " (:supervisor-id supervisor) " at host " (:my-hostname supervisor))
     (reify
      Shutdownable
@@ -591,7 +753,7 @@
         (log-message "Finished downloading code for storm id " storm-id " from " master-code-dir))))
 
 (defn write-log-metadata-to-yaml-file! [storm-id port data conf]
-  (let [file (get-log-metadata-file storm-id port)]
+  (let [file (get-log-metadata-file conf storm-id port)]
     ;;run worker as user needs the directory to have special permissions
     ;; or it is insecure
     (when (not (.exists (.getParentFile file)))
@@ -600,7 +762,7 @@
             (setup-storm-code-dir conf (read-supervisor-storm-conf conf storm-id) (.getCanonicalPath (.getParentFile file))))
         (.mkdirs (.getParentFile file))))
     (let [writer (java.io.FileWriter. file)
-        yaml (Yaml.)]
+          yaml (Yaml.)]
       (try
         (.dump yaml data writer)
         (finally
@@ -632,31 +794,35 @@
     (str arch-resource-root File/pathSeparator resource-root File/pathSeparator (conf JAVA-LIBRARY-PATH))))
 
 (defn substitute-childopts
-  "Generates runtime childopts by replacing keys with topology-id, worker-id, port"
-  [value worker-id topology-id port]
+  "Generates runtime childopts by replacing keys with topology-id, worker-id, port, mem-onheap"
+  [value worker-id topology-id port mem-onheap]
   (let [replacement-map {"%ID%"          (str port)
                          "%WORKER-ID%"   (str worker-id)
                          "%TOPOLOGY-ID%"    (str topology-id)
-                         "%WORKER-PORT%" (str port)}
+                         "%WORKER-PORT%" (str port)
+                         "%HEAP-MEM%" (str mem-onheap)}
         sub-fn #(reduce (fn [string entry]
                           (apply clojure.string/replace string entry))
                         %
                         replacement-map)]
     (cond
       (nil? value) nil
-      (list? value) (map sub-fn value)
+      (sequential? value) (vec (map sub-fn value))
       :else (-> value sub-fn (clojure.string/split #"\s+")))))
 
-(defn java-cmd []
-  (let [java-home (.get (System/getenv) "JAVA_HOME")]
-    (if (nil? java-home)
-      "java"
-      (str java-home file-path-separator "bin" file-path-separator "java")
-      )))
 
+(defn create-artifacts-link
+  "Create a symlink from workder directory to its port artifacts directory"
+  [conf storm-id port worker-id]
+  (let [worker-dir (worker-root conf worker-id)
+        topo-dir (worker-artifacts-root conf storm-id)]
+    (log-message "Creating symlinks for worker-id: " worker-id " storm-id: "
+                 storm-id " to its port artifacts directory")
+    (if (.exists (File. worker-dir))
+      (create-symlink! worker-dir topo-dir "artifacts" port))))
 
 (defmethod launch-worker
-    :distributed [supervisor storm-id port worker-id]
+    :distributed [supervisor storm-id port worker-id mem-onheap]
     (let [conf (:conf supervisor)
           run-worker-as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
           storm-home (System/getProperty "storm.home")
@@ -664,7 +830,11 @@
           storm-conf-file (System/getProperty "storm.conf.file")
           storm-log-dir (or (System/getProperty "storm.log.dir") (str storm-home file-path-separator "logs"))
           storm-log-conf-dir (conf STORM-LOG4J2-CONF-DIR)
-          storm-log4j2-conf-dir (or storm-log-conf-dir (str storm-home file-path-separator "log4j2"))
+          storm-log4j2-conf-dir (if storm-log-conf-dir
+                                  (if (is-absolute-path? storm-log-conf-dir)
+                                    storm-log-conf-dir
+                                    (str storm-home file-path-separator storm-log-conf-dir))
+                                  (str storm-home file-path-separator "log4j2"))
           stormroot (supervisor-stormdist-root conf storm-id)
           jlp (jlp stormroot conf)
           stormjar (supervisor-stormjar-path stormroot)
@@ -676,15 +846,22 @@
                         (add-to-classpath [stormjar])
                         (add-to-classpath topo-classpath))
           top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
-          gc-opts (substitute-childopts (if top-gc-opts top-gc-opts (conf WORKER-GC-CHILDOPTS)) worker-id storm-id port)
+          mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
+                       (int (Math/ceil mem-onheap)) ;; round up
+                       (storm-conf WORKER-HEAP-MEMORY-MB)) ;; otherwise use default value
+          gc-opts (substitute-childopts (if top-gc-opts top-gc-opts (conf WORKER-GC-CHILDOPTS)) worker-id storm-id port mem-onheap)
           topo-worker-logwriter-childopts (storm-conf TOPOLOGY-WORKER-LOGWRITER-CHILDOPTS)
           user (storm-conf TOPOLOGY-SUBMITTER-USER)
+          logfilename "worker.log"
+          workers-artifacts (worker-artifacts-root conf)
           logging-sensitivity (storm-conf TOPOLOGY-LOGGING-SENSITIVITY "S3")
-          logfilename (logs-filename storm-id port)
           worker-childopts (when-let [s (conf WORKER-CHILDOPTS)]
-                             (substitute-childopts s worker-id storm-id port))
+                             (substitute-childopts s worker-id storm-id port mem-onheap))
           topo-worker-childopts (when-let [s (storm-conf TOPOLOGY-WORKER-CHILDOPTS)]
-                                  (substitute-childopts s worker-id storm-id port))
+                                  (substitute-childopts s worker-id storm-id port mem-onheap))
+          worker--profiler-childopts (if (conf WORKER-PROFILER-ENABLED)
+                                       (substitute-childopts (conf WORKER-PROFILER-CHILDOPTS) worker-id storm-id port mem-onheap)
+                                       "")
           topology-worker-environment (if-let [env (storm-conf TOPOLOGY-ENVIRONMENT)]
                                         (merge env {"LD_LIBRARY_PATH" jlp})
                                         {"LD_LIBRARY_PATH" jlp})
@@ -693,24 +870,29 @@
                      topo-worker-logwriter-childopts
                      (str "-Dlogfile.name=" logfilename)
                      (str "-Dstorm.home=" storm-home)
+                     (str "-Dworkers.artifacts=" workers-artifacts)
                      (str "-Dstorm.id=" storm-id)
                      (str "-Dworker.id=" worker-id)
                      (str "-Dworker.port=" port)
                      (str "-Dstorm.log.dir=" storm-log-dir)
                      (str "-Dlog4j.configurationFile=" storm-log4j2-conf-dir file-path-separator "worker.xml")
+                     (str "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector")
                      "backtype.storm.LogWriter"]
                     [(java-cmd) "-server"]
                     worker-childopts
                     topo-worker-childopts
                     gc-opts
+                    worker--profiler-childopts
                     [(str "-Djava.library.path=" jlp)
                      (str "-Dlogfile.name=" logfilename)
                      (str "-Dstorm.home=" storm-home)
+                     (str "-Dworkers.artifacts=" workers-artifacts)
                      (str "-Dstorm.conf.file=" storm-conf-file)
                      (str "-Dstorm.options=" storm-options)
                      (str "-Dstorm.log.dir=" storm-log-dir)
                      (str "-Dlogging.sensitivity=" logging-sensitivity)
                      (str "-Dlog4j.configurationFile=" storm-log4j2-conf-dir file-path-separator "worker.xml")
+                     (str "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector")
                      (str "-Dstorm.id=" storm-id)
                      (str "-Dworker.id=" worker-id)
                      (str "-Dworker.port=" port)
@@ -724,16 +906,17 @@
       (log-message "Launching worker with command: " (shell-cmd command))
       (write-log-metadata! storm-conf user worker-id storm-id port conf)
       (set-worker-user! conf worker-id user)
+      (create-artifacts-link conf storm-id port worker-id)
       (let [log-prefix (str "Worker Process " worker-id)
-           callback (fn [exit-code]
-                          (log-message log-prefix " exited with code: " exit-code)
-                          (add-dead-worker worker-id))]
+            callback (fn [exit-code]
+                       (log-message log-prefix " exited with code: " exit-code)
+                       (add-dead-worker worker-id))
+            worker-dir (worker-root conf worker-id)]
         (remove-dead-worker worker-id)
         (if run-worker-as-user
-          (let [worker-dir (worker-root conf worker-id)]
-            (worker-launcher conf user ["worker" worker-dir (write-script worker-dir command :environment topology-worker-environment)] :log-prefix log-prefix :exit-code-callback callback))
-          (launch-process command :environment topology-worker-environment :log-prefix log-prefix :exit-code-callback callback)
-      ))))
+          (worker-launcher conf user ["worker" worker-dir (write-script worker-dir command :environment topology-worker-environment)] :log-prefix log-prefix :exit-code-callback callback :directory (File. worker-dir))
+          (launch-process command :environment topology-worker-environment :log-prefix log-prefix :exit-code-callback callback :directory (File. worker-dir)))
+        )))
 
 ;; local implementation
 
@@ -771,7 +954,7 @@
 (defmethod mk-code-distributor :local [conf] nil)
 
 (defmethod launch-worker
-    :local [supervisor storm-id port worker-id]
+    :local [supervisor storm-id port worker-id mem-onheap]
     (let [conf (:conf supervisor)
           pid (uuid)
           worker (worker/mk-worker conf
@@ -789,7 +972,9 @@
   (let [conf (read-storm-config)]
     (validate-distributed-mode! conf)
     (let [supervisor (mk-supervisor conf nil supervisor)]
-      (add-shutdown-hook-with-force-kill-in-1-sec #(.shutdown supervisor)))))
+      (add-shutdown-hook-with-force-kill-in-1-sec #(.shutdown supervisor)))
+    (defgauge supervisor:num-slots-used-gauge #(count (my-worker-ids conf)))
+    (start-metrics-reporters)))
 
 (defn standalone-supervisor []
   (let [conf-atom (atom nil)
diff --git a/storm-core/src/clj/backtype/storm/daemon/task.clj b/storm-core/src/clj/backtype/storm/daemon/task.clj
index 9cf2b85..7133fdf 100644
--- a/storm-core/src/clj/backtype/storm/daemon/task.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/task.clj
@@ -18,6 +18,7 @@
   (:use [backtype.storm config util log])
   (:import [backtype.storm.hooks ITaskHook])
   (:import [backtype.storm.tuple Tuple TupleImpl])
+  (:import [backtype.storm.grouping LoadMapping])
   (:import [backtype.storm.generated SpoutSpec Bolt StateSpoutSpec StormTopology])
   (:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo
             EmitInfo BoltFailInfo BoltAckInfo])
@@ -105,7 +106,7 @@
 
 ;; TODO: this is all expensive... should be precomputed
 (defn send-unanchored
-  ([task-data stream values overflow-buffer]
+  [task-data stream values]
     (let [^TopologyContext topology-context (:system-context task-data)
           tasks-fn (:tasks-fn task-data)
           transfer-fn (-> task-data :executor-data :transfer-fn)
@@ -114,17 +115,12 @@
                                  (.getThisTaskId topology-context)
                                  stream)]
       (fast-list-iter [t (tasks-fn stream values)]
-        (transfer-fn t
-                     out-tuple
-                     overflow-buffer)
-        )))
-    ([task-data stream values]
-      (send-unanchored task-data stream values nil)
-      ))
+        (transfer-fn t out-tuple))))
 
 (defn mk-tasks-fn [task-data]
   (let [task-id (:task-id task-data)
         executor-data (:executor-data task-data)
+        ^LoadMapping load-mapping (:load-mapping (:worker executor-data))
         component-id (:component-id executor-data)
         ^WorkerTopologyContext worker-context (:worker-context executor-data)
         storm-conf (:storm-conf executor-data)
@@ -145,11 +141,9 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (builtin-metrics/emitted-tuple! (:builtin-metrics task-data) executor-stats stream)
               (stats/emitted-tuple! executor-stats stream)
               (if out-task-id
-                (stats/transferred-tuples! executor-stats stream 1)
-                (builtin-metrics/transferred-tuple! (:builtin-metrics task-data) executor-stats stream 1)))
+                (stats/transferred-tuples! executor-stats stream 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
@@ -160,7 +154,7 @@
                (when (= :direct grouper)
                   ;;  TODO: this is wrong, need to check how the stream was declared
                   (throw (IllegalArgumentException. "Cannot do regular emit to direct stream")))
-               (let [comp-tasks (grouper task-id values)]
+               (let [comp-tasks (grouper task-id values load-mapping)]
                  (if (or (sequential? comp-tasks) (instance? Collection comp-tasks))
                    (.addAll out-tasks comp-tasks)
                    (.add out-tasks comp-tasks)
@@ -168,9 +162,7 @@
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
                (stats/emitted-tuple! executor-stats stream)
-               (builtin-metrics/emitted-tuple! (:builtin-metrics task-data) executor-stats stream)              
-               (stats/transferred-tuples! executor-stats stream (count out-tasks))
-               (builtin-metrics/transferred-tuple! (:builtin-metrics task-data) executor-stats stream (count out-tasks)))
+               (stats/transferred-tuples! executor-stats stream (count out-tasks)))
              out-tasks)))
     ))
 
@@ -180,7 +172,7 @@
     :task-id task-id
     :system-context (system-topology-context (:worker executor-data) executor-data task-id)
     :user-context (user-topology-context (:worker executor-data) executor-data task-id)
-    :builtin-metrics (builtin-metrics/make-data (:type executor-data))
+    :builtin-metrics (builtin-metrics/make-data (:type executor-data) (:stats executor-data))
     :tasks-fn (mk-tasks-fn <>)
     :object (get-task-object (.getRawTopology ^TopologyContext (:system-context <>)) (:component-id executor-data))))
 
diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj
index 781a959..d978315 100644
--- a/storm-core/src/clj/backtype/storm/daemon/worker.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj
@@ -16,24 +16,32 @@
 (ns backtype.storm.daemon.worker
   (:use [backtype.storm.daemon common])
   (:use [backtype.storm config log util timer local-state])
+  (:require [clj-time.core :as time])
+  (:require [clj-time.coerce :as coerce])
   (:require [backtype.storm.daemon [executor :as executor]])
   (:require [backtype.storm [disruptor :as disruptor] [cluster :as cluster]])
   (:require [clojure.set :as set])
   (:require [backtype.storm.messaging.loader :as msg-loader])
-  (:import [java.util.concurrent Executors])
+  (:import [java.util.concurrent Executors]
+           [backtype.storm.hooks IWorkerHook BaseWorkerHook])
   (:import [java.util ArrayList HashMap])
   (:import [backtype.storm.utils Utils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue])
+  (:import [backtype.storm.grouping LoadMapping])
   (:import [backtype.storm.messaging TransportFactory])
   (:import [backtype.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
   (:import [backtype.storm.daemon Shutdownable])
   (:import [backtype.storm.serialization KryoTupleSerializer])
   (:import [backtype.storm.generated StormTopology])
-  (:import [backtype.storm.tuple Fields])
+  (:import [backtype.storm.tuple AddressedTuple Fields])
   (:import [backtype.storm.task WorkerTopologyContext])
   (:import [backtype.storm Constants])
   (:import [backtype.storm.security.auth AuthUtils])
   (:import [javax.security.auth Subject])
   (:import [java.security PrivilegedExceptionAction])
+  (:import [org.apache.logging.log4j LogManager])
+  (:import [org.apache.logging.log4j Level])
+  (:import [org.apache.logging.log4j.core.config LoggerConfig])
+  (:import [backtype.storm.generated LogConfig LogLevelAction])
   (:gen-class))
 
 (defmulti mk-suicide-fn cluster-mode)
@@ -96,10 +104,15 @@
         flatten
         set )))
 
+(defn get-dest
+  [^AddressedTuple addressed-tuple]
+  "get the destination for an AddressedTuple"
+  (.getDest addressed-tuple))
+
 (defn mk-transfer-local-fn [worker]
   (let [short-executor-receive-queue-map (:short-executor-receive-queue-map worker)
         task->short-executor (:task->short-executor worker)
-        task-getter (comp #(get task->short-executor %) fast-first)]
+        task-getter (comp #(get task->short-executor %) get-dest)]
     (fn [tuple-batch]
       (let [grouped (fast-group-by task-getter tuple-batch)]
         (fast-map-iter [[short-executor pairs] grouped]
@@ -123,12 +136,13 @@
             port (:port worker)
             storm-cluster-state (:storm-cluster-state worker)
             prev-backpressure-flag @(:backpressure worker)]
-        (if executors 
-          (if (reduce #(or %1 %2) (map #(.get-backpressure-flag %1) executors))
-            (reset! (:backpressure worker) true)   ;; at least one executor has set backpressure
-            (reset! (:backpressure worker) false))) ;; no executor has backpressure set
+        (when executors
+          (reset! (:backpressure worker)
+                  (or @(:transfer-backpressure worker)
+                      (reduce #(or %1 %2) (map #(.get-backpressure-flag %1) executors)))))
         ;; update the worker's backpressure flag to zookeeper only when it has changed
-        (if (not= prev-backpressure-flag @(:backpressure worker))
+        (log-debug "BP " @(:backpressure worker) " WAS " prev-backpressure-flag)
+        (when (not= prev-backpressure-flag @(:backpressure worker))
           (.worker-backpressure! storm-cluster-state storm-id assignment-id port @(:backpressure worker)))
         ))))
 
@@ -137,14 +151,11 @@
   check highWaterMark and lowWaterMark for backpressure"
   (disruptor/disruptor-backpressure-handler
     (fn []
-      "When worker's queue is above highWaterMark, we set its backpressure flag"
-      (if (not @(:backpressure worker))
-        (do (reset! (:backpressure worker) true)
-            (WorkerBackpressureThread/notifyBackpressureChecker (:backpressure-trigger worker)))))  ;; set backpressure no matter how the executors are
+      (reset! (:transfer-backpressure worker) true)
+      (WorkerBackpressureThread/notifyBackpressureChecker (:backpressure-trigger worker)))
     (fn []
-      "If worker's queue is below low watermark, we do nothing since we want the
-      WorkerBackPressureThread to also check for all the executors' status"
-      )))
+      (reset! (:transfer-backpressure worker) false)
+      (WorkerBackpressureThread/notifyBackpressureChecker (:backpressure-trigger worker)))))
 
 (defn mk-transfer-fn [worker]
   (let [local-tasks (-> worker :task-ids set)
@@ -155,24 +166,26 @@
 
         transfer-fn
           (fn [^KryoTupleSerializer serializer tuple-batch]
-            (let [local (ArrayList.)
-                  remoteMap (HashMap.)]
-              (fast-list-iter [[task tuple :as pair] tuple-batch]
-                (if (local-tasks task)
-                  (.add local pair)
+            (let [^ArrayList local (ArrayList.)
+                  ^HashMap remoteMap (HashMap.)]
+              (fast-list-iter [^AddressedTuple addressed-tuple tuple-batch]
+                (let [task (.getDest addressed-tuple)
+                      tuple (.getTuple addressed-tuple)]
+                  (if (local-tasks task)
+                    (.add local addressed-tuple)
 
-                  ;;Using java objects directly to avoid performance issues in java code
-                  (do
-                    (when (not (.get remoteMap task))
-                      (.put remoteMap task (ArrayList.)))
-                    (let [remote (.get remoteMap task)]
-                      (if (not-nil? task)
-                        (.add remote (TaskMessage. task (.serialize serializer tuple)))
-                        (log-warn "Can't transfer tuple - task value is nil. tuple type: " (pr-str (type tuple)) " and information: " (pr-str tuple)))
-                     ))))
+                    ;;Using java objects directly to avoid performance issues in java code
+                    (do
+                      (when (not (.get remoteMap task))
+                        (.put remoteMap task (ArrayList.)))
+                      (let [^ArrayList remote (.get remoteMap task)]
+                        (if (not-nil? task)
+                          (.add remote (TaskMessage. task ^bytes (.serialize serializer tuple)))
+                          (log-warn "Can't transfer tuple - task value is nil. tuple type: " (pr-str (type tuple)) " and information: " (pr-str tuple)))
+                       )))))
 
-              (local-transfer local)
-              (disruptor/publish transfer-queue remoteMap)))]
+              (when (not (.isEmpty local)) (local-transfer local))
+              (when (not (.isEmpty remoteMap)) (disruptor/publish transfer-queue remoteMap))))]
     (if try-serialize-local
       (do
         (log-warn "WILL TRY TO SERIALIZE ALL TUPLES (Turn off " TOPOLOGY-TESTING-ALWAYS-TRY-SERIALIZE " for production)")
@@ -187,7 +200,8 @@
        (map (fn [e] [e (disruptor/disruptor-queue (str "receive-queue" e)
                                                   (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                                  :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))]))
+                                                  :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
+                                                  :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))]))
        (into {})
        ))
 
@@ -229,7 +243,8 @@
         executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
         transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                                  :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))
+                                                  :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
+                                                  :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
         executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
 
         receive-queue-map (->> executor-receive-queue-map
@@ -263,8 +278,10 @@
       :topology topology
       :system-topology (system-topology! storm-conf topology)
       :heartbeat-timer (mk-halting-timer "heartbeat-timer")
+      :refresh-load-timer (mk-halting-timer "refresh-load-timer")
       :refresh-connections-timer (mk-halting-timer "refresh-connections-timer")
       :refresh-credentials-timer (mk-halting-timer "refresh-credentials-timer")
+      :reset-log-levels-timer (mk-halting-timer "reset-log-levels-timer")
       :refresh-active-timer (mk-halting-timer "refresh-active-timer")
       :executor-heartbeat-timer (mk-halting-timer "executor-heartbeat-timer")
       :user-timer (mk-halting-timer "user-timer")
@@ -286,10 +303,11 @@
       :default-shared-resources (mk-default-resources <>)
       :user-shared-resources (mk-user-resources <>)
       :transfer-local-fn (mk-transfer-local-fn <>)
-      :receiver-thread-count (get storm-conf WORKER-RECEIVER-THREAD-COUNT)
       :transfer-fn (mk-transfer-fn <>)
+      :load-mapping (LoadMapping.)
       :assignment-versions assignment-versions
       :backpressure (atom false) ;; whether this worker is going slow
+      :transfer-backpressure (atom false) ;; if the transfer queue is backed-up
       :backpressure-trigger (atom false) ;; a trigger for synchronization with executors
       :throttle-on (atom false) ;; whether throttle is activated for spouts
       )))
@@ -302,6 +320,28 @@
     [node (Integer/valueOf port-str)]
     ))
 
+(def LOAD-REFRESH-INTERVAL-MS 5000)
+
+(defn mk-refresh-load [worker]
+  (let [local-tasks (set (:task-ids worker))
+        remote-tasks (set/difference (worker-outbound-tasks worker) local-tasks)
+        short-executor-receive-queue-map (:short-executor-receive-queue-map worker)
+        next-update (atom 0)]
+    (fn this
+      ([]
+        (let [^LoadMapping load-mapping (:load-mapping worker)
+              local-pop (map-val (fn [queue]
+                                   (let [q-metrics (.getMetrics queue)]
+                                     (/ (double (.population q-metrics)) (.capacity q-metrics))))
+                                 short-executor-receive-queue-map)
+              remote-load (reduce merge (for [[np conn] @(:cached-node+port->socket worker)] (into {} (.getLoad conn remote-tasks))))
+              now (System/currentTimeMillis)]
+          (.setLocal load-mapping local-pop)
+          (.setRemote load-mapping remote-load)
+          (when (> now @next-update)
+            (.sendLoadMetrics (:receiver worker) local-pop)
+            (reset! next-update (+ LOAD-REFRESH-INTERVAL-MS now))))))))
+
 (defn mk-refresh-connections [worker]
   (let [outbound-tasks (worker-outbound-tasks worker)
         conf (:conf worker)
@@ -417,16 +457,12 @@
           (schedule timer recur-secs this :check-active false)
             )))))
 
-(defn launch-receive-thread [worker]
-  (log-message "Launching receive-thread for " (:assignment-id worker) ":" (:port worker))
-  (msg-loader/launch-receive-thread!
-    (:mq-context worker)
-    (:receiver worker)
-    (:storm-id worker)
-    (:receiver-thread-count worker)
-    (:port worker)
-    (:transfer-local-fn worker)
-    :kill-fn (fn [t] (exit-process! 11))))
+(defn register-callbacks [worker]
+  (log-message "Registering IConnectionCallbacks for " (:assignment-id worker) ":" (:port worker))
+  (msg-loader/register-callback (:transfer-local-fn worker)
+                                (:receiver worker)
+                                (:storm-conf worker)
+                                (worker-context worker)))
 
 (defn- close-resources [worker]
   (let [dr (:default-shared-resources worker)]
@@ -434,10 +470,102 @@
     (.shutdownNow (get dr WorkerTopologyContext/SHARED_EXECUTOR))
     (log-message "Shut down default resources")))
 
-(defn- override-login-config-with-system-property [conf]
-  (if-let [login_conf_file (System/getProperty "java.security.auth.login.config")]
-    (assoc conf "java.security.auth.login.config" login_conf_file)
-    conf))
+(defn- get-logger-levels []
+  (into {}
+    (let [logger-config (.getConfiguration (LogManager/getContext false))]
+      (for [[logger-name logger] (.getLoggers logger-config)]
+        {logger-name (.getLevel logger)}))))
+
+(defn set-logger-level [logger-context logger-name new-level]
+  (let [config (.getConfiguration logger-context)
+        logger-config (.getLoggerConfig config logger-name)]
+    (if (not (= (.getName logger-config) logger-name))
+      ;; create a new config. Make it additive (true) s.t. inherit
+      ;; parents appenders
+      (let [new-logger-config (LoggerConfig. logger-name new-level true)]
+        (log-message "Adding config for: " new-logger-config " with level: " new-level)
+        (.addLogger config logger-name new-logger-config))
+      (do
+        (log-message "Setting " logger-config " log level to: " new-level)
+        (.setLevel logger-config new-level)))))
+
+;; function called on timer to reset log levels last set to DEBUG
+;; also called from process-log-config-change
+(defn reset-log-levels [latest-log-config-atom]
+  (let [latest-log-config @latest-log-config-atom
+        logger-context (LogManager/getContext false)]
+    (doseq [[logger-name logger-setting] (sort latest-log-config)]
+      (let [timeout (:timeout logger-setting)
+            target-log-level (:target-log-level logger-setting)
+            reset-log-level (:reset-log-level logger-setting)]
+        (when (> (coerce/to-long (time/now)) timeout)
+          (log-message logger-name ": Resetting level to " reset-log-level) 
+          (set-logger-level logger-context logger-name reset-log-level)
+          (swap! latest-log-config-atom
+            (fn [prev]
+              (dissoc prev logger-name))))))
+    (.updateLoggers logger-context)))
+
+;; when a new log level is received from zookeeper, this function is called
+(defn process-log-config-change [latest-log-config original-log-levels log-config]
+  (when log-config
+    (log-debug "Processing received log config: " log-config)
+    ;; merge log configs together
+    (let [loggers (.get_named_logger_level log-config)
+          logger-context (LogManager/getContext false)]
+      (def new-log-configs
+        (into {}
+         ;; merge named log levels
+         (for [[msg-logger-name logger-level] loggers]
+           (let [logger-name (if (= msg-logger-name "ROOT")
+                                LogManager/ROOT_LOGGER_NAME
+                                msg-logger-name)]
+             ;; the new-timeouts map now contains logger => timeout 
+             (when (.is_set_reset_log_level_timeout_epoch logger-level)
+               {logger-name {:action (.get_action logger-level)
+                             :target-log-level (Level/toLevel (.get_target_log_level logger-level))
+                             :reset-log-level (or (.get @original-log-levels logger-name) (Level/INFO))
+                             :timeout (.get_reset_log_level_timeout_epoch logger-level)}})))))
+
+      ;; look for deleted log timeouts
+      (doseq [[logger-name logger-val] (sort @latest-log-config)]
+        (when (not (contains? new-log-configs logger-name))
+          ;; if we had a timeout, but the timeout is no longer active
+          (set-logger-level
+            logger-context logger-name (:reset-log-level logger-val))))
+
+      ;; apply new log settings we just received
+      ;; the merged configs are only for the reset logic
+      (doseq [[msg-logger-name logger-level] (sort (into {} (.get_named_logger_level log-config)))]
+        (let [logger-name (if (= msg-logger-name "ROOT")
+                                LogManager/ROOT_LOGGER_NAME
+                                msg-logger-name)
+              level (Level/toLevel (.get_target_log_level logger-level))
+              action (.get_action logger-level)]
+          (if (= action LogLevelAction/UPDATE)
+            (set-logger-level logger-context logger-name level))))
+   
+      (.updateLoggers logger-context)
+      (reset! latest-log-config new-log-configs)
+      (log-debug "New merged log config is " @latest-log-config))))
+
+(defn run-worker-start-hooks [worker]
+  (let [topology (:topology worker)
+        topo-conf (:storm-conf worker)
+        worker-topology-context (worker-context worker)
+        hooks (.get_worker_hooks topology)]
+    (dofor [hook hooks]
+      (let [hook-bytes (Utils/toByteArray hook)
+            deser-hook (Utils/javaDeserialize hook-bytes BaseWorkerHook)]
+        (.start deser-hook topo-conf worker-topology-context)))))
+
+(defn run-worker-shutdown-hooks [worker]
+  (let [topology (:topology worker)
+        hooks (.get_worker_hooks topology)]
+    (dofor [hook hooks]
+      (let [hook-bytes (Utils/toByteArray hook)
+            deser-hook (Utils/javaDeserialize hook-bytes BaseWorkerHook)]
+        (.shutdown deser-hook)))))
 
 ;; TODO: should worker even take the storm-id as input? this should be
 ;; deducable from cluster state (by searching through assignments)
@@ -451,7 +579,16 @@
   ;; because in local mode, its not a separate
   ;; process. supervisor will register it in this case
   (when (= :distributed (cluster-mode conf))
-    (touch (worker-pid-path conf worker-id (process-pid))))
+    (let [pid (process-pid)]
+      (touch (worker-pid-path conf worker-id pid))
+      (spit (worker-artifacts-pid-path conf storm-id port) pid)))
+
+  (declare establish-log-setting-callback)
+
+  ;; start out with empty list of timeouts 
+  (def latest-log-config (atom {}))
+  (def original-log-levels (atom {}))
+
   (let [storm-conf (read-supervisor-storm-conf conf storm-id)
         storm-conf (override-login-config-with-system-property storm-conf)
         acls (Utils/getWorkerACL storm-conf)
@@ -475,9 +612,10 @@
         _ (schedule-recurring (:heartbeat-timer worker) 0 (conf WORKER-HEARTBEAT-FREQUENCY-SECS) heartbeat-fn)
         _ (schedule-recurring (:executor-heartbeat-timer worker) 0 (conf TASK-HEARTBEAT-FREQUENCY-SECS) #(do-executor-heartbeats worker :executors @executors))
 
-        receive-thread-shutdown (launch-receive-thread worker)
+        _ (register-callbacks worker)
 
         refresh-connections (mk-refresh-connections worker)
+        refresh-load (mk-refresh-load worker)
 
         _ (refresh-connections nil)
 
@@ -485,6 +623,7 @@
 
         _ (refresh-storm-active worker nil)
 
+        _ (run-worker-start-hooks worker)
 
         _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e initial-credentials)))
 
@@ -513,9 +652,6 @@
                       ;; this will do best effort flushing since the linger period
                       ;; was set on creation
                       (.close socket))
-                    (log-message "Shutting down receive thread")
-                    (receive-thread-shutdown)
-                    (log-message "Shut down receive thread")
                     (log-message "Terminating messaging context")
                     (log-message "Shutting down executors")
                     (doseq [executor @executors] (.shutdown executor))
@@ -539,10 +675,12 @@
                     (cancel-timer (:refresh-active-timer worker))
                     (cancel-timer (:executor-heartbeat-timer worker))
                     (cancel-timer (:user-timer worker))
+                    (cancel-timer (:refresh-load-timer worker))
 
                     (close-resources worker)
 
-                    ;; TODO: here need to invoke the "shutdown" method of WorkerHook
+                    (log-message "Trigger any worker shutdown hooks")
+                    (run-worker-shutdown-hooks worker)
 
                     (.remove-worker-heartbeat! (:storm-cluster-state worker) storm-id assignment-id port)
                     (log-message "Disconnecting from storm cluster state context")
@@ -559,6 +697,7 @@
                (and
                  (timer-waiting? (:heartbeat-timer worker))
                  (timer-waiting? (:refresh-connections-timer worker))
+                 (timer-waiting? (:refresh-load-timer worker))
                  (timer-waiting? (:refresh-credentials-timer worker))
                  (timer-waiting? (:refresh-active-timer worker))
                  (timer-waiting? (:executor-heartbeat-timer worker))
@@ -577,14 +716,29 @@
                                                  (let [throttle-on (.topology-backpressure (:storm-cluster-state worker) storm-id cb)]
                                                    (reset! (:throttle-on worker) throttle-on)))
                                       new-throttle-on (.topology-backpressure (:storm-cluster-state worker) storm-id callback)]
-                                    (reset! (:throttle-on worker) new-throttle-on)))]
+                                    (reset! (:throttle-on worker) new-throttle-on)))
+        check-log-config-changed (fn []
+                                  (let [log-config (.topology-log-config (:storm-cluster-state worker) storm-id nil)]
+                                    (process-log-config-change latest-log-config original-log-levels log-config)
+                                    (establish-log-setting-callback)))]
+    (reset! original-log-levels (get-logger-levels))
+    (log-message "Started with log levels: " @original-log-levels)
+  
+    (defn establish-log-setting-callback []
+      (.topology-log-config (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
+
+    (establish-log-setting-callback)
     (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed)))
     (schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
                         (fn [& args]
                           (check-credentials-changed)
                           (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
                             (check-throttle-changed))))
+    ;; The jitter allows the clients to get the data at different times, and avoids thundering herd
+    (when-not (.get conf TOPOLOGY-DISABLE-LOADAWARE-MESSAGING)
+      (schedule-recurring-with-jitter (:refresh-load-timer worker) 0 1 500 refresh-load))
     (schedule-recurring (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections)
+    (schedule-recurring (:reset-log-levels-timer worker) 0 (conf WORKER-LOG-LEVEL-RESET-POLL-SECS) (fn [] (reset-log-levels latest-log-config)))
     (schedule-recurring (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
 
     (log-message "Worker has topology config " (redact-value (:storm-conf worker) STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
@@ -605,4 +759,4 @@
     (setup-default-uncaught-exception-handler)
     (validate-distributed-mode! conf)
     (let [worker (mk-worker conf nil storm-id assignment-id (Integer/parseInt port-str) worker-id)]
-      (add-shutdown-hook-with-force-kill-in-1-sec #(.shutdown worker)))))
+      (add-shutdown-hook-with-force-kill-in-1-sec #(.shutdown worker)))))
\ No newline at end of file
diff --git a/storm-core/src/clj/backtype/storm/disruptor.clj b/storm-core/src/clj/backtype/storm/disruptor.clj
index 25e0050..bbfe048 100644
--- a/storm-core/src/clj/backtype/storm/disruptor.clj
+++ b/storm-core/src/clj/backtype/storm/disruptor.clj
@@ -16,39 +16,21 @@
 
 (ns backtype.storm.disruptor
   (:import [backtype.storm.utils DisruptorQueue WorkerBackpressureCallback DisruptorBackpressureCallback])
-  (:import [com.lmax.disruptor MultiThreadedClaimStrategy SingleThreadedClaimStrategy
-            BlockingWaitStrategy SleepingWaitStrategy YieldingWaitStrategy
-            BusySpinWaitStrategy])
+  (:import [com.lmax.disruptor.dsl ProducerType])
   (:require [clojure [string :as str]])
   (:require [clojure [set :as set]])
   (:use [clojure walk])
   (:use [backtype.storm util log]))
 
-(def CLAIM-STRATEGY
-  {:multi-threaded (fn [size] (MultiThreadedClaimStrategy. (int size)))
-   :single-threaded (fn [size] (SingleThreadedClaimStrategy. (int size)))})
+(def PRODUCER-TYPE
+  {:multi-threaded ProducerType/MULTI
+   :single-threaded ProducerType/SINGLE})
 
-(def WAIT-STRATEGY
-  {:block (fn [] (BlockingWaitStrategy.))
-   :yield (fn [] (YieldingWaitStrategy.))
-   :sleep (fn [] (SleepingWaitStrategy.))
-   :spin (fn [] (BusySpinWaitStrategy.))})
-
-(defn- mk-wait-strategy
-  [spec]
-  (if (keyword? spec)
-    ((WAIT-STRATEGY spec))
-    (-> (str spec) new-instance)))
-
-;; :block strategy requires using a timeout on waitFor (implemented in DisruptorQueue), as sometimes the consumer stays blocked even when there's an item on the queue.
-;; This would manifest itself in Trident when doing 1 batch at a time processing, and the ack_init message
-;; wouldn't make it to the acker until the batch timed out and another tuple was played into the queue,
-;; unblocking the consumer
 (defnk disruptor-queue
-  [^String queue-name buffer-size timeout :claim-strategy :multi-threaded :wait-strategy :block]
+  [^String queue-name buffer-size timeout :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
   (DisruptorQueue. queue-name
-                   ((CLAIM-STRATEGY claim-strategy) buffer-size)
-                   (mk-wait-strategy wait-strategy) timeout))
+                   (PRODUCER-TYPE producer-type) buffer-size
+                   timeout batch-size batch-timeout))
 
 (defn clojure-handler
   [afn]
@@ -79,14 +61,8 @@
   `(clojure-handler (fn ~@args)))
 
 (defn publish
-  ([^DisruptorQueue q o block?]
-   (.publish q o block?))
-  ([q o]
-   (publish q o true)))
-
-(defn try-publish
   [^DisruptorQueue q o]
-  (.tryPublish q o))
+  (.publish q o))
 
 (defn consume-batch
   [^DisruptorQueue queue handler]
@@ -96,10 +72,6 @@
   [^DisruptorQueue queue handler]
   (.consumeBatchWhenAvailable queue handler))
 
-(defn consumer-started!
-  [^DisruptorQueue queue]
-  (.consumerStarted queue))
-
 (defn halt-with-interrupt!
   [^DisruptorQueue queue]
   (.haltWithInterrupt queue))
@@ -107,11 +79,10 @@
 (defnk consume-loop*
   [^DisruptorQueue queue handler
    :kill-fn (fn [error] (exit-process! 1 "Async loop died!"))]
-  (let [ret (async-loop
-              (fn [] (consume-batch-when-available queue handler) 0)
-              :kill-fn kill-fn
-              :thread-name (.getName queue))]
-     (consumer-started! queue) ret))
+  (async-loop
+          (fn [] (consume-batch-when-available queue handler) 0)
+          :kill-fn kill-fn
+          :thread-name (.getName queue)))
 
 (defmacro consume-loop [queue & handler-args]
   `(let [handler# (handler ~@handler-args)]
diff --git a/storm-core/src/clj/backtype/storm/local_state.clj b/storm-core/src/clj/backtype/storm/local_state.clj
index 41e3675..bf9567d 100644
--- a/storm-core/src/clj/backtype/storm/local_state.clj
+++ b/storm-core/src/clj/backtype/storm/local_state.clj
@@ -19,13 +19,38 @@
             InvalidTopologyException GlobalStreamId
             LSSupervisorId LSApprovedWorkers
             LSSupervisorAssignments LocalAssignment
-            ExecutorInfo LSWorkerHeartbeat])
+            ExecutorInfo LSWorkerHeartbeat
+            LSTopoHistory LSTopoHistoryList
+            WorkerResources])
   (:import [backtype.storm.utils LocalState]))
 
 (def LS-WORKER-HEARTBEAT "worker-heartbeat")
 (def LS-ID "supervisor-id")
 (def LS-LOCAL-ASSIGNMENTS "local-assignments")
 (def LS-APPROVED-WORKERS "approved-workers")
+(def LS-TOPO-HISTORY "topo-hist")
+
+(defn ->LSTopoHistory
+  [{topoid :topoid timestamp :timestamp users :users groups :groups}]
+  (LSTopoHistory. topoid timestamp users groups))
+
+(defn ->topo-history
+  [thrift-topo-hist]
+  {
+    :topoid (.get_topology_id thrift-topo-hist)
+    :timestamp (.get_time_stamp thrift-topo-hist)
+    :users (.get_users thrift-topo-hist)
+    :groups (.get_groups thrift-topo-hist)})
+
+(defn ls-topo-hist!
+  [^LocalState local-state hist-list]
+  (.put local-state LS-TOPO-HISTORY
+    (LSTopoHistoryList. (map ->LSTopoHistory hist-list))))
+
+(defn ls-topo-hist
+  [^LocalState local-state]
+  (if-let [thrift-hist-list (.get local-state LS-TOPO-HISTORY)]
+    (map ->topo-history (.get_topo_history thrift-hist-list))))
 
 (defn ls-supervisor-id!
   [^LocalState local-state ^String id]
@@ -59,18 +84,25 @@
       [(.get_task_start exec-info) (.get_task_end exec-info)])))
 
 (defn ->LocalAssignment
-  [{storm-id :storm-id executors :executors}]
-  (LocalAssignment. storm-id (->ExecutorInfo-list executors)))
+  [{storm-id :storm-id executors :executors resources :resources}]
+  (let [assignment (LocalAssignment. storm-id (->ExecutorInfo-list executors))]
+    (if resources (.set_resources assignment
+                                  (doto (WorkerResources. )
+                                    (.set_mem_on_heap (first resources))
+                                    (.set_mem_off_heap (second resources))
+                                    (.set_cpu (last resources)))))
+    assignment))
 
 (defn mk-local-assignment
-  [storm-id executors]
-  {:storm-id storm-id :executors executors})
+  [storm-id executors resources]
+  {:storm-id storm-id :executors executors :resources resources})
 
 (defn ->local-assignment
   [^LocalAssignment thrift-local-assignment]
     (mk-local-assignment
       (.get_topology_id thrift-local-assignment)
-      (->executor-list (.get_executors thrift-local-assignment))))
+      (->executor-list (.get_executors thrift-local-assignment))
+      (.get_resources thrift-local-assignment)))
 
 (defn ls-local-assignments!
   [^LocalState local-state assignments]
diff --git a/storm-core/src/clj/backtype/storm/log.clj b/storm-core/src/clj/backtype/storm/log.clj
index 0fcf822..abe9b32 100644
--- a/storm-core/src/clj/backtype/storm/log.clj
+++ b/storm-core/src/clj/backtype/storm/log.clj
@@ -15,7 +15,9 @@
 ;; limitations under the License.
 
 (ns backtype.storm.log
-  (:require [clojure.tools [logging :as log]]))
+  (:require [clojure.tools.logging :as log])
+  (:use [clojure pprint])
+  (:import [java.io StringWriter]))
 
 (defmacro log-message
   [& args]
@@ -44,3 +46,11 @@
 (defn log-stream
   [& args]
   (apply log/log-stream args))
+
+(defmacro log-pprint
+  [& args]
+  `(let [^StringWriter writer# (StringWriter.)]
+     (doall
+       (for [object# [~@args]]
+         (pprint object# writer#)))
+     (log-message "\n" writer#)))
diff --git a/storm-core/src/clj/backtype/storm/messaging/loader.clj b/storm-core/src/clj/backtype/storm/messaging/loader.clj
index c154ed8..72dd382 100644
--- a/storm-core/src/clj/backtype/storm/messaging/loader.clj
+++ b/storm-core/src/clj/backtype/storm/messaging/loader.clj
@@ -14,71 +14,21 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.messaging.loader
-  (:use [backtype.storm util log])
-  (:import [java.util ArrayList Iterator])
-  (:import [backtype.storm.messaging IContext IConnection TaskMessage])
-  (:import [backtype.storm.utils DisruptorQueue MutableObject])
-  (:require [backtype.storm.messaging [local :as local]])
-  (:require [backtype.storm [disruptor :as disruptor]]))
+  (:import [backtype.storm.messaging IConnection DeserializingConnectionCallback])
+  (:require [backtype.storm.messaging [local :as local]]))
 
 (defn mk-local-context []
   (local/mk-context))
 
-(defn- mk-receive-thread [storm-id port transfer-local-fn  daemon kill-fn priority socket thread-id]
-    (async-loop
-       (fn []
-         (log-message "Starting receive-thread: [stormId: " storm-id ", port: " port ", thread-id: " thread-id  " ]")
-         (fn []
-           (let [batched (ArrayList.)
-                 ^Iterator iter (.recv ^IConnection socket 0 thread-id)
-                 closed (atom false)]
-             (when iter
-               (while (and (not @closed) (.hasNext iter)) 
-                  (let [packet (.next iter)
-                        task (if packet (.task ^TaskMessage packet))
-                        message (if packet (.message ^TaskMessage packet))]
-                      (if (= task -1)
-                         (do (log-message "Receiving-thread:[" storm-id ", " port "] received shutdown notice")
-                           (.close socket)
-                           (reset! closed  true))
-                         (when packet (.add batched [task message]))))))
-             
-             (when (not @closed)
-               (do
-                 (if (> (.size batched) 0)
-                   (transfer-local-fn batched))
-                 0)))))
-         :factory? true
-         :daemon daemon
-         :kill-fn kill-fn
-         :priority priority
-         :thread-name (str "worker-receiver-thread-" thread-id)))
+(defn- mk-connection-callback
+  "make an IConnectionCallback"
+  [transfer-local-fn storm-conf worker-context]
+  (DeserializingConnectionCallback. storm-conf
+                                    worker-context
+                                    (fn [batch]
+                                      (transfer-local-fn batch))))
 
-(defn- mk-receive-threads [storm-id port transfer-local-fn  daemon kill-fn priority socket thread-count]
-  (into [] (for [thread-id (range thread-count)] 
-             (mk-receive-thread storm-id port transfer-local-fn  daemon kill-fn priority socket thread-id))))
-
-
-(defnk launch-receive-thread!
-  [context socket storm-id receiver-thread-count port transfer-local-fn
-   :daemon true
-   :kill-fn (fn [t] (System/exit 1))
-   :priority Thread/NORM_PRIORITY]
-  (let [local-hostname (memoized-local-hostname)
-        thread-count (if receiver-thread-count receiver-thread-count 1)
-        vthreads (mk-receive-threads storm-id port transfer-local-fn daemon kill-fn priority socket thread-count)]
-    (fn []
-      (let [kill-socket (.connect ^IContext context storm-id local-hostname port)]
-        (log-message "Shutting down receiving-thread: [" storm-id ", " port "]")
-        (.send ^IConnection kill-socket
-                  -1 (byte-array []))
-        
-        (.close ^IConnection kill-socket)
-        
-        (log-message "Waiting for receiving-thread:[" storm-id ", " port "] to die")
-        
-        (for [thread-id (range thread-count)] 
-             (.join (vthreads thread-id)))
-        
-        (log-message "Shutdown receiving-thread: [" storm-id ", " port "]")
-        ))))
+(defn register-callback
+  "register the local-transfer-fn with the server"
+  [transfer-local-fn ^IConnection socket storm-conf worker-context]
+  (.registerRecv socket (mk-connection-callback transfer-local-fn storm-conf worker-context)))
diff --git a/storm-core/src/clj/backtype/storm/messaging/local.clj b/storm-core/src/clj/backtype/storm/messaging/local.clj
index 4aa67ab..b99a77a 100644
--- a/storm-core/src/clj/backtype/storm/messaging/local.clj
+++ b/storm-core/src/clj/backtype/storm/messaging/local.clj
@@ -14,60 +14,10 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns backtype.storm.messaging.local
-  (:refer-clojure :exclude [send])
-  (:use [backtype.storm log])
-  (:import [backtype.storm.messaging IContext IConnection TaskMessage])
-  (:import [java.util.concurrent LinkedBlockingQueue])
-  (:import [java.util Map Iterator])
-  (:import [java.util Iterator ArrayList])
-  (:gen-class))
-
-(defn add-queue! [queues-map lock storm-id port]
-  (let [id (str storm-id "-" port)]
-    (locking lock
-      (when-not (contains? @queues-map id)
-        (swap! queues-map assoc id (LinkedBlockingQueue.))))
-    (@queues-map id)))
-
-(deftype LocalConnection [storm-id port queues-map lock queue]
-  IConnection
-  (^Iterator recv [this ^int flags ^int clientId]
-    (when-not queue
-      (throw (IllegalArgumentException. "Cannot receive on this socket")))
-    (let [ret (ArrayList.)
-          msg (if (= flags 1) (.poll queue) (.take queue))]
-      (if msg
-        (do 
-          (.add ret msg)
-          (.iterator ret))
-        nil)))
-  (^void send [this ^int taskId ^bytes payload]
-    (let [send-queue (add-queue! queues-map lock storm-id port)]
-      (.put send-queue (TaskMessage. taskId payload))
-      ))
-  (^void send [this ^Iterator iter]
-    (let [send-queue (add-queue! queues-map lock storm-id port)]
-      (while (.hasNext iter) 
-         (.put send-queue (.next iter)))
-      ))
-  (^void close [this]
-    ))
-
-
-(deftype LocalContext [^{:unsynchronized-mutable true} queues-map
-                       ^{:unsynchronized-mutable true} lock]
-  IContext
-  (^void prepare [this ^Map storm-conf]
-    (set! queues-map (atom {}))
-    (set! lock (Object.)))
-  (^IConnection bind [this ^String storm-id ^int port]
-    (LocalConnection. storm-id port queues-map lock (add-queue! queues-map lock storm-id port)))
-  (^IConnection connect [this ^String storm-id ^String host ^int port]
-    (LocalConnection. storm-id port queues-map lock nil))
-  (^void term [this]
-    ))
+  (:import [backtype.storm.messaging IContext])
+  (:import [backtype.storm.messaging.local Context]))
 
 (defn mk-context [] 
-  (let [context  (LocalContext. nil nil)]
+  (let [context  (Context.)]
     (.prepare ^IContext context nil)
     context))
diff --git a/storm-core/src/clj/backtype/storm/stats.clj b/storm-core/src/clj/backtype/storm/stats.clj
index db8930c..ea4efe4 100644
--- a/storm-core/src/clj/backtype/storm/stats.clj
+++ b/storm-core/src/clj/backtype/storm/stats.clj
@@ -17,236 +17,144 @@
 (ns backtype.storm.stats
   (:import [backtype.storm.generated Nimbus Nimbus$Processor Nimbus$Iface StormTopology ShellComponent
             NotAliveException AlreadyAliveException InvalidTopologyException GlobalStreamId
-            ClusterSummary TopologyInfo TopologySummary ExecutorSummary ExecutorStats ExecutorSpecificStats
-            SpoutStats BoltStats ErrorInfo SupervisorSummary])
-  (:use [backtype.storm util log])
+            ClusterSummary TopologyInfo TopologySummary ExecutorInfo ExecutorSummary ExecutorStats
+            ExecutorSpecificStats SpoutStats BoltStats ErrorInfo
+            SupervisorSummary CommonAggregateStats ComponentAggregateStats
+            ComponentPageInfo ComponentType BoltAggregateStats
+            ExecutorAggregateStats SpecificAggregateStats
+            SpoutAggregateStats TopologyPageInfo TopologyStats])
+  (:import [backtype.storm.utils Utils])
+  (:import [backtype.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
+  (:use [backtype.storm log util])
   (:use [clojure.math.numeric-tower :only [ceil]]))
 
-;;TODO: consider replacing this with some sort of RRD
-
-(defn curr-time-bucket
-  [^Integer time-secs ^Integer bucket-size-secs]
-  (* bucket-size-secs (unchecked-divide-int time-secs bucket-size-secs)))
-
-(defrecord RollingWindow
-  [updater merger extractor bucket-size-secs num-buckets buckets])
-
-(defn rolling-window
-  [updater merger extractor bucket-size-secs num-buckets]
-  (RollingWindow. updater merger extractor bucket-size-secs num-buckets {}))
-
-(defn update-rolling-window
-  ([^RollingWindow rw time-secs & args]
-   ;; this is 2.5x faster than using update-in...
-   (let [time-bucket (curr-time-bucket time-secs (:bucket-size-secs rw))
-         buckets (:buckets rw)
-         curr (get buckets time-bucket)
-         curr (apply (:updater rw) curr args)]
-     (assoc rw :buckets (assoc buckets time-bucket curr)))))
-
-(defn value-rolling-window
-  [^RollingWindow rw]
-  ((:extractor rw)
-   (let [values (vals (:buckets rw))]
-     (apply (:merger rw) values))))
-
-(defn cleanup-rolling-window
-  [^RollingWindow rw]
-  (let [buckets (:buckets rw)
-        cutoff (- (current-time-secs)
-                  (* (:num-buckets rw)
-                     (:bucket-size-secs rw)))
-        to-remove (filter #(< % cutoff) (keys buckets))
-        buckets (apply dissoc buckets to-remove)]
-    (assoc rw :buckets buckets)))
-
-(defn rolling-window-size
-  [^RollingWindow rw]
-  (* (:bucket-size-secs rw) (:num-buckets rw)))
-
-(defrecord RollingWindowSet [updater extractor windows all-time])
-
-(defn rolling-window-set [updater merger extractor num-buckets & bucket-sizes]
-  (RollingWindowSet. updater extractor (dofor [s bucket-sizes] (rolling-window updater merger extractor s num-buckets)) nil)
-  )
-
-(defn update-rolling-window-set
-  ([^RollingWindowSet rws & args]
-   (let [now (current-time-secs)
-         new-windows (dofor [w (:windows rws)]
-                            (apply update-rolling-window w now args))]
-     (assoc rws
-       :windows new-windows
-       :all-time (apply (:updater rws) (:all-time rws) args)))))
-
-(defn cleanup-rolling-window-set
-  ([^RollingWindowSet rws]
-   (let [windows (:windows rws)]
-     (assoc rws :windows (map cleanup-rolling-window windows)))))
-
-(defn value-rolling-window-set
-  [^RollingWindowSet rws]
-  (merge
-    (into {}
-          (for [w (:windows rws)]
-            {(rolling-window-size w) (value-rolling-window w)}
-            ))
-    {:all-time ((:extractor rws) (:all-time rws))}))
-
-(defn- incr-val
-  ([amap key]
-   (incr-val amap key 1))
-  ([amap key amt]
-   (let [val (get amap key (long 0))]
-     (assoc amap key (+ val amt)))))
-
-(defn- update-avg
-  [curr val]
-  (if curr
-    [(+ (first curr) val) (inc (second curr))]
-    [val (long 1)]))
-
-(defn- merge-avg
-  [& avg]
-  [(apply + (map first avg))
-   (apply + (map second avg))
-   ])
-
-(defn- extract-avg
-  [pair]
-  (double (/ (first pair) (second pair))))
-
-(defn- update-keyed-avg
-  [amap key val]
-  (assoc amap key (update-avg (get amap key) val)))
-
-(defn- merge-keyed-avg [& vals]
-  (apply merge-with merge-avg vals))
-
-(defn- extract-keyed-avg [vals]
-  (map-val extract-avg vals))
-
-(defn- counter-extract [v]
-  (if v v {}))
-
-(defn keyed-counter-rolling-window-set
-  [num-buckets & bucket-sizes]
-  (apply rolling-window-set incr-val (partial merge-with +) counter-extract num-buckets bucket-sizes))
-
-(defn avg-rolling-window-set
-  [num-buckets & bucket-sizes]
-  (apply rolling-window-set update-avg merge-avg extract-avg num-buckets bucket-sizes))
-
-(defn keyed-avg-rolling-window-set
-  [num-buckets & bucket-sizes]
-  (apply rolling-window-set update-keyed-avg merge-keyed-avg extract-keyed-avg num-buckets bucket-sizes))
-
-;; (defn choose-bucket [val buckets]
-;;   (let [ret (find-first #(<= val %) buckets)]
-;;     (if ret
-;;       ret
-;;       (* 10 (first buckets)))
-;;     ))
-
-;; ;; buckets must be between 1 and 9
-;; (defn to-proportional-bucket
-;;   "Maps to a bucket in the values order of magnitude. So if buckets are [1 2 5],
-;;    3 -> 5
-;;    7 -> 10
-;;    1234 -> 2000
-;;    etc."
-;;   [val buckets]
-;;   (cond (= 0 val) 0
-;;         (between? val 1 9) (choose-bucket val buckets)
-;;         :else (* 10 (to-proportional-bucket (ceil (/ val 10))
-;;                                             buckets))))
+(def TEN-MIN-IN-SECONDS (* 10 60))
 
 (def COMMON-FIELDS [:emitted :transferred])
-(defrecord CommonStats [emitted transferred rate])
+(defrecord CommonStats [^MultiCountStatAndMetric emitted
+                        ^MultiCountStatAndMetric transferred
+                        rate])
 
 (def BOLT-FIELDS [:acked :failed :process-latencies :executed :execute-latencies])
 ;;acked and failed count individual tuples
-(defrecord BoltExecutorStats [common acked failed process-latencies executed execute-latencies])
+(defrecord BoltExecutorStats [^CommonStats common
+                              ^MultiCountStatAndMetric acked
+                              ^MultiCountStatAndMetric failed
+                              ^MultiLatencyStatAndMetric process-latencies
+                              ^MultiCountStatAndMetric executed
+                              ^MultiLatencyStatAndMetric execute-latencies])
 
 (def SPOUT-FIELDS [:acked :failed :complete-latencies])
 ;;acked and failed count tuple completion
-(defrecord SpoutExecutorStats [common acked failed complete-latencies])
+(defrecord SpoutExecutorStats [^CommonStats common
+                               ^MultiCountStatAndMetric acked
+                               ^MultiCountStatAndMetric failed
+                               ^MultiLatencyStatAndMetric complete-latencies])
 
 (def NUM-STAT-BUCKETS 20)
-;; 10 minutes, 3 hours, 1 day
-(def STAT-BUCKETS [30 540 4320])
 
 (defn- mk-common-stats
   [rate]
   (CommonStats.
-    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
+    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
     rate))
 
 (defn mk-bolt-stats
   [rate]
   (BoltExecutorStats.
     (mk-common-stats rate)
-    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-    (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-    (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))))
+    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
+    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
+    (MultiLatencyStatAndMetric. NUM-STAT-BUCKETS)
+    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
+    (MultiLatencyStatAndMetric. NUM-STAT-BUCKETS)))
 
 (defn mk-spout-stats
   [rate]
   (SpoutExecutorStats.
     (mk-common-stats rate)
-    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-    (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
-    (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))))
-
-(defmacro update-executor-stat!
-  [stats path & args]
-  (let [path (collectify path)]
-    `(swap! (-> ~stats ~@path) update-rolling-window-set ~@args)))
+    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
+    (MultiCountStatAndMetric. NUM-STAT-BUCKETS)
+    (MultiLatencyStatAndMetric. NUM-STAT-BUCKETS)))
 
 (defmacro stats-rate
   [stats]
   `(-> ~stats :common :rate))
 
+(defmacro stats-emitted
+  [stats]
+  `(-> ~stats :common :emitted))
+
+(defmacro stats-transferred
+  [stats]
+  `(-> ~stats :common :transferred))
+
+(defmacro stats-executed
+  [stats]
+  `(:executed ~stats))
+
+(defmacro stats-acked
+  [stats]
+  `(:acked ~stats))
+
+(defmacro stats-failed
+  [stats]
+  `(:failed ~stats))
+
+(defmacro stats-execute-latencies
+  [stats]
+  `(:execute-latencies ~stats))
+
+(defmacro stats-process-latencies
+  [stats]
+  `(:process-latencies ~stats))
+
+(defmacro stats-complete-latencies
+  [stats]
+  `(:complete-latencies ~stats))
+
 (defn emitted-tuple!
   [stats stream]
-  (update-executor-stat! stats [:common :emitted] stream (stats-rate stats)))
+  (.incBy ^MultiCountStatAndMetric (stats-emitted stats) ^Object stream ^long (stats-rate stats)))
 
 (defn transferred-tuples!
   [stats stream amt]
-  (update-executor-stat! stats [:common :transferred] stream (* (stats-rate stats) amt)))
+  (.incBy ^MultiCountStatAndMetric (stats-transferred stats) ^Object stream ^long (* (stats-rate stats) amt)))
 
 (defn bolt-execute-tuple!
   [^BoltExecutorStats stats component stream latency-ms]
-  (let [key [component stream]]
-    (update-executor-stat! stats :executed key (stats-rate stats))
-    (update-executor-stat! stats :execute-latencies key latency-ms)))
+  (let [key [component stream]
+        ^MultiCountStatAndMetric executed (stats-executed stats)
+        ^MultiLatencyStatAndMetric exec-lat (stats-execute-latencies stats)]
+    (.incBy executed key (stats-rate stats))
+    (.record exec-lat key latency-ms)))
 
 (defn bolt-acked-tuple!
   [^BoltExecutorStats stats component stream latency-ms]
-  (let [key [component stream]]
-    (update-executor-stat! stats :acked key (stats-rate stats))
-    (update-executor-stat! stats :process-latencies key latency-ms)))
+  (let [key [component stream]
+        ^MultiCountStatAndMetric acked (stats-acked stats)
+        ^MultiLatencyStatAndMetric process-lat (stats-process-latencies stats)]
+    (.incBy acked key (stats-rate stats))
+    (.record process-lat key latency-ms)))
 
 (defn bolt-failed-tuple!
   [^BoltExecutorStats stats component stream latency-ms]
-  (let [key [component stream]]
-    (update-executor-stat! stats :failed key (stats-rate stats))))
+  (let [key [component stream]
+        ^MultiCountStatAndMetric failed (stats-failed stats)]
+    (.incBy failed key (stats-rate stats))))
 
 (defn spout-acked-tuple!
   [^SpoutExecutorStats stats stream latency-ms]
-  (update-executor-stat! stats :acked stream (stats-rate stats))
-  (update-executor-stat! stats :complete-latencies stream latency-ms))
+  (.incBy ^MultiCountStatAndMetric (stats-acked stats) stream (stats-rate stats))
+  (.record ^MultiLatencyStatAndMetric (stats-complete-latencies stats) stream latency-ms))
 
 (defn spout-failed-tuple!
   [^SpoutExecutorStats stats stream latency-ms]
-  (update-executor-stat! stats :failed stream (stats-rate stats))
-  )
+  (.incBy ^MultiCountStatAndMetric (stats-failed stats) stream (stats-rate stats)))
 
 (defn- cleanup-stat! [stat]
-  (swap! stat cleanup-rolling-window-set))
+  (.close stat))
 
 (defn- cleanup-common-stats!
   [^CommonStats stats]
@@ -268,7 +176,9 @@
 (defn- value-stats
   [stats fields]
   (into {} (dofor [f fields]
-                  [f (value-rolling-window-set @(f stats))])))
+                  [f (if (instance? MultiCountStatAndMetric (f stats))
+                         (.getTimeCounts ^MultiCountStatAndMetric (f stats))
+                         (.getTimeLatAvg ^MultiLatencyStatAndMetric (f stats)))])))
 
 (defn- value-common-stats
   [^CommonStats stats]
@@ -322,16 +232,16 @@
   [(.get_componentId global-stream-id) (.get_streamId global-stream-id)])
 
 (defmethod clojurify-specific-stats BoltStats [^BoltStats stats]
-  [(window-set-converter (.get_acked stats) from-global-stream-id symbol)
-   (window-set-converter (.get_failed stats) from-global-stream-id symbol)
-   (window-set-converter (.get_process_ms_avg stats) from-global-stream-id symbol)
-   (window-set-converter (.get_executed stats) from-global-stream-id symbol)
-   (window-set-converter (.get_execute_ms_avg stats) from-global-stream-id symbol)])
+  [(window-set-converter (.get_acked stats) from-global-stream-id identity)
+   (window-set-converter (.get_failed stats) from-global-stream-id identity)
+   (window-set-converter (.get_process_ms_avg stats) from-global-stream-id identity)
+   (window-set-converter (.get_executed stats) from-global-stream-id identity)
+   (window-set-converter (.get_execute_ms_avg stats) from-global-stream-id identity)])
 
 (defmethod clojurify-specific-stats SpoutStats [^SpoutStats stats]
-  [(window-set-converter (.get_acked stats) symbol)
-   (window-set-converter (.get_failed stats) symbol)
-   (window-set-converter (.get_complete_ms_avg stats) symbol)])
+  [(.get_acked stats)
+   (.get_failed stats)
+   (.get_complete_ms_avg stats)])
 
 
 (defn clojurify-executor-stats
@@ -340,7 +250,9 @@
          is_bolt? (.is_set_bolt specific-stats)
          specific-stats (if is_bolt? (.get_bolt specific-stats) (.get_spout specific-stats))
          specific-stats (clojurify-specific-stats specific-stats)
-         common-stats (CommonStats. (window-set-converter (.get_emitted stats) symbol) (window-set-converter (.get_transferred stats) symbol) (.get_rate stats))]
+         common-stats (CommonStats. (.get_emitted stats)
+                                    (.get_transferred stats)
+                                    (.get_rate stats))]
     (if is_bolt?
       ; worker heart beat does not store the BoltExecutorStats or SpoutExecutorStats , instead it stores the result returned by render-stats!
       ; which flattens the BoltExecutorStats/SpoutExecutorStats by extracting values from all atoms and merging all values inside :common to top
@@ -373,4 +285,1237 @@
     (ExecutorStats. (window-set-converter (:emitted stats) str)
       (window-set-converter (:transferred stats) str)
       specific-stats
-      rate)))
\ No newline at end of file
+      rate)))
+
+(defn valid-number?
+  "Returns true if x is a number that is not NaN or Infinity, false otherwise"
+  [x]
+  (and (number? x)
+       (not (Double/isNaN x))
+       (not (Double/isInfinite x))))
+
+(defn apply-default
+  [f defaulting-fn & args]
+  (apply f (map defaulting-fn args)))
+
+(defn apply-or-0
+  [f & args]
+  (apply apply-default
+         f
+         #(if (valid-number? %) % 0)
+         args))
+
+(defn sum-or-0
+  [& args]
+  (apply apply-or-0 + args))
+
+(defn product-or-0
+  [& args]
+  (apply apply-or-0 * args))
+
+(defn max-or-0
+  [& args]
+  (apply apply-or-0 max args))
+
+(defn- agg-bolt-lat-and-count
+  "Aggregates number executed, process latency, and execute latency across all
+  streams."
+  [idk->exec-avg idk->proc-avg idk->num-executed]
+  (letfn [(weight-avg [[id avg]]
+            (let [num-e (get idk->num-executed id)]
+              (product-or-0 avg num-e)))]
+    {:executeLatencyTotal (sum (map weight-avg idk->exec-avg))
+     :processLatencyTotal (sum (map weight-avg idk->proc-avg))
+     :executed (sum (vals idk->num-executed))}))
+
+(defn- agg-spout-lat-and-count
+  "Aggregates number acked and complete latencies across all streams."
+  [sid->comp-avg sid->num-acked]
+  (letfn [(weight-avg [[id avg]]
+            (product-or-0 avg (get sid->num-acked id)))]
+    {:completeLatencyTotal (sum (map weight-avg sid->comp-avg))
+     :acked (sum (vals sid->num-acked))}))
+
+(defn add-pairs
+  ([] [0 0])
+  ([[a1 a2] [b1 b2]]
+   [(+ a1 b1) (+ a2 b2)]))
+
+(defn mk-include-sys-fn
+  [include-sys?]
+  (if include-sys?
+    (fn [_] true)
+    (fn [stream] (and (string? stream) (not (Utils/isSystemId stream))))))
+
+(defn mk-include-sys-filter
+  "Returns a function that includes or excludes map entries whose keys are
+  system ids."
+  [include-sys?]
+  (if include-sys?
+    identity
+    (partial filter-key (mk-include-sys-fn false))))
+
+(defn- agg-bolt-streams-lat-and-count
+  "Aggregates number executed and process & execute latencies."
+  [idk->exec-avg idk->proc-avg idk->executed]
+  (letfn [(weight-avg [id avg]
+            (let [num-e (idk->executed id)]
+              (product-or-0 avg num-e)))]
+    (into {}
+      (for [k (keys idk->exec-avg)]
+        [k {:executeLatencyTotal (weight-avg k (get idk->exec-avg k))
+            :processLatencyTotal (weight-avg k (get idk->proc-avg k))
+            :executed (idk->executed k)}]))))
+
+(defn- agg-spout-streams-lat-and-count
+  "Aggregates number acked and complete latencies."
+  [idk->comp-avg idk->acked]
+  (letfn [(weight-avg [id avg]
+            (let [num-e (get idk->acked id)]
+              (product-or-0 avg num-e)))]
+    (into {}
+      (for [k (keys idk->comp-avg)]
+        [k {:completeLatencyTotal (weight-avg k (get idk->comp-avg k))
+            :acked (get idk->acked k)}]))))
+
+(defn swap-map-order
+  "For a nested map, rearrange data such that the top-level keys become the
+  nested map's keys and vice versa.
+  Example:
+  {:a {:X :banana, :Y :pear}, :b {:X :apple, :Y :orange}}
+  -> {:Y {:a :pear, :b :orange}, :X {:a :banana, :b :apple}}"
+  [m]
+  (apply merge-with
+         merge
+         (map (fn [[k v]]
+                (into {}
+                      (for [[k2 v2] v]
+                        [k2 {k v2}])))
+              m)))
+
+(defn- compute-agg-capacity
+  "Computes the capacity metric for one executor given its heartbeat data and
+  uptime."
+  [m uptime]
+  (when uptime
+    (->>
+      ;; For each stream, create weighted averages and counts.
+      (merge-with (fn weighted-avg+count-fn
+                    [avg cnt]
+                    [(* avg cnt) cnt])
+                  (get (:execute-latencies m) (str TEN-MIN-IN-SECONDS))
+                  (get (:executed m) (str TEN-MIN-IN-SECONDS)))
+      vals ;; Ignore the stream ids.
+      (reduce add-pairs
+              [0. 0]) ;; Combine weighted averages and counts.
+      ((fn [[weighted-avg cnt]]
+        (div weighted-avg (* 1000 (min uptime TEN-MIN-IN-SECONDS))))))))
+
+(defn agg-pre-merge-comp-page-bolt
+  [{exec-id :exec-id
+    host :host
+    port :port
+    uptime :uptime
+    comp-id :comp-id
+    num-tasks :num-tasks
+    statk->w->sid->num :stats}
+   window
+   include-sys?]
+  (let [str-key (partial map-key str)
+        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
+    {:executor-id exec-id,
+     :host host,
+     :port port,
+     :uptime uptime,
+     :num-executors 1,
+     :num-tasks num-tasks,
+     :capacity (compute-agg-capacity statk->w->sid->num uptime)
+     :cid+sid->input-stats
+     (merge-with
+       merge
+       (swap-map-order
+         {:acked (-> statk->w->sid->num
+                     :acked
+                     str-key
+                     (get window))
+          :failed (-> statk->w->sid->num
+                      :failed
+                      str-key
+                      (get window))})
+       (agg-bolt-streams-lat-and-count (-> statk->w->sid->num
+                                           :execute-latencies
+                                           str-key
+                                           (get window))
+                                       (-> statk->w->sid->num
+                                           :process-latencies
+                                           str-key
+                                           (get window))
+                                       (-> statk->w->sid->num
+                                           :executed
+                                           str-key
+                                           (get window)))),
+     :sid->output-stats
+     (swap-map-order
+       {:emitted (-> statk->w->sid->num
+                     :emitted
+                     str-key
+                     (get window)
+                     handle-sys-components-fn)
+        :transferred (-> statk->w->sid->num
+                         :transferred
+                         str-key
+                         (get window)
+                         handle-sys-components-fn)})}))
+
+(defn agg-pre-merge-comp-page-spout
+  [{exec-id :exec-id
+    host :host
+    port :port
+    uptime :uptime
+    comp-id :comp-id
+    num-tasks :num-tasks
+    statk->w->sid->num :stats}
+   window
+   include-sys?]
+  (let [str-key (partial map-key str)
+        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
+    {:executor-id exec-id,
+     :host host,
+     :port port,
+     :uptime uptime,
+     :num-executors 1,
+     :num-tasks num-tasks,
+     :sid->output-stats
+     (merge-with
+       merge
+       (agg-spout-streams-lat-and-count (-> statk->w->sid->num
+                                            :complete-latencies
+                                            str-key
+                                            (get window))
+                                        (-> statk->w->sid->num
+                                            :acked
+                                            str-key
+                                            (get window)))
+       (swap-map-order
+         {:acked (-> statk->w->sid->num
+                     :acked
+                     str-key
+                     (get window))
+          :failed (-> statk->w->sid->num
+                      :failed
+                      str-key
+                      (get window))
+          :emitted (-> statk->w->sid->num
+                       :emitted
+                       str-key
+                       (get window)
+                       handle-sys-components-fn)
+          :transferred (-> statk->w->sid->num
+                           :transferred
+                           str-key
+                           (get window)
+                           handle-sys-components-fn)}))}))
+
+(defn agg-pre-merge-topo-page-bolt
+  [{comp-id :comp-id
+    num-tasks :num-tasks
+    statk->w->sid->num :stats
+    uptime :uptime}
+   window
+   include-sys?]
+  (let [str-key (partial map-key str)
+        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
+    {comp-id
+     (merge
+       (agg-bolt-lat-and-count (-> statk->w->sid->num
+                                   :execute-latencies
+                                   str-key
+                                   (get window))
+                               (-> statk->w->sid->num
+                                   :process-latencies
+                                   str-key
+                                   (get window))
+                               (-> statk->w->sid->num
+                                   :executed
+                                   str-key
+                                   (get window)))
+       {:num-executors 1
+        :num-tasks num-tasks
+        :emitted (-> statk->w->sid->num
+                     :emitted
+                     str-key
+                     (get window)
+                     handle-sys-components-fn
+                     vals
+                     sum)
+        :transferred (-> statk->w->sid->num
+                         :transferred
+                         str-key
+                         (get window)
+                         handle-sys-components-fn
+                         vals
+                         sum)
+        :capacity (compute-agg-capacity statk->w->sid->num uptime)
+        :acked (-> statk->w->sid->num
+                   :acked
+                   str-key
+                   (get window)
+                   vals
+                   sum)
+        :failed (-> statk->w->sid->num
+                    :failed
+                    str-key
+                    (get window)
+                    vals
+                    sum)})}))
+
+(defn agg-pre-merge-topo-page-spout
+  [{comp-id :comp-id
+    num-tasks :num-tasks
+    statk->w->sid->num :stats}
+   window
+   include-sys?]
+  (let [str-key (partial map-key str)
+        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
+    {comp-id
+     (merge
+       (agg-spout-lat-and-count (-> statk->w->sid->num
+                                    :complete-latencies
+                                    str-key
+                                    (get window))
+                                (-> statk->w->sid->num
+                                    :acked
+                                    str-key
+                                    (get window)))
+       {:num-executors 1
+        :num-tasks num-tasks
+        :emitted (-> statk->w->sid->num
+                     :emitted
+                     str-key
+                     (get window)
+                     handle-sys-components-fn
+                     vals
+                     sum)
+        :transferred (-> statk->w->sid->num
+                         :transferred
+                         str-key
+                         (get window)
+                         handle-sys-components-fn
+                         vals
+                         sum)
+        :failed (-> statk->w->sid->num
+                    :failed
+                    str-key
+                    (get window)
+                    vals
+                    sum)})}))
+
+(defn merge-agg-comp-stats-comp-page-bolt
+  [{acc-in :cid+sid->input-stats
+    acc-out :sid->output-stats
+    :as acc-bolt-stats}
+   {bolt-in :cid+sid->input-stats
+    bolt-out :sid->output-stats
+    :as bolt-stats}]
+  {:num-executors (inc (or (:num-executors acc-bolt-stats) 0)),
+   :num-tasks (sum-or-0 (:num-tasks acc-bolt-stats) (:num-tasks bolt-stats)),
+   :sid->output-stats (merge-with (partial merge-with sum-or-0)
+                                  acc-out
+                                  bolt-out),
+   :cid+sid->input-stats (merge-with (partial merge-with sum-or-0)
+                                     acc-in
+                                     bolt-in),
+   :executor-stats
+   (let [sum-streams (fn [m k] (->> m vals (map k) (apply sum-or-0)))
+         executed (sum-streams bolt-in :executed)]
+     (conj (:executor-stats acc-bolt-stats)
+           (merge
+             (select-keys bolt-stats
+                          [:executor-id :uptime :host :port :capacity])
+             {:emitted (sum-streams bolt-out :emitted)
+              :transferred (sum-streams bolt-out :transferred)
+              :acked (sum-streams bolt-in :acked)
+              :failed (sum-streams bolt-in :failed)
+              :executed executed}
+             (->>
+               (if (and executed (pos? executed))
+                 [(div (sum-streams bolt-in :executeLatencyTotal) executed)
+                  (div (sum-streams bolt-in :processLatencyTotal) executed)]
+                 [nil nil])
+               (mapcat vector [:execute-latency :process-latency])
+               (apply assoc {})))))})
+
+(defn merge-agg-comp-stats-comp-page-spout
+  [{acc-out :sid->output-stats
+    :as acc-spout-stats}
+   {spout-out :sid->output-stats
+    :as spout-stats}]
+  {:num-executors (inc (or (:num-executors acc-spout-stats) 0)),
+   :num-tasks (sum-or-0 (:num-tasks acc-spout-stats) (:num-tasks spout-stats)),
+   :sid->output-stats (merge-with (partial merge-with sum-or-0)
+                                  acc-out
+                                  spout-out),
+   :executor-stats
+   (let [sum-streams (fn [m k] (->> m vals (map k) (apply sum-or-0)))
+         acked (sum-streams spout-out :acked)]
+     (conj (:executor-stats acc-spout-stats)
+           (merge
+             (select-keys spout-stats [:executor-id :uptime :host :port])
+             {:emitted (sum-streams spout-out :emitted)
+              :transferred (sum-streams spout-out :transferred)
+              :acked acked
+              :failed (sum-streams spout-out :failed)}
+             {:complete-latency (if (and acked (pos? acked))
+                                  (div (sum-streams spout-out
+                                                    :completeLatencyTotal)
+                                       acked)
+                                  nil)})))})
+
+(defn merge-agg-comp-stats-topo-page-bolt
+  [acc-bolt-stats bolt-stats]
+  {:num-executors (inc (or (:num-executors acc-bolt-stats) 0))
+   :num-tasks (sum-or-0 (:num-tasks acc-bolt-stats) (:num-tasks bolt-stats))
+   :emitted (sum-or-0 (:emitted acc-bolt-stats) (:emitted bolt-stats))
+   :transferred (sum-or-0 (:transferred acc-bolt-stats)
+                          (:transferred bolt-stats))
+   :capacity (max-or-0 (:capacity acc-bolt-stats) (:capacity bolt-stats))
+   ;; We sum average latency totals here to avoid dividing at each step.
+   ;; Compute the average latencies by dividing the total by the count.
+   :executeLatencyTotal (sum-or-0 (:executeLatencyTotal acc-bolt-stats)
+                                  (:executeLatencyTotal bolt-stats))
+   :processLatencyTotal (sum-or-0 (:processLatencyTotal acc-bolt-stats)
+                                  (:processLatencyTotal bolt-stats))
+   :executed (sum-or-0 (:executed acc-bolt-stats) (:executed bolt-stats))
+   :acked (sum-or-0 (:acked acc-bolt-stats) (:acked bolt-stats))
+   :failed (sum-or-0 (:failed acc-bolt-stats) (:failed bolt-stats))})
+
+(defn merge-agg-comp-stats-topo-page-spout
+  [acc-spout-stats spout-stats]
+  {:num-executors (inc (or (:num-executors acc-spout-stats) 0))
+   :num-tasks (sum-or-0 (:num-tasks acc-spout-stats) (:num-tasks spout-stats))
+   :emitted (sum-or-0 (:emitted acc-spout-stats) (:emitted spout-stats))
+   :transferred (sum-or-0 (:transferred acc-spout-stats) (:transferred spout-stats))
+   ;; We sum average latency totals here to avoid dividing at each step.
+   ;; Compute the average latencies by dividing the total by the count.
+   :completeLatencyTotal (sum-or-0 (:completeLatencyTotal acc-spout-stats)
+                            (:completeLatencyTotal spout-stats))
+   :acked (sum-or-0 (:acked acc-spout-stats) (:acked spout-stats))
+   :failed (sum-or-0 (:failed acc-spout-stats) (:failed spout-stats))})
+
+(defn aggregate-count-streams
+  [stats]
+  (->> stats
+       (map-val #(reduce + (vals %)))))
+
+(defn- agg-topo-exec-stats*
+  "A helper function that does the common work to aggregate stats of one
+  executor with the given map for the topology page."
+  [window
+   include-sys?
+   {:keys [workers-set
+           bolt-id->stats
+           spout-id->stats
+           window->emitted
+           window->transferred
+           window->comp-lat-wgt-avg
+           window->acked
+           window->failed] :as acc-stats}
+   {:keys [stats] :as new-data}
+   pre-merge-fn
+   merge-fn
+   comp-key]
+  (let [cid->statk->num (pre-merge-fn new-data window include-sys?)
+        {w->compLatWgtAvg :completeLatencyTotal
+         w->acked :acked}
+          (if (:complete-latencies stats)
+            (swap-map-order
+              (into {}
+                    (for [w (keys (:acked stats))]
+                         [w (agg-spout-lat-and-count
+                              (get (:complete-latencies stats) w)
+                              (get (:acked stats) w))])))
+            {:completeLatencyTotal nil
+             :acks (aggregate-count-streams (:acked stats))})
+        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
+    (assoc {:workers-set (conj workers-set
+                               [(:host new-data) (:port new-data)])
+            :bolt-id->stats bolt-id->stats
+            :spout-id->stats spout-id->stats
+            :window->emitted (->> (:emitted stats)
+                                  (map-val handle-sys-components-fn)
+                                  aggregate-count-streams
+                                  (merge-with + window->emitted))
+            :window->transferred (->> (:transferred stats)
+                                      (map-val handle-sys-components-fn)
+                                      aggregate-count-streams
+                                      (merge-with + window->transferred))
+            :window->comp-lat-wgt-avg (merge-with +
+                                                  window->comp-lat-wgt-avg
+                                                  w->compLatWgtAvg)
+            :window->acked (if (= :spout (:type stats))
+                             (merge-with + window->acked w->acked)
+                             window->acked)
+            :window->failed (if (= :spout (:type stats))
+                              (->> (:failed stats)
+                                   aggregate-count-streams
+                                   (merge-with + window->failed))
+                              window->failed)}
+           comp-key (merge-with merge-fn
+                                (acc-stats comp-key)
+                                cid->statk->num)
+           :type (:type stats))))
+
+(defmulti agg-topo-exec-stats
+  "Combines the aggregate stats of one executor with the given map, selecting
+  the appropriate window and including system components as specified."
+  (fn dispatch-fn [& args] (:type (last args))))
+
+(defmethod agg-topo-exec-stats :bolt
+  [window include-sys? acc-stats new-data]
+  (agg-topo-exec-stats* window
+                        include-sys?
+                        acc-stats
+                        new-data
+                        agg-pre-merge-topo-page-bolt
+                        merge-agg-comp-stats-topo-page-bolt
+                        :bolt-id->stats))
+
+(defmethod agg-topo-exec-stats :spout
+  [window include-sys? acc-stats new-data]
+  (agg-topo-exec-stats* window
+                        include-sys?
+                        acc-stats
+                        new-data
+                        agg-pre-merge-topo-page-spout
+                        merge-agg-comp-stats-topo-page-spout
+                        :spout-id->stats))
+
+(defmethod agg-topo-exec-stats :default [_ _ acc-stats _] acc-stats)
+
+(defn get-last-error
+  [storm-cluster-state storm-id component-id]
+  (if-let [e (.last-error storm-cluster-state storm-id component-id)]
+    (ErrorInfo. (:error e) (:time-secs e))))
+
+(defn component-type
+  "Returns the component type (either :bolt or :spout) for a given
+  topology and component id. Returns nil if not found."
+  [^StormTopology topology id]
+  (let [bolts (.get_bolts topology)
+        spouts (.get_spouts topology)]
+    (cond
+      (Utils/isSystemId id) :bolt
+      (.containsKey bolts id) :bolt
+      (.containsKey spouts id) :spout)))
+
+(defn extract-nodeinfos-from-hb-for-comp
+  ([exec->host+port task->component include-sys? comp-id]
+   (distinct (for [[[start end :as executor] [host port]] exec->host+port
+         :let [id (task->component start)]
+         :when (and (or (nil? comp-id) (= comp-id id))
+                 (or include-sys? (not (Utils/isSystemId id))))]
+     {:host host
+      :port port}))))
+
+(defn extract-data-from-hb
+  ([exec->host+port task->component beats include-sys? topology comp-id]
+   (for [[[start end :as executor] [host port]] exec->host+port
+         :let [beat (beats executor)
+               id (task->component start)]
+         :when (and (or (nil? comp-id) (= comp-id id))
+                    (or include-sys? (not (Utils/isSystemId id))))]
+     {:exec-id executor
+      :comp-id id
+      :num-tasks (count (range start (inc end)))
+      :host host
+      :port port
+      :uptime (:uptime beat)
+      :stats (:stats beat)
+      :type (or (:type (:stats beat))
+                (component-type topology id))}))
+  ([exec->host+port task->component beats include-sys? topology]
+    (extract-data-from-hb exec->host+port
+                          task->component
+                          beats
+                          include-sys?
+                          topology
+                          nil)))
+
+(defn aggregate-topo-stats
+  [window include-sys? data]
+  (let [init-val {:workers-set #{}
+                  :bolt-id->stats {}
+                  :spout-id->stats {}
+                  :window->emitted {}
+                  :window->transferred {}
+                  :window->comp-lat-wgt-avg {}
+                  :window->acked {}
+                  :window->failed {}}
+        reducer-fn (partial agg-topo-exec-stats
+                            window
+                            include-sys?)]
+    (reduce reducer-fn init-val data)))
+
+(defn- compute-weighted-averages-per-window
+  [acc-data wgt-avg-key divisor-key]
+  (into {} (for [[window wgt-avg] (wgt-avg-key acc-data)
+                 :let [divisor ((divisor-key acc-data) window)]
+                 :when (and divisor (pos? divisor))]
+             [(str window) (div wgt-avg divisor)])))
+
+(defn- post-aggregate-topo-stats
+  [task->component exec->node+port last-err-fn acc-data]
+  {:num-tasks (count task->component)
+   :num-workers (count (:workers-set acc-data))
+   :num-executors (count exec->node+port)
+   :bolt-id->stats
+     (into {} (for [[id m] (:bolt-id->stats acc-data)
+                    :let [executed (:executed m)]]
+                     [id (-> m
+                             (assoc :execute-latency
+                                    (if (and executed (pos? executed))
+                                      (div (or (:executeLatencyTotal m) 0)
+                                           executed)
+                                      0)
+                                    :process-latency
+                                    (if (and executed (pos? executed))
+                                      (div (or (:processLatencyTotal m) 0)
+                                           executed)
+                                      0))
+                             (dissoc :executeLatencyTotal
+                                     :processLatencyTotal)
+                             (assoc :lastError (last-err-fn id)))]))
+   :spout-id->stats
+     (into {} (for [[id m] (:spout-id->stats acc-data)
+                    :let [acked (:acked m)]]
+                    [id (-> m
+                            (assoc :complete-latency
+                                   (if (and acked (pos? acked))
+                                     (div (:completeLatencyTotal m)
+                                          (:acked m))
+                                     0))
+                            (dissoc :completeLatencyTotal)
+                            (assoc :lastError (last-err-fn id)))]))
+   :window->emitted (map-key str (:window->emitted acc-data))
+   :window->transferred (map-key str (:window->transferred acc-data))
+   :window->complete-latency
+     (compute-weighted-averages-per-window acc-data
+                                           :window->comp-lat-wgt-avg
+                                           :window->acked)
+   :window->acked (map-key str (:window->acked acc-data))
+   :window->failed (map-key str (:window->failed acc-data))})
+
+(defn- thriftify-common-agg-stats
+  [^ComponentAggregateStats s
+   {:keys [num-tasks
+           emitted
+           transferred
+           acked
+           failed
+           num-executors] :as statk->num}]
+  (let [cas (CommonAggregateStats.)]
+    (and num-executors (.set_num_executors cas num-executors))
+    (and num-tasks (.set_num_tasks cas num-tasks))
+    (and emitted (.set_emitted cas emitted))
+    (and transferred (.set_transferred cas transferred))
+    (and acked (.set_acked cas acked))
+    (and failed (.set_failed cas failed))
+    (.set_common_stats s cas)))
+
+(defn thriftify-bolt-agg-stats
+  [statk->num]
+  (let [{:keys [lastError
+                execute-latency
+                process-latency
+                executed
+                capacity]} statk->num
+        s (ComponentAggregateStats.)]
+    (.set_type s ComponentType/BOLT)
+    (and lastError (.set_last_error s lastError))
+    (thriftify-common-agg-stats s statk->num)
+    (.set_specific_stats s
+      (SpecificAggregateStats/bolt
+        (let [bas (BoltAggregateStats.)]
+          (and execute-latency (.set_execute_latency_ms bas execute-latency))
+          (and process-latency (.set_process_latency_ms bas process-latency))
+          (and executed (.set_executed bas executed))
+          (and capacity (.set_capacity bas capacity))
+          bas)))
+    s))
+
+(defn thriftify-spout-agg-stats
+  [statk->num]
+  (let [{:keys [lastError
+                complete-latency]} statk->num
+        s (ComponentAggregateStats.)]
+    (.set_type s ComponentType/SPOUT)
+    (and lastError (.set_last_error s lastError))
+    (thriftify-common-agg-stats s statk->num)
+    (.set_specific_stats s
+      (SpecificAggregateStats/spout
+        (let [sas (SpoutAggregateStats.)]
+          (and complete-latency (.set_complete_latency_ms sas complete-latency))
+          sas)))
+    s))
+
+(defn thriftify-topo-page-data
+  [topology-id data]
+  (let [{:keys [num-tasks
+                num-workers
+                num-executors
+                spout-id->stats
+                bolt-id->stats
+                window->emitted
+                window->transferred
+                window->complete-latency
+                window->acked
+                window->failed]} data
+        spout-agg-stats (into {}
+                              (for [[id m] spout-id->stats
+                                    :let [m (assoc m :type :spout)]]
+                                [id
+                                 (thriftify-spout-agg-stats m)]))
+        bolt-agg-stats (into {}
+                             (for [[id m] bolt-id->stats
+                                   :let [m (assoc m :type :bolt)]]
+                              [id
+                               (thriftify-bolt-agg-stats m)]))
+        topology-stats (doto (TopologyStats.)
+                         (.set_window_to_emitted window->emitted)
+                         (.set_window_to_transferred window->transferred)
+                         (.set_window_to_complete_latencies_ms
+                           window->complete-latency)
+                         (.set_window_to_acked window->acked)
+                         (.set_window_to_failed window->failed))
+      topo-page-info (doto (TopologyPageInfo. topology-id)
+                       (.set_num_tasks num-tasks)
+                       (.set_num_workers num-workers)
+                       (.set_num_executors num-executors)
+                       (.set_id_to_spout_agg_stats spout-agg-stats)
+                       (.set_id_to_bolt_agg_stats bolt-agg-stats)
+                       (.set_topology_stats topology-stats))]
+    topo-page-info))
+
+(defn agg-topo-execs-stats
+  "Aggregate various executor statistics for a topology from the given
+  heartbeats."
+  [topology-id
+   exec->node+port
+   task->component
+   beats
+   topology
+   window
+   include-sys?
+   last-err-fn]
+  (->> ;; This iterates over each executor one time, because of lazy evaluation.
+    (extract-data-from-hb exec->node+port
+                          task->component
+                          beats
+                          include-sys?
+                          topology)
+    (aggregate-topo-stats window include-sys?)
+    (post-aggregate-topo-stats task->component exec->node+port last-err-fn)
+    (thriftify-topo-page-data topology-id)))
+
+(defn- agg-bolt-exec-win-stats
+  "A helper function that aggregates windowed stats from one bolt executor."
+  [acc-stats new-stats include-sys?]
+  (let [{w->execLatWgtAvg :executeLatencyTotal
+         w->procLatWgtAvg :processLatencyTotal
+         w->executed :executed}
+          (swap-map-order
+            (into {} (for [w (keys (:executed new-stats))]
+                       [w (agg-bolt-lat-and-count
+                            (get (:execute-latencies new-stats) w)
+                            (get (:process-latencies new-stats) w)
+                            (get (:executed new-stats) w))])))
+        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
+    {:window->emitted (->> (:emitted new-stats)
+                           (map-val handle-sys-components-fn)
+                           aggregate-count-streams
+                           (merge-with + (:window->emitted acc-stats)))
+     :window->transferred (->> (:transferred new-stats)
+                               (map-val handle-sys-components-fn)
+                               aggregate-count-streams
+                               (merge-with + (:window->transferred acc-stats)))
+     :window->exec-lat-wgt-avg (merge-with +
+                                           (:window->exec-lat-wgt-avg acc-stats)
+                                           w->execLatWgtAvg)
+     :window->proc-lat-wgt-avg (merge-with +
+                                           (:window->proc-lat-wgt-avg acc-stats)
+                                           w->procLatWgtAvg)
+     :window->executed (merge-with + (:window->executed acc-stats) w->executed)
+     :window->acked (->> (:acked new-stats)
+                         aggregate-count-streams
+                         (merge-with + (:window->acked acc-stats)))
+     :window->failed (->> (:failed new-stats)
+                          aggregate-count-streams
+                          (merge-with + (:window->failed acc-stats)))}))
+
+(defn- agg-spout-exec-win-stats
+  "A helper function that aggregates windowed stats from one spout executor."
+  [acc-stats new-stats include-sys?]
+  (let [{w->compLatWgtAvg :completeLatencyTotal
+         w->acked :acked}
+          (swap-map-order
+            (into {} (for [w (keys (:acked new-stats))]
+                       [w (agg-spout-lat-and-count
+                            (get (:complete-latencies new-stats) w)
+                            (get (:acked new-stats) w))])))
+        handle-sys-components-fn (mk-include-sys-filter include-sys?)]
+    {:window->emitted (->> (:emitted new-stats)
+                           (map-val handle-sys-components-fn)
+                           aggregate-count-streams
+                           (merge-with + (:window->emitted acc-stats)))
+     :window->transferred (->> (:transferred new-stats)
+                               (map-val handle-sys-components-fn)
+                               aggregate-count-streams
+                               (merge-with + (:window->transferred acc-stats)))
+     :window->comp-lat-wgt-avg (merge-with +
+                                           (:window->comp-lat-wgt-avg acc-stats)
+                                           w->compLatWgtAvg)
+     :window->acked (->> (:acked new-stats)
+                         aggregate-count-streams
+                         (merge-with + (:window->acked acc-stats)))
+     :window->failed (->> (:failed new-stats)
+                          aggregate-count-streams
+                          (merge-with + (:window->failed acc-stats)))}))
+
+(defmulti agg-comp-exec-stats
+  "Combines the aggregate stats of one executor with the given map, selecting
+  the appropriate window and including system components as specified."
+  (fn dispatch-fn [_ _ init-val _] (:type init-val)))
+
+(defmethod agg-comp-exec-stats :bolt
+  [window include-sys? acc-stats new-data]
+  (assoc (agg-bolt-exec-win-stats acc-stats (:stats new-data) include-sys?)
+         :stats (merge-agg-comp-stats-comp-page-bolt
+                  (:stats acc-stats)
+                  (agg-pre-merge-comp-page-bolt new-data window include-sys?))
+         :type :bolt))
+
+(defmethod agg-comp-exec-stats :spout
+  [window include-sys? acc-stats new-data]
+  (assoc (agg-spout-exec-win-stats acc-stats (:stats new-data) include-sys?)
+         :stats (merge-agg-comp-stats-comp-page-spout
+                  (:stats acc-stats)
+                  (agg-pre-merge-comp-page-spout new-data window include-sys?))
+         :type :spout))
+
+(defn- aggregate-comp-stats*
+  [window include-sys? data init-val]
+  (-> (partial agg-comp-exec-stats
+               window
+               include-sys?)
+      (reduce init-val data)))
+
+(defmulti aggregate-comp-stats
+  (fn dispatch-fn [& args] (-> args last first :type)))
+
+(defmethod aggregate-comp-stats :bolt
+  [& args]
+  (let [init-val {:type :bolt
+                  :cid+sid->input-stats {}
+                  :sid->output-stats {}
+                  :executor-stats []
+                  :window->emitted {}
+                  :window->transferred {}
+                  :window->exec-lat-wgt-avg {}
+                  :window->executed {}
+                  :window->proc-lat-wgt-avg {}
+                  :window->acked {}
+                  :window->failed {}}]
+    (apply aggregate-comp-stats* (concat args (list init-val)))))
+
+(defmethod aggregate-comp-stats :spout
+  [& args]
+  (let [init-val {:type :spout
+                  :sid->output-stats {}
+                  :executor-stats []
+                  :window->emitted {}
+                  :window->transferred {}
+                  :window->comp-lat-wgt-avg {}
+                  :window->acked {}
+                  :window->failed {}}]
+    (apply aggregate-comp-stats* (concat args (list init-val)))))
+
+(defmethod aggregate-comp-stats :default [& _] {})
+
+(defmulti post-aggregate-comp-stats
+  (fn [_ _ data] (:type data)))
+
+(defmethod post-aggregate-comp-stats :bolt
+  [task->component
+   exec->host+port
+   {{i-stats :cid+sid->input-stats
+     o-stats :sid->output-stats
+     num-tasks :num-tasks
+     num-executors :num-executors} :stats
+    comp-type :type :as acc-data}]
+  {:type comp-type
+   :num-tasks num-tasks
+   :num-executors num-executors
+   :cid+sid->input-stats
+   (->> i-stats
+        (map-val (fn [m]
+                     (let [executed (:executed m)
+                           lats (if (and executed (pos? executed))
+                                  {:execute-latency
+                                   (div (or (:executeLatencyTotal m) 0)
+                                        executed)
+                                   :process-latency
+                                   (div (or (:processLatencyTotal m) 0)
+                                        executed)}
+                                  {:execute-latency 0
+                                   :process-latency 0})]
+                       (-> m (merge lats) (dissoc :executeLatencyTotal
+                                                  :processLatencyTotal))))))
+   :sid->output-stats o-stats
+   :executor-stats (:executor-stats (:stats acc-data))
+   :window->emitted (map-key str (:window->emitted acc-data))
+   :window->transferred (map-key str (:window->transferred acc-data))
+   :window->execute-latency
+     (compute-weighted-averages-per-window acc-data
+                                           :window->exec-lat-wgt-avg
+                                           :window->executed)
+   :window->executed (map-key str (:window->executed acc-data))
+   :window->process-latency
+     (compute-weighted-averages-per-window acc-data
+                                           :window->proc-lat-wgt-avg
+                                           :window->executed)
+   :window->acked (map-key str (:window->acked acc-data))
+   :window->failed (map-key str (:window->failed acc-data))})
+
+(defmethod post-aggregate-comp-stats :spout
+  [task->component
+   exec->host+port
+   {{o-stats :sid->output-stats
+     num-tasks :num-tasks
+     num-executors :num-executors} :stats
+    comp-type :type :as acc-data}]
+  {:type comp-type
+   :num-tasks num-tasks
+   :num-executors num-executors
+   :sid->output-stats
+   (->> o-stats
+        (map-val (fn [m]
+                     (let [acked (:acked m)
+                           lat (if (and acked (pos? acked))
+                                 {:complete-latency
+                                  (div (or (:completeLatencyTotal m) 0) acked)}
+                                 {:complete-latency 0})]
+                       (-> m (merge lat) (dissoc :completeLatencyTotal))))))
+   :executor-stats (:executor-stats (:stats acc-data))
+   :window->emitted (map-key str (:window->emitted acc-data))
+   :window->transferred (map-key str (:window->transferred acc-data))
+   :window->complete-latency
+     (compute-weighted-averages-per-window acc-data
+                                           :window->comp-lat-wgt-avg
+                                           :window->acked)
+   :window->acked (map-key str (:window->acked acc-data))
+   :window->failed (map-key str (:window->failed acc-data))})
+
+(defmethod post-aggregate-comp-stats :default [& _] {})
+
+(defn thriftify-exec-agg-stats
+  [comp-id comp-type {:keys [executor-id host port uptime] :as stats}]
+  (doto (ExecutorAggregateStats.)
+    (.set_exec_summary (ExecutorSummary. (apply #(ExecutorInfo. %1 %2)
+                                                executor-id)
+                                         comp-id
+                                         host
+                                         port
+                                         (or uptime 0)))
+    (.set_stats ((condp = comp-type
+                   :bolt thriftify-bolt-agg-stats
+                   :spout thriftify-spout-agg-stats) stats))))
+
+(defn- thriftify-bolt-input-stats
+  [cid+sid->input-stats]
+  (into {} (for [[cid+sid input-stats] cid+sid->input-stats]
+             [(to-global-stream-id cid+sid)
+              (thriftify-bolt-agg-stats input-stats)])))
+
+(defn- thriftify-bolt-output-stats
+  [sid->output-stats]
+  (map-val thriftify-bolt-agg-stats sid->output-stats))
+
+(defn- thriftify-spout-output-stats
+  [sid->output-stats]
+  (map-val thriftify-spout-agg-stats sid->output-stats))
+
+(defn thriftify-comp-page-data
+  [topo-id topology comp-id data]
+  (let [w->stats (swap-map-order
+                   (merge
+                     {:emitted (:window->emitted data)
+                      :transferred (:window->transferred data)
+                      :acked (:window->acked data)
+                      :failed (:window->failed data)}
+                     (condp = (:type data)
+                       :bolt {:execute-latency (:window->execute-latency data)
+                              :process-latency (:window->process-latency data)
+                              :executed (:window->executed data)}
+                       :spout {:complete-latency
+                               (:window->complete-latency data)}
+                       {}))) ; default
+        [compType exec-stats w->stats gsid->input-stats sid->output-stats]
+          (condp = (component-type topology comp-id)
+            :bolt [ComponentType/BOLT
+                   (->
+                     (partial thriftify-exec-agg-stats comp-id :bolt)
+                     (map (:executor-stats data)))
+                   (map-val thriftify-bolt-agg-stats w->stats)
+                   (thriftify-bolt-input-stats (:cid+sid->input-stats data))
+                   (thriftify-bolt-output-stats (:sid->output-stats data))]
+            :spout [ComponentType/SPOUT
+                    (->
+                      (partial thriftify-exec-agg-stats comp-id :spout)
+                      (map (:executor-stats data)))
+                    (map-val thriftify-spout-agg-stats w->stats)
+                    nil ;; spouts do not have input stats
+                    (thriftify-spout-output-stats (:sid->output-stats data))]),
+        num-executors (:num-executors data)
+        num-tasks (:num-tasks data)
+        ret (doto (ComponentPageInfo. comp-id compType)
+              (.set_topology_id topo-id)
+              (.set_topology_name nil)
+              (.set_window_to_stats w->stats)
+              (.set_sid_to_output_stats sid->output-stats)
+              (.set_exec_stats exec-stats))]
+    (and num-executors (.set_num_executors ret num-executors))
+    (and num-tasks (.set_num_tasks ret num-tasks))
+    (and gsid->input-stats
+         (.set_gsid_to_input_stats ret gsid->input-stats))
+    ret))
+
+(defn agg-comp-execs-stats
+  "Aggregate various executor statistics for a component from the given
+  heartbeats."
+  [exec->host+port
+   task->component
+   beats
+   window
+   include-sys?
+   topology-id
+   topology
+   component-id]
+  (->> ;; This iterates over each executor one time, because of lazy evaluation.
+    (extract-data-from-hb exec->host+port
+                          task->component
+                          beats
+                          include-sys?
+                          topology
+                          component-id)
+    (aggregate-comp-stats window include-sys?)
+    (post-aggregate-comp-stats task->component exec->host+port)
+    (thriftify-comp-page-data topology-id topology component-id)))
+
+(defn expand-averages
+  [avg counts]
+  (let [avg (clojurify-structure avg)
+        counts (clojurify-structure counts)]
+    (into {}
+          (for [[slice streams] counts]
+            [slice
+             (into {}
+                   (for [[stream c] streams]
+                     [stream
+                      [(* c (get-in avg [slice stream]))
+                       c]]
+                     ))]))))
+
+(defn expand-averages-seq
+  [average-seq counts-seq]
+  (->> (map vector average-seq counts-seq)
+       (map #(apply expand-averages %))
+       (apply merge-with (fn [s1 s2] (merge-with add-pairs s1 s2)))))
+
+(defn- val-avg
+  [[t c]]
+  (if (= t 0) 0
+    (double (/ t c))))
+
+(defn aggregate-averages
+  [average-seq counts-seq]
+  (->> (expand-averages-seq average-seq counts-seq)
+       (map-val
+         (fn [s]
+           (map-val val-avg s)))))
+
+(defn aggregate-avg-streams
+  [avg counts]
+  (let [expanded (expand-averages avg counts)]
+    (->> expanded
+         (map-val #(reduce add-pairs (vals %)))
+         (map-val val-avg))))
+
+(defn pre-process
+  [stream-summary include-sys?]
+  (let [filter-fn (mk-include-sys-fn include-sys?)
+        emitted (:emitted stream-summary)
+        emitted (into {} (for [[window stat] emitted]
+                           {window (filter-key filter-fn stat)}))
+        transferred (:transferred stream-summary)
+        transferred (into {} (for [[window stat] transferred]
+                               {window (filter-key filter-fn stat)}))
+        stream-summary (-> stream-summary (dissoc :emitted) (assoc :emitted emitted))
+        stream-summary (-> stream-summary (dissoc :transferred) (assoc :transferred transferred))]
+    stream-summary))
+
+(defn aggregate-counts
+  [counts-seq]
+  (->> counts-seq
+       (map clojurify-structure)
+       (apply merge-with
+              (fn [s1 s2]
+                (merge-with + s1 s2)))))
+
+(defn aggregate-common-stats
+  [stats-seq]
+  {:emitted (aggregate-counts (map #(.get_emitted ^ExecutorStats %) stats-seq))
+   :transferred (aggregate-counts (map #(.get_transferred ^ExecutorStats %) stats-seq))})
+
+(defn aggregate-bolt-stats
+  [stats-seq include-sys?]
+  (let [stats-seq (collectify stats-seq)]
+    (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
+           {:acked
+            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_acked)
+                                   stats-seq))
+            :failed
+            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_failed)
+                                   stats-seq))
+            :executed
+            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_executed)
+                                   stats-seq))
+            :process-latencies
+            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_process_ms_avg)
+                                     stats-seq)
+                                (map #(.. ^ExecutorStats % get_specific get_bolt get_acked)
+                                     stats-seq))
+            :execute-latencies
+            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_execute_ms_avg)
+                                     stats-seq)
+                                (map #(.. ^ExecutorStats % get_specific get_bolt get_executed)
+                                     stats-seq))})))
+
+(defn aggregate-spout-stats
+  [stats-seq include-sys?]
+  (let [stats-seq (collectify stats-seq)]
+    (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
+           {:acked
+            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_acked)
+                                   stats-seq))
+            :failed
+            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_failed)
+                                   stats-seq))
+            :complete-latencies
+            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_spout get_complete_ms_avg)
+                                     stats-seq)
+                                (map #(.. ^ExecutorStats % get_specific get_spout get_acked)
+                                     stats-seq))})))
+
+(defn get-filled-stats
+  [summs]
+  (->> summs
+       (map #(.get_stats ^ExecutorSummary %))
+       (filter not-nil?)))
+
+(defn aggregate-spout-streams
+  [stats]
+  {:acked (aggregate-count-streams (:acked stats))
+   :failed (aggregate-count-streams (:failed stats))
+   :emitted (aggregate-count-streams (:emitted stats))
+   :transferred (aggregate-count-streams (:transferred stats))
+   :complete-latencies (aggregate-avg-streams (:complete-latencies stats)
+                                              (:acked stats))})
+
+(defn spout-streams-stats
+  [summs include-sys?]
+  (let [stats-seq (get-filled-stats summs)]
+    (aggregate-spout-streams
+      (aggregate-spout-stats
+        stats-seq include-sys?))))
+
+(defn aggregate-bolt-streams
+  [stats]
+  {:acked (aggregate-count-streams (:acked stats))
+   :failed (aggregate-count-streams (:failed stats))
+   :emitted (aggregate-count-streams (:emitted stats))
+   :transferred (aggregate-count-streams (:transferred stats))
+   :process-latencies (aggregate-avg-streams (:process-latencies stats)
+                                             (:acked stats))
+   :executed (aggregate-count-streams (:executed stats))
+   :execute-latencies (aggregate-avg-streams (:execute-latencies stats)
+                                             (:executed stats))})
+
+(defn compute-executor-capacity
+  [^ExecutorSummary e]
+  (let [stats (.get_stats e)
+        stats (if stats
+                (-> stats
+                    (aggregate-bolt-stats true)
+                    (aggregate-bolt-streams)
+                    swap-map-order
+                    (get (str TEN-MIN-IN-SECONDS))))
+        uptime (nil-to-zero (.get_uptime_secs e))
+        window (if (< uptime TEN-MIN-IN-SECONDS) uptime TEN-MIN-IN-SECONDS)
+        executed (-> stats :executed nil-to-zero)
+        latency (-> stats :execute-latencies nil-to-zero)]
+    (if (> window 0)
+      (div (* executed latency) (* 1000 window)))))
+
+(defn bolt-streams-stats
+  [summs include-sys?]
+  (let [stats-seq (get-filled-stats summs)]
+    (aggregate-bolt-streams
+      (aggregate-bolt-stats
+        stats-seq include-sys?))))
+
+(defn total-aggregate-stats
+  [spout-summs bolt-summs include-sys?]
+  (let [spout-stats (get-filled-stats spout-summs)
+        bolt-stats (get-filled-stats bolt-summs)
+        agg-spout-stats (-> spout-stats
+                            (aggregate-spout-stats include-sys?)
+                            aggregate-spout-streams)
+        agg-bolt-stats (-> bolt-stats
+                           (aggregate-bolt-stats include-sys?)
+                           aggregate-bolt-streams)]
+    (merge-with
+      (fn [s1 s2]
+        (merge-with + s1 s2))
+      (select-keys
+        agg-bolt-stats
+        ;; Include only keys that will be used.  We want to count acked and
+        ;; failed only for the "tuple trees," so we do not include those keys
+        ;; from the bolt executors.
+        [:emitted :transferred])
+      agg-spout-stats)))
+
+(defn error-subset
+  [error-str]
+  (apply str (take 200 error-str)))
+
+(defn most-recent-error
+  [errors-list]
+  (let [error (->> errors-list
+                   (sort-by #(.get_error_time_secs ^ErrorInfo %))
+                   reverse
+                   first)]
+    (if error
+      (error-subset (.get_error ^ErrorInfo error))
+      "")))
+
+(defn float-str [n]
+  (if n
+    (format "%.3f" (float n))
+    "0"))
+
+(defn compute-bolt-capacity
+  [executors]
+  (->> executors
+       (map compute-executor-capacity)
+       (map nil-to-zero)
+       (apply max)))
diff --git a/storm-core/src/clj/backtype/storm/testing.clj b/storm-core/src/clj/backtype/storm/testing.clj
index b0c6637..2c98b07 100644
--- a/storm-core/src/clj/backtype/storm/testing.clj
+++ b/storm-core/src/clj/backtype/storm/testing.clj
@@ -281,25 +281,42 @@
     (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
   (.submitTopologyWithOpts nimbus storm-name nil (to-json conf) topology submit-opts))
 
-(defn mocked-compute-new-topology->executor->node+port [storm-name executor->node+port]
-  (fn [nimbus existing-assignments topologies scratch-topology-id]
-    (let [topology (.getByName topologies storm-name)
-          topology-id (.getId topology)
+(defn mocked-convert-assignments-to-worker->resources [storm-cluster-state storm-name worker->resources]
+  (fn [existing-assignments]
+    (let [topology-id (common/get-storm-id storm-cluster-state storm-name)
+          existing-assignments (into {} (for [[tid assignment] existing-assignments]
+                                          {tid (:worker->resources assignment)}))
+          new-assignments (assoc existing-assignments topology-id worker->resources)]
+      new-assignments)))
+
+(defn mocked-compute-new-topology->executor->node+port [storm-cluster-state storm-name executor->node+port]
+  (fn [new-scheduler-assignments existing-assignments]
+    (let [topology-id (common/get-storm-id storm-cluster-state storm-name)
           existing-assignments (into {} (for [[tid assignment] existing-assignments]
                                           {tid (:executor->node+port assignment)}))
           new-assignments (assoc existing-assignments topology-id executor->node+port)]
       new-assignments)))
 
+(defn mocked-compute-new-scheduler-assignments []
+  (fn [nimbus existing-assignments topologies scratch-topology-id]
+    existing-assignments))
+
 (defn submit-mocked-assignment
-  [nimbus storm-name conf topology task->component executor->node+port]
+  [nimbus storm-cluster-state storm-name conf topology task->component executor->node+port worker->resources]
   (with-var-roots [common/storm-task-info (fn [& ignored] task->component)
+                   nimbus/compute-new-scheduler-assignments (mocked-compute-new-scheduler-assignments)
+                   nimbus/convert-assignments-to-worker->resources (mocked-convert-assignments-to-worker->resources
+                                                          storm-cluster-state
+                                                          storm-name
+                                                          worker->resources)
                    nimbus/compute-new-topology->executor->node+port (mocked-compute-new-topology->executor->node+port
+                                                                      storm-cluster-state
                                                                       storm-name
                                                                       executor->node+port)]
-                  (submit-local-topology nimbus storm-name conf topology)))
+    (submit-local-topology nimbus storm-name conf topology)))
 
 (defn mk-capture-launch-fn [capture-atom]
-  (fn [supervisor storm-id port worker-id]
+  (fn [supervisor storm-id port worker-id mem-onheap]
     (let [supervisor-id (:supervisor-id supervisor)
           conf (:conf supervisor)
           existing (get @capture-atom [supervisor-id port] [])]
@@ -498,6 +515,7 @@
       (startup spout))
 
     (submit-local-topology (:nimbus cluster-map) storm-name storm-conf topology)
+    (advance-cluster-time cluster-map 11)
 
     (let [storm-id (common/get-storm-id state storm-name)]
       ;;Give the topology time to come up without using it to wait for the spouts to complete
@@ -594,7 +612,7 @@
                 ;; (log-message "Transferring: " transfer-args#)
                 (increment-global! id# "transferred" 1)
                 (apply transferrer# args2#)))))]
-       (with-local-cluster [~cluster-sym ~@cluster-args]
+       (with-simulated-time-local-cluster [~cluster-sym ~@cluster-args]
                            (let [~cluster-sym (assoc-track-id ~cluster-sym id#)]
                              ~@body)))
      (RegisteredGlobalState/clearState id#)))
diff --git a/storm-core/src/clj/backtype/storm/timer.clj b/storm-core/src/clj/backtype/storm/timer.clj
index 2c76ce2..b5f73f7 100644
--- a/storm-core/src/clj/backtype/storm/timer.clj
+++ b/storm-core/src/clj/backtype/storm/timer.clj
@@ -16,7 +16,7 @@
 
 (ns backtype.storm.timer
   (:import [backtype.storm.utils Time])
-  (:import [java.util PriorityQueue Comparator])
+  (:import [java.util PriorityQueue Comparator Random])
   (:import [java.util.concurrent Semaphore])
   (:use [backtype.storm util log]))
 
@@ -79,6 +79,7 @@
      :queue queue
      :active active
      :lock lock
+     :random (Random.)
      :cancel-notifier notifier}))
 
 (defn- check-active!
@@ -87,12 +88,14 @@
     (throw (IllegalStateException. "Timer is not active"))))
 
 (defnk schedule
-  [timer delay-secs afn :check-active true]
+  [timer delay-secs afn :check-active true :jitter-ms 0]
   (when check-active (check-active! timer))
   (let [id (uuid)
-        ^PriorityQueue queue (:queue timer)]
+        ^PriorityQueue queue (:queue timer)
+        end-time-ms (+ (current-time-millis) (secs-to-millis-long delay-secs))
+        end-time-ms (if (< 0 jitter-ms) (+ (.nextInt (:random timer) jitter-ms) end-time-ms) end-time-ms)]
     (locking (:lock timer)
-      (.add queue [(+ (current-time-millis) (secs-to-millis-long delay-secs)) afn id]))))
+      (.add queue [end-time-ms afn id]))))
 
 (defn schedule-recurring
   [timer delay-secs recur-secs afn]
@@ -103,6 +106,15 @@
               ; This avoids a race condition with cancel-timer.
               (schedule timer recur-secs this :check-active false))))
 
+(defn schedule-recurring-with-jitter
+  [timer delay-secs recur-secs jitter-ms afn]
+  (schedule timer
+            delay-secs
+            (fn this []
+              (afn)
+              ; This avoids a race condition with cancel-timer.
+              (schedule timer recur-secs this :check-active false :jitter-ms jitter-ms))))
+
 (defn cancel-timer
   [timer]
   (check-active! timer)
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index 414bfb1..cc6632a 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -21,43 +21,67 @@
         ring.middleware.multipart-params)
   (:use [ring.middleware.json :only [wrap-json-params]])
   (:use [hiccup core page-helpers])
-  (:use [backtype.storm config util log tuple zookeeper])
+  (:use [backtype.storm config util log stats tuple zookeeper converter])
   (:use [backtype.storm.ui helpers])
   (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
-                                              ACKER-FAIL-STREAM-ID system-id? mk-authorization-handler]]])
-  (:use [clojure.string :only [blank? lower-case trim]])
+                                              ACKER-FAIL-STREAM-ID mk-authorization-handler
+                                              start-metrics-reporters]]])
   (:import [backtype.storm.utils Utils]
            [backtype.storm.generated NimbusSummary])
+  (:use [clojure.string :only [blank? lower-case trim split]])
   (:import [backtype.storm.generated ExecutorSpecificStats
             ExecutorStats ExecutorSummary ExecutorInfo TopologyInfo SpoutStats BoltStats
             ErrorInfo ClusterSummary SupervisorSummary TopologySummary
             Nimbus$Client StormTopology GlobalStreamId RebalanceOptions
-            KillOptions GetInfoOptions NumErrorsChoice DebugOptions])
+            KillOptions GetInfoOptions NumErrorsChoice DebugOptions TopologyPageInfo
+            TopologyStats CommonAggregateStats ComponentAggregateStats
+            ComponentType BoltAggregateStats SpoutAggregateStats
+            ExecutorAggregateStats SpecificAggregateStats ComponentPageInfo
+            LogConfig LogLevel LogLevelAction])
   (:import [backtype.storm.security.auth AuthUtils ReqContext])
-  (:import [backtype.storm.generated AuthorizationException])
+  (:import [backtype.storm.generated AuthorizationException ProfileRequest ProfileAction NodeInfo])
   (:import [backtype.storm.security.auth AuthUtils])
   (:import [backtype.storm.utils VersionInfo])
+  (:import [backtype.storm Config])
   (:import [java.io File])
   (:require [compojure.route :as route]
             [compojure.handler :as handler]
             [ring.util.response :as resp]
             [backtype.storm [thrift :as thrift]])
+  (:require [metrics.meters :refer [defmeter mark!]])
   (:import [org.apache.commons.lang StringEscapeUtils])
+  (:import [org.apache.logging.log4j Level])
   (:gen-class))
 
 (def ^:dynamic *STORM-CONF* (read-storm-config))
 (def ^:dynamic *UI-ACL-HANDLER* (mk-authorization-handler (*STORM-CONF* NIMBUS-AUTHORIZER) *STORM-CONF*))
 (def ^:dynamic *UI-IMPERSONATION-HANDLER* (mk-authorization-handler (*STORM-CONF* NIMBUS-IMPERSONATION-AUTHORIZER) *STORM-CONF*))
-
 (def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
 
-(defn assert-authorized-user
-  ([servlet-request op]
-    (assert-authorized-user servlet-request op nil))
-  ([servlet-request op topology-conf]
-    (let [context (ReqContext/context)]
-      (if http-creds-handler (.populateContext http-creds-handler context servlet-request))
+(defmeter ui:num-cluster-configuration-http-requests)
+(defmeter ui:num-cluster-summary-http-requests)
+(defmeter ui:num-nimbus-summary-http-requests)
+(defmeter ui:num-supervisor-summary-http-requests)
+(defmeter ui:num-all-topologies-summary-http-requests)
+(defmeter ui:num-topology-page-http-requests)
+(defmeter ui:num-build-visualization-http-requests)
+(defmeter ui:num-mk-visualization-data-http-requests)
+(defmeter ui:num-component-page-http-requests)
+(defmeter ui:num-log-config-http-requests)
+(defmeter ui:num-activate-topology-http-requests)
+(defmeter ui:num-deactivate-topology-http-requests)
+(defmeter ui:num-debug-topology-http-requests)
+(defmeter ui:num-component-op-response-http-requests)
+(defmeter ui:num-topology-op-response-http-requests)
+(defmeter ui:num-topology-op-response-http-requests)
+(defmeter ui:num-topology-op-response-http-requests)
+(defmeter ui:num-main-page-http-requests)
 
+(defn assert-authorized-user
+  ([op]
+    (assert-authorized-user op nil))
+  ([op topology-conf]
+    (let [context (ReqContext/context)]
       (if (.isImpersonating context)
         (if *UI-IMPERSONATION-HANDLER*
             (if-not (.permit *UI-IMPERSONATION-HANDLER* context op topology-conf)
@@ -80,94 +104,18 @@
            (throw (AuthorizationException.
                    (str "UI request '" op "' for '" user "' user is not authorized")))))))))
 
-(defn get-filled-stats
-  [summs]
-  (->> summs
-       (map #(.get_stats ^ExecutorSummary %))
-       (filter not-nil?)))
 
-(defn component-type
-  "Returns the component type (either :bolt or :spout) for a given
-  topology and component id. Returns nil if not found."
-  [^StormTopology topology id]
-  (let [bolts (.get_bolts topology)
-        spouts (.get_spouts topology)]
-    (cond
-      (.containsKey bolts id) :bolt
-      (.containsKey spouts id) :spout)))
+(defn assert-authorized-profiler-action
+  [op]
+  (if-not (*STORM-CONF* WORKER-PROFILER-ENABLED)
+    (throw (AuthorizationException.
+             (str "UI request for profiler action '" op "' is disabled.")))))
+
 
 (defn executor-summary-type
   [topology ^ExecutorSummary s]
   (component-type topology (.get_component_id s)))
 
-(defn add-pairs
-  ([] [0 0])
-  ([[a1 a2] [b1 b2]]
-   [(+ a1 b1) (+ a2 b2)]))
-
-(defn expand-averages
-  [avg counts]
-  (let [avg (clojurify-structure avg)
-        counts (clojurify-structure counts)]
-    (into {}
-          (for [[slice streams] counts]
-            [slice
-             (into {}
-                   (for [[stream c] streams]
-                     [stream
-                      [(* c (get-in avg [slice stream]))
-                       c]]
-                     ))]))))
-
-(defn expand-averages-seq
-  [average-seq counts-seq]
-  (->> (map vector average-seq counts-seq)
-       (map #(apply expand-averages %))
-       (apply merge-with (fn [s1 s2] (merge-with add-pairs s1 s2)))))
-
-(defn- val-avg
-  [[t c]]
-  (if (= t 0) 0
-    (double (/ t c))))
-
-(defn aggregate-averages
-  [average-seq counts-seq]
-  (->> (expand-averages-seq average-seq counts-seq)
-       (map-val
-         (fn [s]
-           (map-val val-avg s)))))
-
-(defn aggregate-counts
-  [counts-seq]
-  (->> counts-seq
-       (map clojurify-structure)
-       (apply merge-with
-              (fn [s1 s2]
-                (merge-with + s1 s2)))))
-
-(defn aggregate-avg-streams
-  [avg counts]
-  (let [expanded (expand-averages avg counts)]
-    (->> expanded
-         (map-val #(reduce add-pairs (vals %)))
-         (map-val val-avg))))
-
-(defn aggregate-count-streams
-  [stats]
-  (->> stats
-       (map-val #(reduce + (vals %)))))
-
-(defn aggregate-common-stats
-  [stats-seq]
-  {:emitted (aggregate-counts (map #(.get_emitted ^ExecutorStats %) stats-seq))
-   :transferred (aggregate-counts (map #(.get_transferred ^ExecutorStats %) stats-seq))})
-
-(defn mk-include-sys-fn
-  [include-sys?]
-  (if include-sys?
-    (fn [_] true)
-    (fn [stream] (and (string? stream) (not (system-id? stream))))))
-
 (defn is-ack-stream
   [stream]
   (let [acker-streams
@@ -176,80 +124,6 @@
          ACKER-FAIL-STREAM-ID]]
     (every? #(not= %1 stream) acker-streams)))
 
-(defn pre-process
-  [stream-summary include-sys?]
-  (let [filter-fn (mk-include-sys-fn include-sys?)
-        emitted (:emitted stream-summary)
-        emitted (into {} (for [[window stat] emitted]
-                           {window (filter-key filter-fn stat)}))
-        transferred (:transferred stream-summary)
-        transferred (into {} (for [[window stat] transferred]
-                               {window (filter-key filter-fn stat)}))
-        stream-summary (-> stream-summary (dissoc :emitted) (assoc :emitted emitted))
-        stream-summary (-> stream-summary (dissoc :transferred) (assoc :transferred transferred))]
-    stream-summary))
-
-(defn aggregate-bolt-stats
-  [stats-seq include-sys?]
-  (let [stats-seq (collectify stats-seq)]
-    (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
-           {:acked
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_acked)
-                                   stats-seq))
-            :failed
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_failed)
-                                   stats-seq))
-            :executed
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_bolt get_executed)
-                                   stats-seq))
-            :process-latencies
-            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_process_ms_avg)
-                                     stats-seq)
-                                (map #(.. ^ExecutorStats % get_specific get_bolt get_acked)
-                                     stats-seq))
-            :execute-latencies
-            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_bolt get_execute_ms_avg)
-                                     stats-seq)
-                                (map #(.. ^ExecutorStats % get_specific get_bolt get_executed)
-                                     stats-seq))})))
-
-(defn aggregate-spout-stats
-  [stats-seq include-sys?]
-  (let [stats-seq (collectify stats-seq)]
-    (merge (pre-process (aggregate-common-stats stats-seq) include-sys?)
-           {:acked
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_acked)
-                                   stats-seq))
-            :failed
-            (aggregate-counts (map #(.. ^ExecutorStats % get_specific get_spout get_failed)
-                                   stats-seq))
-            :complete-latencies
-            (aggregate-averages (map #(.. ^ExecutorStats % get_specific get_spout get_complete_ms_avg)
-                                     stats-seq)
-                                (map #(.. ^ExecutorStats % get_specific get_spout get_acked)
-                                     stats-seq))})))
-
-(defn aggregate-bolt-streams
-  [stats]
-  {:acked (aggregate-count-streams (:acked stats))
-   :failed (aggregate-count-streams (:failed stats))
-   :emitted (aggregate-count-streams (:emitted stats))
-   :transferred (aggregate-count-streams (:transferred stats))
-   :process-latencies (aggregate-avg-streams (:process-latencies stats)
-                                             (:acked stats))
-   :executed (aggregate-count-streams (:executed stats))
-   :execute-latencies (aggregate-avg-streams (:execute-latencies stats)
-                                             (:executed stats))})
-
-(defn aggregate-spout-streams
-  [stats]
-  {:acked (aggregate-count-streams (:acked stats))
-   :failed (aggregate-count-streams (:failed stats))
-   :emitted (aggregate-count-streams (:emitted stats))
-   :transferred (aggregate-count-streams (:transferred stats))
-   :complete-latencies (aggregate-avg-streams (:complete-latencies stats)
-                                              (:acked stats))})
-
 (defn spout-summary?
   [topology s]
   (= :spout (executor-summary-type topology s)))
@@ -263,29 +137,6 @@
   (let [ret (group-by #(.get_component_id ^ExecutorSummary %) summs)]
     (into (sorted-map) ret )))
 
-(defn error-subset
-  [error-str]
-  (apply str (take 200 error-str)))
-
-(defn most-recent-error
-  [errors-list]
-  (let [error (->> errors-list
-                   (sort-by #(.get_error_time_secs ^ErrorInfo %))
-                   reverse
-                   first)]
-     error))
-
-(defn component-task-summs
-  [^TopologyInfo summ topology id]
-  (let [spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
-        bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
-        spout-comp-summs (group-by-comp spout-summs)
-        bolt-comp-summs (group-by-comp bolt-summs)
-        ret (if (contains? spout-comp-summs id)
-              (spout-comp-summs id)
-              (bolt-comp-summs id))]
-    (sort-by #(-> ^ExecutorSummary % .get_executor_info .get_task_start) ret)))
-
 (defn logviewer-link [host fname secure?]
   (if (and secure? (*STORM-CONF* LOGVIEWER-HTTPS-PORT))
     (url-format "https://%s:%s/log?file=%s"
@@ -295,69 +146,18 @@
     (url-format "http://%s:%s/log?file=%s"
       host
       (*STORM-CONF* LOGVIEWER-PORT)
-      fname))
-  )
-
-(defn executor-has-task-id? [task-id executor-info]
-  (between? task-id (.get_task_start executor-info) (.get_task_end executor-info)))
-
-(defn get-host-port [task-id executor-summs]
-  (let [ex-sum (some #(if (executor-has-task-id? task-id (.get_executor_info %)) %) executor-summs)]
-    {:host (.get_host ex-sum) :port (.get_port ex-sum)}))
-
-(defn get-sorted-eventlogger-task-ids [executor-summs]
-  (let [executor-infos (map #(.get_executor_info %) executor-summs)]
-  (sort (flatten (map #(range (.get_task_start %) (inc (.get_task_end %))) executor-infos)))))
-
-(defn get-eventlogger-executor-summs [^TopologyInfo topology-info topology]
-  (let [bolt-summs (filter (partial bolt-summary? topology) (.get_executors topology-info))]
-        ((group-by-comp bolt-summs) "__eventlogger")))
-
-;
-; The eventlogger uses fields grouping on the component-id so that events from same component
-; always goes to the same event logger task. Here we use the same fields grouping
-; to find the correct eventlogger task.
-(defn get-mapped-task-id [sorted-task-ids ^String component-id]
-  (nth sorted-task-ids (mod (list-hash-code [component-id]) (count sorted-task-ids))))
+      fname)))
 
 (defn event-log-link
-  [topology-id ^TopologyInfo topology-info topology component-id secure?]
-  (let [executor-summs (get-eventlogger-executor-summs topology-info topology)
-        sorted-task-ids (get-sorted-eventlogger-task-ids executor-summs)
-        mapped-task-id (get-mapped-task-id sorted-task-ids component-id)
-        host-port (get-host-port mapped-task-id executor-summs)
-        fname (event-logs-filename topology-id (host-port :port))]
-    (logviewer-link (host-port :host) fname secure?)))
+  [topology-id component-id host port secure?]
+  (logviewer-link host (event-logs-filename topology-id port) secure?))
 
 (defn worker-log-link [host port topology-id secure?]
   (let [fname (logs-filename topology-id port)]
     (logviewer-link host fname secure?)))
 
 (defn nimbus-log-link [host port]
-  (url-format "http://%s:%s/log?file=nimbus.log" host (*STORM-CONF* LOGVIEWER-PORT) port))
-
-(defn compute-executor-capacity
-  [^ExecutorSummary e]
-  (let [stats (.get_stats e)
-        stats (if stats
-                (-> stats
-                    (aggregate-bolt-stats true)
-                    (aggregate-bolt-streams)
-                    swap-map-order
-                    (get "600")))
-        uptime (nil-to-zero (.get_uptime_secs e))
-        window (if (< uptime 600) uptime 600)
-        executed (-> stats :executed nil-to-zero)
-        latency (-> stats :execute-latencies nil-to-zero)]
-    (if (> window 0)
-      (div (* executed latency) (* 1000 window)))))
-
-(defn compute-bolt-capacity
-  [executors]
-  (->> executors
-       (map compute-executor-capacity)
-       (map nil-to-zero)
-       (apply max)))
+  (url-format "http://%s:%s/daemonlog?file=nimbus.log" host (*STORM-CONF* LOGVIEWER-PORT) port))
 
 (defn get-error-time
   [error]
@@ -371,7 +171,7 @@
     ""))
 
 (defn get-error-port
-  [error error-host top-id]
+  [error]
   (if error
     (.get_port ^ErrorInfo error)
     ""))
@@ -382,40 +182,18 @@
     (.get_host ^ErrorInfo error)
     ""))
 
-(defn spout-streams-stats
-  [summs include-sys?]
-  (let [stats-seq (get-filled-stats summs)]
-    (aggregate-spout-streams
-      (aggregate-spout-stats
-        stats-seq include-sys?))))
+(defn get-error-time
+  [error]
+  (if error
+    (.get_error_time_secs ^ErrorInfo error)
+    ""))
 
-(defn bolt-streams-stats
-  [summs include-sys?]
-  (let [stats-seq (get-filled-stats summs)]
-    (aggregate-bolt-streams
-      (aggregate-bolt-stats
-        stats-seq include-sys?))))
-
-(defn total-aggregate-stats
-  [spout-summs bolt-summs include-sys?]
-  (let [spout-stats (get-filled-stats spout-summs)
-        bolt-stats (get-filled-stats bolt-summs)
-        agg-spout-stats (-> spout-stats
-                            (aggregate-spout-stats include-sys?)
-                            aggregate-spout-streams)
-        agg-bolt-stats (-> bolt-stats
-                           (aggregate-bolt-stats include-sys?)
-                           aggregate-bolt-streams)]
-    (merge-with
-      (fn [s1 s2]
-        (merge-with + s1 s2))
-      (select-keys
-        agg-bolt-stats
-        ;; Include only keys that will be used.  We want to count acked and
-        ;; failed only for the "tuple trees," so we do not include those keys
-        ;; from the bolt executors.
-        [:emitted :transferred])
-      agg-spout-stats)))
+(defn worker-dump-link [host port topology-id]
+  (url-format "http://%s:%s/dumps/%s/%s"
+              (url-encode host)
+              (*STORM-CONF* LOGVIEWER-PORT)
+              (url-encode topology-id)
+              (str (url-encode host) ":" (url-encode port))))
 
 (defn stats-times
   [stats-map]
@@ -431,16 +209,6 @@
     "All time"
     (pretty-uptime-sec window)))
 
-(defn topology-action-button
-  [id name action command is-wait default-wait enabled]
-  [:input {:type "button"
-           :value action
-           (if enabled :enabled :disabled) ""
-           :onclick (str "confirmAction('"
-                         (StringEscapeUtils/escapeJavaScript id) "', '"
-                         (StringEscapeUtils/escapeJavaScript name) "', '"
-                         command "', " is-wait ", " default-wait ")")}])
-
 (defn sanitize-stream-name
   [name]
   (let [sym-regex #"(?![A-Za-z_\-:\.])."]
@@ -516,6 +284,12 @@
     (map (fn [row]
            {:row row}) (partition 4 4 nil streams))))
 
+(defn- get-topology-info
+  ([^Nimbus$Client nimbus id]
+    (.getTopologyInfo nimbus id))
+  ([^Nimbus$Client nimbus id options]
+    (.getTopologyInfoWithOpts nimbus id options)))
+
 (defn mk-visualization-data
   [id window include-sys?]
   (thrift/with-configured-nimbus-connection
@@ -534,9 +308,7 @@
           spout-comp-summs (group-by-comp spout-summs)
           bolt-comp-summs (group-by-comp bolt-summs)
           bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?)
-                                      bolt-comp-summs)
-          topology-conf (from-json
-                          (.getTopologyConf ^Nimbus$Client nimbus id))]
+                                      bolt-comp-summs)]
       (visualization-data
        (merge (hashmap-to-persistent spouts)
               (hashmap-to-persistent bolts))
@@ -579,6 +351,14 @@
   (thrift/with-configured-nimbus-connection nimbus
     (.getNimbusConf ^Nimbus$Client nimbus)))
 
+(defn topology-history-info
+  ([user]
+    (thrift/with-configured-nimbus-connection nimbus
+      (topology-history-info (.getTopologyHistory ^Nimbus$Client nimbus user) user)))
+  ([history user]
+    {"topo-history"
+     (into [] (.get_topo_ids history))}))
+
 (defn cluster-summary
   ([user]
      (thrift/with-configured-nimbus-connection nimbus
@@ -613,7 +393,8 @@
       "nimbusLogLink" (nimbus-log-link host port)
       "status" "Offline"
       "version" "Not applicable"
-      "nimbusUpTime" "Not applicable"}
+      "nimbusUpTime" "Not applicable"
+      "nimbusUptimeSeconds" "Not applicable"}
     ))
 
 (defn nimbus-summary
@@ -628,14 +409,16 @@
           offline-nimbuses-summary (map #(convert-to-nimbus-summary %1) offline-nimbuses)]
       {"nimbuses"
        (concat offline-nimbuses-summary
-       (for [^NimbusSummary n nimbuses]
+       (for [^NimbusSummary n nimbuses
+             :let [uptime (.get_uptime_secs n)]]
          {
           "host" (.get_host n)
           "port" (.get_port n)
           "nimbusLogLink" (nimbus-log-link (.get_host n) (.get_port n))
           "status" (if (.is_isLeader n) "Leader" "Not a Leader")
           "version" (.get_version n)
-          "nimbusUpTime" (pretty-uptime-sec (.get_uptime_secs n))}))})))
+          "nimbusUpTime" (pretty-uptime-sec uptime)
+          "nimbusUpTimeSeconds" uptime}))})))
 
 (defn supervisor-summary
   ([]
@@ -648,9 +431,15 @@
       {"id" (.get_supervisor_id s)
        "host" (.get_host s)
        "uptime" (pretty-uptime-sec (.get_uptime_secs s))
+       "uptimeSeconds" (.get_uptime_secs s)
        "slotsTotal" (.get_num_workers s)
        "slotsUsed" (.get_num_used_workers s)
-       "version" (.get_version s)})}))
+       "totalMem" (get (.get_total_resources s) Config/SUPERVISOR_MEMORY_CAPACITY_MB)
+       "totalCpu" (get (.get_total_resources s) Config/SUPERVISOR_CPU_CAPACITY)
+       "usedMem" (.get_used_mem s)
+       "usedCpu" (.get_used_cpu s)
+       "version" (.get_version s)})
+    "schedulerDisplayResource" (*STORM-CONF* Config/SCHEDULER_DISPLAY_RESOURCE)}))
 
 (defn all-topologies-summary
   ([]
@@ -668,185 +457,186 @@
        "name" (.get_name t)
        "status" (.get_status t)
        "uptime" (pretty-uptime-sec (.get_uptime_secs t))
+       "uptimeSeconds" (.get_uptime_secs t)
        "tasksTotal" (.get_num_tasks t)
        "workersTotal" (.get_num_workers t)
        "executorsTotal" (.get_num_executors t)
        "replicationCount" (.get_replication_count t)
-       "schedulerInfo" (.get_sched_status t)})}))
+       "schedulerInfo" (.get_sched_status t)
+       "requestedMemOnHeap" (.get_requested_memonheap t)
+       "requestedMemOffHeap" (.get_requested_memoffheap t)
+       "requestedTotalMem" (+ (.get_requested_memonheap t) (.get_requested_memoffheap t))
+       "requestedCpu" (.get_requested_cpu t)
+       "assignedMemOnHeap" (.get_assigned_memonheap t)
+       "assignedMemOffHeap" (.get_assigned_memoffheap t)
+       "assignedTotalMem" (+ (.get_assigned_memonheap t) (.get_assigned_memoffheap t))
+       "assignedCpu" (.get_assigned_cpu t)})
+    "schedulerDisplayResource" (*STORM-CONF* Config/SCHEDULER_DISPLAY_RESOURCE)}))
 
-(defn topology-stats [id window stats]
+(defn topology-stats [window stats]
   (let [times (stats-times (:emitted stats))
         display-map (into {} (for [t times] [t pretty-uptime-sec]))
         display-map (assoc display-map ":all-time" (fn [_] "All time"))]
-    (for [k (concat times [":all-time"])
-          :let [disp ((display-map k) k)]]
+    (for [w (concat times [":all-time"])
+          :let [disp ((display-map w) w)]]
       {"windowPretty" disp
-       "window" k
-       "emitted" (get-in stats [:emitted k])
-       "transferred" (get-in stats [:transferred k])
-       "completeLatency" (float-str (get-in stats [:complete-latencies k]))
-       "acked" (get-in stats [:acked k])
-       "failed" (get-in stats [:failed k])})))
+       "window" w
+       "emitted" (get-in stats [:emitted w])
+       "transferred" (get-in stats [:transferred w])
+       "completeLatency" (float-str (get-in stats [:complete-latencies w]))
+       "acked" (get-in stats [:acked w])
+       "failed" (get-in stats [:failed w])})))
 
-(defn spout-comp [top-id summ-map errors window include-sys? secure?]
-  (for [[id summs] summ-map
-        :let [stats-seq (get-filled-stats summs)
-              stats (aggregate-spout-streams
-                     (aggregate-spout-stats
-                      stats-seq include-sys?))
-              last-error (most-recent-error (get errors id))
-              error-host (get-error-host last-error)
-              error-port (get-error-port last-error error-host top-id)]]
-    {"spoutId" id
-     "encodedSpoutId" (url-encode id)
-     "executors" (count summs)
-     "tasks" (sum-tasks summs)
-     "emitted" (get-in stats [:emitted window])
-     "transferred" (get-in stats [:transferred window])
-     "completeLatency" (float-str (get-in stats [:complete-latencies window]))
-     "acked" (get-in stats [:acked window])
-     "failed" (get-in stats [:failed window])
-     "errorHost" error-host
-     "errorPort" error-port
-     "errorWorkerLogLink" (worker-log-link error-host error-port top-id secure?)
-     "errorLapsedSecs" (get-error-time last-error)
-     "lastError" (get-error-data last-error)
-     "time" (if last-error (* 1000 (long (.get_error_time_secs ^ErrorInfo last-error))))}))
+(defn build-visualization [id window include-sys?]
+  (thrift/with-configured-nimbus-connection nimbus
+    (let [window (if window window ":all-time")
+          topology-info (->> (doto
+                               (GetInfoOptions.)
+                               (.set_num_err_choice NumErrorsChoice/ONE))
+                             (.getTopologyInfoWithOpts ^Nimbus$Client nimbus
+                                                       id))
+          storm-topology (.getTopology ^Nimbus$Client nimbus id)
+          spout-executor-summaries (filter (partial spout-summary? storm-topology) (.get_executors topology-info))
+          bolt-executor-summaries (filter (partial bolt-summary? storm-topology) (.get_executors topology-info))
+          spout-comp-id->executor-summaries (group-by-comp spout-executor-summaries)
+          bolt-comp-id->executor-summaries (group-by-comp bolt-executor-summaries)
+          bolt-comp-id->executor-summaries (filter-key (mk-include-sys-fn include-sys?) bolt-comp-id->executor-summaries)
+          id->spout-spec (.get_spouts storm-topology)
+          id->bolt (.get_bolts storm-topology)
+          visualizer-data (visualization-data (merge (hashmap-to-persistent id->spout-spec)
+                                                     (hashmap-to-persistent id->bolt))
+                                              spout-comp-id->executor-summaries
+                                              bolt-comp-id->executor-summaries
+                                              window
+                                              id)]
+       {"visualizationTable" (stream-boxes visualizer-data)})))
 
-(defn bolt-comp [top-id summ-map errors window include-sys? secure?]
-  (for [[id summs] summ-map
-        :let [stats-seq (get-filled-stats summs)
-              stats (aggregate-bolt-streams
-                     (aggregate-bolt-stats
-                      stats-seq include-sys?))
-              last-error (most-recent-error (get errors id))
-              error-host (get-error-host last-error)
-              error-port (get-error-port last-error error-host top-id)]]
-    {"boltId" id
-     "encodedBoltId" (url-encode id)
-     "executors" (count summs)
-     "tasks" (sum-tasks summs)
-     "emitted" (get-in stats [:emitted window])
-     "transferred" (get-in stats [:transferred window])
-     "capacity" (float-str (nil-to-zero (compute-bolt-capacity summs)))
-     "executeLatency" (float-str (get-in stats [:execute-latencies window]))
-     "executed" (get-in stats [:executed window])
-     "processLatency" (float-str (get-in stats [:process-latencies window]))
-     "acked" (get-in stats [:acked window])
-     "failed" (get-in stats [:failed window])
-     "errorHost" error-host
-     "errorPort" error-port
-     "errorWorkerLogLink" (worker-log-link error-host error-port top-id secure?)
-     "errorLapsedSecs" (get-error-time last-error)
-     "lastError" (get-error-data last-error)
-     "time" (if last-error (* 1000 (long (.get_error_time_secs ^ErrorInfo last-error))))}))
+(defn- get-error-json
+  [topo-id error-info secure?]
+  (let [host (get-error-host error-info)
+        port (get-error-port error-info)]
+    {"lastError" (get-error-data error-info)
+     "errorTime" (get-error-time error-info)
+     "errorHost" host
+     "errorPort" port
+     "errorLapsedSecs" (get-error-time error-info)
+     "errorWorkerLogLink" (worker-log-link host port topo-id secure?)}))
 
-(defn topology-summary [^TopologyInfo summ]
-  (let [executors (.get_executors summ)
-        workers (set (for [^ExecutorSummary e executors]
-                       [(.get_host e) (.get_port e)]))
-        topology-id (.get_id summ)
-        debug-options (get (.get_component_debug summ) topology-id)]
-      {"id" topology-id
-       "encodedId" (url-encode (.get_id summ))
-       "owner" (.get_owner summ)
-       "name" (.get_name summ)
-       "status" (.get_status summ)
-       "uptime" (pretty-uptime-sec (.get_uptime_secs summ))
-       "tasksTotal" (sum-tasks executors)
-       "workersTotal" (count workers)
-       "executorsTotal" (count executors)
-       "schedulerInfo" (.get_sched_status summ)
-       "debug" (if (not-nil? debug-options) (.is_enable debug-options) false)
-       "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 10)
-       "replicationCount" (.get_replication_count summ)}))
+(defn- common-agg-stats-json
+  "Returns a JSON representation of a common aggregated statistics."
+  [^CommonAggregateStats common-stats]
+  {"executors" (.get_num_executors common-stats)
+   "tasks" (.get_num_tasks common-stats)
+   "emitted" (.get_emitted common-stats)
+   "transferred" (.get_transferred common-stats)
+   "acked" (.get_acked common-stats)
+   "failed" (.get_failed common-stats)})
 
-(defn spout-summary-json [topology-id id stats window]
-  (let [times (stats-times (:emitted stats))
-        display-map (into {} (for [t times] [t pretty-uptime-sec]))
-        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
-     (for [k (concat times [":all-time"])
-           :let [disp ((display-map k) k)]]
-       {"windowPretty" disp
-        "window" k
-        "emitted" (get-in stats [:emitted k])
-        "transferred" (get-in stats [:transferred k])
-        "completeLatency" (float-str (get-in stats [:complete-latencies k]))
-        "acked" (get-in stats [:acked k])
-        "failed" (get-in stats [:failed k])})))
+(defmulti comp-agg-stats-json
+  "Returns a JSON representation of aggregated statistics."
+  (fn [_ _ [id ^ComponentAggregateStats s]] (.get_type s)))
+
+(defmethod comp-agg-stats-json ComponentType/SPOUT
+  [topo-id secure? [id ^ComponentAggregateStats s]]
+  (let [^SpoutAggregateStats ss (.. s get_specific_stats get_spout)
+        cs (.get_common_stats s)]
+    (merge
+      (common-agg-stats-json cs)
+      (get-error-json topo-id (.get_last_error s) secure?)
+      {"spoutId" id
+       "encodedSpoutId" (url-encode id)
+       "completeLatency" (float-str (.get_complete_latency_ms ss))})))
+
+(defmethod comp-agg-stats-json ComponentType/BOLT
+  [topo-id secure? [id ^ComponentAggregateStats s]]
+  (let [^BoltAggregateStats ss (.. s get_specific_stats get_bolt)
+        cs (.get_common_stats s)]
+    (merge
+      (common-agg-stats-json cs)
+      (get-error-json topo-id (.get_last_error s) secure?)
+      {"boltId" id
+       "encodedBoltId" (url-encode id)
+       "capacity" (float-str (.get_capacity ss))
+       "executeLatency" (float-str (.get_execute_latency_ms ss))
+       "executed" (.get_executed ss)
+       "processLatency" (float-str (.get_process_latency_ms ss))})))
+
+(defn- unpack-topology-page-info
+  "Unpacks the serialized object to data structures"
+  [^TopologyPageInfo topo-info window secure?]
+  (let [id (.get_id topo-info)
+        ^TopologyStats topo-stats (.get_topology_stats topo-info)
+        stat->window->number
+          {:emitted (.get_window_to_emitted topo-stats)
+           :transferred (.get_window_to_transferred topo-stats)
+           :complete-latencies (.get_window_to_complete_latencies_ms topo-stats)
+           :acked (.get_window_to_acked topo-stats)
+           :failed (.get_window_to_failed topo-stats)}
+        topo-stats (topology-stats window stat->window->number)
+        [debugEnabled
+         samplingPct] (if-let [debug-opts (.get_debug_options topo-info)]
+                        [(.is_enable debug-opts)
+                         (.get_samplingpct debug-opts)])
+        uptime (.get_uptime_secs topo-info)]
+    {"id" id
+     "encodedId" (url-encode id)
+     "owner" (.get_owner topo-info)
+     "name" (.get_name topo-info)
+     "status" (.get_status topo-info)
+     "uptime" (pretty-uptime-sec uptime)
+     "uptimeSeconds" uptime
+     "tasksTotal" (.get_num_tasks topo-info)
+     "workersTotal" (.get_num_workers topo-info)
+     "executorsTotal" (.get_num_executors topo-info)
+     "schedulerInfo" (.get_sched_status topo-info)
+     "requestedMemOnHeap" (.get_requested_memonheap topo-info)
+     "requestedMemOffHeap" (.get_requested_memoffheap topo-info)
+     "requestedCpu" (.get_requested_cpu topo-info)
+     "assignedMemOnHeap" (.get_assigned_memonheap topo-info)
+     "assignedMemOffHeap" (.get_assigned_memoffheap topo-info)
+     "assignedTotalMem" (+ (.get_assigned_memonheap topo-info) (.get_assigned_memoffheap topo-info))
+     "assignedCpu" (.get_assigned_cpu topo-info)
+     "topologyStats" topo-stats
+     "spouts" (map (partial comp-agg-stats-json id secure?)
+                   (.get_id_to_spout_agg_stats topo-info))
+     "bolts" (map (partial comp-agg-stats-json id secure?)
+                  (.get_id_to_bolt_agg_stats topo-info))
+     "configuration" (.get_topology_conf topo-info)
+     "debug" (or debugEnabled false)
+     "samplingPct" (or samplingPct 10)
+     "replicationCount" (.get_replication_count topo-info)}))
+
+(defn exec-host-port
+  [executors]
+  (for [^ExecutorSummary e executors]
+    {"host" (.get_host e)
+     "port" (.get_port e)}))
+
+(defn worker-host-port
+  "Get the set of all worker host/ports"
+  [id]
+  (thrift/with-configured-nimbus-connection nimbus
+    (distinct (exec-host-port (.get_executors (get-topology-info nimbus id))))))
 
 (defn topology-page [id window include-sys? user secure?]
   (thrift/with-configured-nimbus-connection nimbus
     (let [window (if window window ":all-time")
           window-hint (window-hint window)
-          summ (->> (doto
-                      (GetInfoOptions.)
-                      (.set_num_err_choice NumErrorsChoice/ONE))
-                    (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
-          topology (.getTopology ^Nimbus$Client nimbus id)
-          topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus id))
-          spout-summs (filter (partial spout-summary? topology) (.get_executors summ))
-          bolt-summs (filter (partial bolt-summary? topology) (.get_executors summ))
-          spout-comp-summs (group-by-comp spout-summs)
-          bolt-comp-summs (group-by-comp bolt-summs)
-          bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?) bolt-comp-summs)
-          name (.get_name summ)
-          status (.get_status summ)
-          msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)
-          spouts (.get_spouts topology)
-          bolts (.get_bolts topology)
-          replication-count (.get_replication_count summ)
-          visualizer-data (visualization-data (merge (hashmap-to-persistent spouts)
-                                                     (hashmap-to-persistent bolts))
-                                              spout-comp-summs
-                                              bolt-comp-summs
-                                              window
-                                              id)]
+          topo-page-info (.getTopologyPageInfo ^Nimbus$Client nimbus
+                                               id
+                                               window
+                                               include-sys?)
+          topology-conf (from-json (.get_topology_conf topo-page-info))
+          msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)]
       (merge
-       (topology-summary summ)
+       (unpack-topology-page-info topo-page-info window secure?)
        {"user" user
         "window" window
         "windowHint" window-hint
         "msgTimeout" msg-timeout
-        "topologyStats" (topology-stats id window (total-aggregate-stats spout-summs bolt-summs include-sys?))
-        "spouts" (spout-comp id spout-comp-summs (.get_errors summ) window include-sys? secure?)
-        "bolts" (bolt-comp id bolt-comp-summs (.get_errors summ) window include-sys? secure?)
         "configuration" topology-conf
-        "visualizationTable" (stream-boxes visualizer-data)
-        "replicationCount" replication-count}))))
-
-(defn spout-output-stats
-  [stream-summary window]
-  (let [stream-summary (map-val swap-map-order (swap-map-order stream-summary))]
-    (for [[s stats] (stream-summary window)]
-      {"stream" s
-       "emitted" (nil-to-zero (:emitted stats))
-       "transferred" (nil-to-zero (:transferred stats))
-       "completeLatency" (float-str (:complete-latencies stats))
-       "acked" (nil-to-zero (:acked stats))
-       "failed" (nil-to-zero (:failed stats))})))
-
-(defn spout-executor-stats
-  [topology-id executors window include-sys? secure?]
-  (for [^ExecutorSummary e executors
-        :let [stats (.get_stats e)
-              stats (if stats
-                      (-> stats
-                          (aggregate-spout-stats include-sys?)
-                          aggregate-spout-streams
-                          swap-map-order
-                          (get window)))]]
-    {"id" (pretty-executor-info (.get_executor_info e))
-     "encodedId" (url-encode (pretty-executor-info (.get_executor_info e)))
-     "uptime" (pretty-uptime-sec (.get_uptime_secs e))
-     "host" (.get_host e)
-     "port" (.get_port e)
-     "emitted" (nil-to-zero (:emitted stats))
-     "transferred" (nil-to-zero (:transferred stats))
-     "completeLatency" (float-str (:complete-latencies stats))
-     "acked" (nil-to-zero (:acked stats))
-     "failed" (nil-to-zero (:failed stats))
-     "workerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id secure?)}))
+        "visualizationTable" []
+        "schedulerDisplayResource" (*STORM-CONF* Config/SCHEDULER_DISPLAY_RESOURCE)}))))
 
 (defn component-errors
   [errors-list topology-id secure?]
@@ -858,142 +648,248 @@
        {"time" (* 1000 (long (.get_error_time_secs e)))
         "errorHost" (.get_host e)
         "errorPort"  (.get_port e)
-        "errorWorkerLogLink"  (worker-log-link (.get_host e) (.get_port e) topology-id secure?)
+        "errorWorkerLogLink"  (worker-log-link (.get_host e)
+                                               (.get_port e)
+                                               topology-id
+                                               secure?)
         "errorLapsedSecs" (get-error-time e)
         "error" (.get_error e)})}))
 
-(defn spout-stats
-  [window ^TopologyInfo topology-info component executors include-sys? secure?]
-  (let [window-hint (str " (" (window-hint window) ")")
-        stats (get-filled-stats executors)
-        stream-summary (-> stats (aggregate-spout-stats include-sys?))
-        summary (-> stream-summary aggregate-spout-streams)]
-    {"spoutSummary" (spout-summary-json
-                      (.get_id topology-info) component summary window)
-     "outputStats" (spout-output-stats stream-summary window)
-     "executorStats" (spout-executor-stats (.get_id topology-info)
-                                           executors window include-sys? secure?)}))
+(defmulti unpack-comp-agg-stat
+  (fn [[_ ^ComponentAggregateStats s]] (.get_type s)))
 
-(defn bolt-summary
-  [topology-id id stats window]
-  (let [times (stats-times (:emitted stats))
-        display-map (into {} (for [t times] [t pretty-uptime-sec]))
-        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
-    (for [k (concat times [":all-time"])
-          :let [disp ((display-map k) k)]]
-      {"window" k
-       "windowPretty" disp
-       "emitted" (get-in stats [:emitted k])
-       "transferred" (get-in stats [:transferred k])
-       "executeLatency" (float-str (get-in stats [:execute-latencies k]))
-       "executed" (get-in stats [:executed k])
-       "processLatency" (float-str (get-in stats [:process-latencies k]))
-       "acked" (get-in stats [:acked k])
-       "failed" (get-in stats [:failed k])})))
+(defmethod unpack-comp-agg-stat ComponentType/BOLT
+  [[window ^ComponentAggregateStats s]]
+  (let [^CommonAggregateStats comm-s (.get_common_stats s)
+        ^SpecificAggregateStats spec-s (.get_specific_stats s)
+        ^BoltAggregateStats bolt-s (.get_bolt spec-s)]
+    {"window" window
+     "windowPretty" (window-hint window)
+     "emitted" (.get_emitted comm-s)
+     "transferred" (.get_transferred comm-s)
+     "acked" (.get_acked comm-s)
+     "failed" (.get_failed comm-s)
+     "executeLatency" (float-str (.get_execute_latency_ms bolt-s))
+     "processLatency"  (float-str (.get_process_latency_ms bolt-s))
+     "executed" (.get_executed bolt-s)
+     "capacity" (float-str (.get_capacity bolt-s))}))
 
-(defn bolt-output-stats
-  [stream-summary window]
-  (let [stream-summary (-> stream-summary
-                           swap-map-order
-                           (get window)
-                           (select-keys [:emitted :transferred])
-                           swap-map-order)]
-    (for [[s stats] stream-summary]
-      {"stream" s
-        "emitted" (nil-to-zero (:emitted stats))
-        "transferred" (nil-to-zero (:transferred stats))})))
+(defmethod unpack-comp-agg-stat ComponentType/SPOUT
+  [[window ^ComponentAggregateStats s]]
+  (let [^CommonAggregateStats comm-s (.get_common_stats s)
+        ^SpecificAggregateStats spec-s (.get_specific_stats s)
+        ^SpoutAggregateStats spout-s (.get_spout spec-s)]
+    {"window" window
+     "windowPretty" (window-hint window)
+     "emitted" (.get_emitted comm-s)
+     "transferred" (.get_transferred comm-s)
+     "acked" (.get_acked comm-s)
+     "failed" (.get_failed comm-s)
+     "completeLatency" (float-str (.get_complete_latency_ms spout-s))}))
 
-(defn bolt-input-stats
-  [stream-summary window]
-  (let [stream-summary
-        (-> stream-summary
-            swap-map-order
-            (get window)
-            (select-keys [:acked :failed :process-latencies
-                          :executed :execute-latencies])
-            swap-map-order)]
-    (for [[^GlobalStreamId s stats] stream-summary]
-      {"component" (.get_componentId s)
-       "encodedComponent" (url-encode (.get_componentId s))
-       "stream" (.get_streamId s)
-       "executeLatency" (float-str (:execute-latencies stats))
-       "processLatency" (float-str (:process-latencies stats))
-       "executed" (nil-to-zero (:executed stats))
-       "acked" (nil-to-zero (:acked stats))
-       "failed" (nil-to-zero (:failed stats))})))
+(defn- unpack-bolt-input-stat
+  [[^GlobalStreamId s ^ComponentAggregateStats stats]]
+  (let [^SpecificAggregateStats sas (.get_specific_stats stats)
+        ^BoltAggregateStats bas (.get_bolt sas)
+        ^CommonAggregateStats cas (.get_common_stats stats)
+        comp-id (.get_componentId s)]
+    {"component" comp-id
+     "encodedComponentId" (url-encode comp-id)
+     "stream" (.get_streamId s)
+     "executeLatency" (float-str (.get_execute_latency_ms bas))
+     "processLatency" (float-str (.get_process_latency_ms bas))
+     "executed" (nil-to-zero (.get_executed bas))
+     "acked" (nil-to-zero (.get_acked cas))
+     "failed" (nil-to-zero (.get_failed cas))}))
 
-(defn bolt-executor-stats
-  [topology-id executors window include-sys? secure?]
-  (for [^ExecutorSummary e executors
-        :let [stats (.get_stats e)
-              stats (if stats
-                      (-> stats
-                          (aggregate-bolt-stats include-sys?)
-                          (aggregate-bolt-streams)
-                          swap-map-order
-                          (get window)))]]
-    {"id" (pretty-executor-info (.get_executor_info e))
-     "encodedId" (url-encode (pretty-executor-info (.get_executor_info e)))
-     "uptime" (pretty-uptime-sec (.get_uptime_secs e))
-     "host" (.get_host e)
-     "port" (.get_port e)
-     "emitted" (nil-to-zero (:emitted stats))
-     "transferred" (nil-to-zero (:transferred stats))
-     "capacity" (float-str (nil-to-zero (compute-executor-capacity e)))
-     "executeLatency" (float-str (:execute-latencies stats))
-     "executed" (nil-to-zero (:executed stats))
-     "processLatency" (float-str (:process-latencies stats))
-     "acked" (nil-to-zero (:acked stats))
-     "failed" (nil-to-zero (:failed stats))
-     "workerLogLink" (worker-log-link (.get_host e) (.get_port e) topology-id secure?)}))
+(defmulti unpack-comp-output-stat
+  (fn [[_ ^ComponentAggregateStats s]] (.get_type s)))
 
-(defn bolt-stats
-  [window ^TopologyInfo topology-info component executors include-sys? secure?]
-  (let [window-hint (str " (" (window-hint window) ")")
-        stats (get-filled-stats executors)
-        stream-summary (-> stats (aggregate-bolt-stats include-sys?))
-        summary (-> stream-summary aggregate-bolt-streams)]
-    {"boltStats" (bolt-summary (.get_id topology-info) component summary window)
-     "inputStats" (bolt-input-stats stream-summary window)
-     "outputStats" (bolt-output-stats stream-summary window)
-     "executorStats" (bolt-executor-stats
-                       (.get_id topology-info) executors window include-sys? secure?)}))
+(defmethod unpack-comp-output-stat ComponentType/BOLT
+  [[stream-id ^ComponentAggregateStats stats]]
+  (let [^CommonAggregateStats cas (.get_common_stats stats)]
+    {"stream" stream-id
+     "emitted" (nil-to-zero (.get_emitted cas))
+     "transferred" (nil-to-zero (.get_transferred cas))}))
+
+(defmethod unpack-comp-output-stat ComponentType/SPOUT
+  [[stream-id ^ComponentAggregateStats stats]]
+  (let [^CommonAggregateStats cas (.get_common_stats stats)
+        ^SpecificAggregateStats spec-s (.get_specific_stats stats)
+        ^SpoutAggregateStats spout-s (.get_spout spec-s)]
+    {"stream" stream-id
+     "emitted" (nil-to-zero (.get_emitted cas))
+     "transferred" (nil-to-zero (.get_transferred cas))
+     "completeLatency" (float-str (.get_complete_latency_ms spout-s))
+     "acked" (nil-to-zero (.get_acked cas))
+     "failed" (nil-to-zero (.get_failed cas))}))
+
+(defmulti unpack-comp-exec-stat
+  (fn [_ _ ^ComponentAggregateStats cas] (.get_type (.get_stats ^ExecutorAggregateStats cas))))
+
+(defmethod unpack-comp-exec-stat ComponentType/BOLT
+  [topology-id secure? ^ExecutorAggregateStats eas]
+  (let [^ExecutorSummary summ (.get_exec_summary eas)
+        ^ExecutorInfo info (.get_executor_info summ)
+        ^ComponentAggregateStats stats (.get_stats eas)
+        ^SpecificAggregateStats ss (.get_specific_stats stats)
+        ^BoltAggregateStats bas (.get_bolt ss)
+        ^CommonAggregateStats cas (.get_common_stats stats)
+        host (.get_host summ)
+        port (.get_port summ)
+        exec-id (pretty-executor-info info)
+        uptime (.get_uptime_secs summ)]
+    {"id" exec-id
+     "encodedId" (url-encode exec-id)
+     "uptime" (pretty-uptime-sec uptime)
+     "uptimeSeconds" uptime
+     "host" host
+     "port" port
+     "emitted" (nil-to-zero (.get_emitted cas))
+     "transferred" (nil-to-zero (.get_transferred cas))
+     "capacity" (float-str (nil-to-zero (.get_capacity bas)))
+     "executeLatency" (float-str (.get_execute_latency_ms bas))
+     "executed" (nil-to-zero (.get_executed bas))
+     "processLatency" (float-str (.get_process_latency_ms bas))
+     "acked" (nil-to-zero (.get_acked cas))
+     "failed" (nil-to-zero (.get_failed cas))
+     "workerLogLink" (worker-log-link host port topology-id secure?)}))
+
+(defmethod unpack-comp-exec-stat ComponentType/SPOUT
+  [topology-id secure? ^ExecutorAggregateStats eas]
+  (let [^ExecutorSummary summ (.get_exec_summary eas)
+        ^ExecutorInfo info (.get_executor_info summ)
+        ^ComponentAggregateStats stats (.get_stats eas)
+        ^SpecificAggregateStats ss (.get_specific_stats stats)
+        ^SpoutAggregateStats sas (.get_spout ss)
+        ^CommonAggregateStats cas (.get_common_stats stats)
+        host (.get_host summ)
+        port (.get_port summ)
+        exec-id (pretty-executor-info info)
+        uptime (.get_uptime_secs summ)]
+    {"id" exec-id
+     "encodedId" (url-encode exec-id)
+     "uptime" (pretty-uptime-sec uptime)
+     "uptimeSeconds" uptime
+     "host" host
+     "port" port
+     "emitted" (nil-to-zero (.get_emitted cas))
+     "transferred" (nil-to-zero (.get_transferred cas))
+     "completeLatency" (float-str (.get_complete_latency_ms sas))
+     "acked" (nil-to-zero (.get_acked cas))
+     "failed" (nil-to-zero (.get_failed cas))
+     "workerLogLink" (worker-log-link host port topology-id secure?)}))
+
+(defmulti unpack-component-page-info
+  "Unpacks component-specific info to clojure data structures"
+  (fn [^ComponentPageInfo info & _]
+    (.get_component_type info)))
+
+(defmethod unpack-component-page-info ComponentType/BOLT
+  [^ComponentPageInfo info topology-id window include-sys? secure?]
+  (merge
+    {"boltStats" (map unpack-comp-agg-stat (.get_window_to_stats info))
+     "inputStats" (map unpack-bolt-input-stat (.get_gsid_to_input_stats info))
+     "outputStats" (map unpack-comp-output-stat (.get_sid_to_output_stats info))
+     "executorStats" (map (partial unpack-comp-exec-stat topology-id secure?)
+                          (.get_exec_stats info))}
+    (-> info .get_errors (component-errors topology-id secure?))))
+
+(defmethod unpack-component-page-info ComponentType/SPOUT
+  [^ComponentPageInfo info topology-id window include-sys? secure?]
+  (merge
+    {"spoutSummary" (map unpack-comp-agg-stat (.get_window_to_stats info))
+     "outputStats" (map unpack-comp-output-stat (.get_sid_to_output_stats info))
+     "executorStats" (map (partial unpack-comp-exec-stat topology-id secure?)
+                          (.get_exec_stats info))}
+    (-> info .get_errors (component-errors topology-id secure?))))
+
+(defn get-active-profile-actions
+  [nimbus topology-id component]
+  (let [profile-actions  (.getComponentPendingProfileActions nimbus
+                                               topology-id
+                                               component
+                                 ProfileAction/JPROFILE_STOP)
+        latest-profile-actions (map clojurify-profile-request profile-actions)
+        active-actions (map (fn [profile-action]
+                              {"host" (:host profile-action)
+                               "port" (str (:port profile-action))
+                               "dumplink" (worker-dump-link (:host profile-action) (str (:port profile-action)) topology-id)
+                               "timestamp" (str (- (:timestamp profile-action) (System/currentTimeMillis)))})
+                            latest-profile-actions)]
+    (log-message "Latest-active actions are: " (pr active-actions))
+    active-actions))
 
 (defn component-page
   [topology-id component window include-sys? user secure?]
   (thrift/with-configured-nimbus-connection nimbus
-    (let [window (if window window ":all-time")
-          summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
-          topology (.getTopology ^Nimbus$Client nimbus topology-id)
-          topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))
-          type (component-type topology component)
-          summs (component-task-summs summ topology component)
-          spec (cond (= type :spout) (spout-stats window summ component summs include-sys? secure?)
-                     (= type :bolt) (bolt-stats window summ component summs include-sys? secure?))
-          errors (component-errors (get (.get_errors summ) component) topology-id secure?)
-          component->debug (.get_component_debug summ)
-          debug-options (get component->debug component (get component->debug topology-id))]
-      (merge
-        {"user" user
-         "id" component
-         "encodedId" (url-encode component)
-         "name" (.get_name summ)
-         "executors" (count summs)
-         "tasks" (sum-tasks summs)
-         "topologyId" topology-id
-         "topologyStatus" (.get_status summ)
-         "encodedTopologyId" (url-encode topology-id)
-         "window" window
-         "componentType" (name type)
-         "windowHint" (window-hint window)
-         "debug" (if (not-nil? debug-options) (.is_enable debug-options) false)
-         "samplingPct" (if (not-nil? debug-options) (.get_samplingpct debug-options) 10)
-         "eventLogLink" (event-log-link topology-id summ topology component secure?)}
-       spec errors))))
+    (let [window (or window ":all-time")
+          window-hint (window-hint window)
+          comp-page-info (.getComponentPageInfo ^Nimbus$Client nimbus
+                                                topology-id
+                                                component
+                                                window
+                                                include-sys?)
+          topology-conf (from-json (.getTopologyConf ^Nimbus$Client nimbus
+                                                     topology-id))
+          msg-timeout (topology-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)
+          [debugEnabled
+           samplingPct] (if-let [debug-opts (.get_debug_options comp-page-info)]
+                          [(.is_enable debug-opts)
+                           (.get_samplingpct debug-opts)])]
+      (assoc
+       (unpack-component-page-info comp-page-info
+                                   topology-id
+                                   window
+                                   include-sys?
+                                   secure?)
+       "user" user
+       "id" component
+       "encodedId" (url-encode component)
+       "name" (.get_topology_name comp-page-info)
+       "executors" (.get_num_executors comp-page-info)
+       "tasks" (.get_num_tasks comp-page-info)
+       "topologyId" topology-id
+       "topologyStatus" (.get_topology_status comp-page-info)
+       "encodedTopologyId" (url-encode topology-id)
+       "window" window
+       "componentType" (-> comp-page-info .get_component_type str lower-case)
+       "windowHint" window-hint
+       "debug" (or debugEnabled false)
+       "samplingPct" (or samplingPct 10)
+       "eventLogLink" (event-log-link topology-id
+                                      component
+                                      (.get_eventlog_host comp-page-info)
+                                      (.get_eventlog_port comp-page-info)
+                                      secure?)
+       "profileActionEnabled" (*STORM-CONF* WORKER-PROFILER-ENABLED)
+       "profilerActive" (if (*STORM-CONF* WORKER-PROFILER-ENABLED)
+                          (get-active-profile-actions nimbus topology-id component)
+                          [])))))
+    
+(defn- level-to-dict [level]
+  (if level
+    (let [timeout (.get_reset_log_level_timeout_secs level)
+          timeout-epoch (.get_reset_log_level_timeout_epoch level)
+          target-level (.get_target_log_level level)
+          reset-level (.get_reset_log_level level)]
+          {"target_level" (.toString (Level/toLevel target-level))
+           "reset_level" (.toString (Level/toLevel reset-level))
+           "timeout" timeout
+           "timeout_epoch" timeout-epoch})))
+
+(defn log-config [topology-id]
+  (thrift/with-configured-nimbus-connection
+    nimbus
+    (let [log-config (.getLogConfig ^Nimbus$Client nimbus topology-id)
+          named-logger-levels (into {}
+                                (for [[key val] (.get_named_logger_level log-config)]
+                                  [(str key) (level-to-dict val)]))]
+      {"namedLoggerLevels" named-logger-levels})))
 
 (defn topology-config [topology-id]
   (thrift/with-configured-nimbus-connection nimbus
-     (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))))
+    (from-json (.getTopologyConf ^Nimbus$Client nimbus topology-id))))
 
 (defn topology-op-response [topology-id op]
   {"topologyOperation" op,
@@ -1012,57 +908,89 @@
   [sys?]
   (if (or (nil? sys?) (= "false" sys?)) false true))
 
-(defn wrap-json-in-callback [callback response]
-  (str callback "(" response ");"))
-
-(defnk json-response
-  [data callback :serialize-fn to-json :status 200]
-     {:status status
-      :headers (merge {"Cache-Control" "no-cache, no-store"
-                       "Access-Control-Allow-Origin" "*"
-                       "Access-Control-Allow-Headers" "Content-Type, Access-Control-Allow-Headers, Access-Controler-Allow-Origin, X-Requested-By, Authorization, X-Requested-With"}
-                      (if (not-nil? callback) {"Content-Type" "application/javascript;charset=utf-8"}
-                          {"Content-Type" "application/json;charset=utf-8"}))
-      :body (if (not-nil? callback)
-              (wrap-json-in-callback callback (serialize-fn data))
-              (serialize-fn data))})
-
 (def http-creds-handler (AuthUtils/GetUiHttpCredentialsPlugin *STORM-CONF*))
 
+(defn populate-context!
+  "Populate the Storm RequestContext from an servlet-request. This should be called in each handler"
+  [servlet-request]
+    (when http-creds-handler
+      (.populateContext http-creds-handler (ReqContext/context) servlet-request)))
+
+(defn get-user-name
+  [servlet-request]
+  (.getUserName http-creds-handler servlet-request))
+
 (defroutes main-routes
   (GET "/api/v1/cluster/configuration" [& m]
-       (json-response (cluster-configuration)
-                      (:callback m) :serialize-fn identity))
+    (mark! ui:num-cluster-configuration-http-requests)
+    (json-response (cluster-configuration)
+                   (:callback m) :serialize-fn identity))
   (GET "/api/v1/cluster/summary" [:as {:keys [cookies servlet-request]} & m]
-       (let [user (.getUserName http-creds-handler servlet-request)]
-         (assert-authorized-user servlet-request "getClusterInfo")
-         (json-response (cluster-summary user) (:callback m))))
+    (mark! ui:num-cluster-summary-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getClusterInfo")
+    (let [user (get-user-name servlet-request)]
+      (json-response (assoc (cluster-summary user)
+                          "bugtracker-url" (*STORM-CONF* UI-PROJECT-BUGTRACKER-URL)
+                          "central-log-url" (*STORM-CONF* UI-CENTRAL-LOGGING-URL)) (:callback m))))
   (GET "/api/v1/nimbus/summary" [:as {:keys [cookies servlet-request]} & m]
-    (assert-authorized-user servlet-request "getClusterInfo")
+    (mark! ui:num-nimbus-summary-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getClusterInfo")
     (json-response (nimbus-summary) (:callback m)))
+  (GET "/api/v1/history/summary" [:as {:keys [cookies servlet-request]} & m]
+    (let [user (.getUserName http-creds-handler servlet-request)]
+      (json-response (topology-history-info user) (:callback m))))
   (GET "/api/v1/supervisor/summary" [:as {:keys [cookies servlet-request]} & m]
-       (assert-authorized-user servlet-request "getClusterInfo")
-       (json-response (supervisor-summary) (:callback m)))
+    (mark! ui:num-supervisor-summary-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getClusterInfo")
+    (json-response (assoc (supervisor-summary)
+                     "logviewerPort" (*STORM-CONF* LOGVIEWER-PORT)) (:callback m)))
   (GET "/api/v1/topology/summary" [:as {:keys [cookies servlet-request]} & m]
-       (assert-authorized-user servlet-request "getClusterInfo")
-       (json-response (all-topologies-summary) (:callback m)))
-  (GET  "/api/v1/topology/:id" [:as {:keys [cookies servlet-request schema]} id & m]
-        (let [user (.getUserName http-creds-handler servlet-request)]
-          (assert-authorized-user servlet-request "getTopology" (topology-config id))
-          (json-response (topology-page id (:window m) (check-include-sys? (:sys m)) user (= schema :https)) (:callback m))))
+    (mark! ui:num-all-topologies-summary-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getClusterInfo")
+    (json-response (all-topologies-summary) (:callback m)))
+  (GET  "/api/v1/topology-workers/:id" [:as {:keys [cookies servlet-request]} id & m]
+    (let [id (url-decode id)]
+      (json-response {"hostPortList" (worker-host-port id)
+                      "logviewerPort" (*STORM-CONF* LOGVIEWER-PORT)} (:callback m))))
+  (GET "/api/v1/topology/:id" [:as {:keys [cookies servlet-request scheme]} id & m]
+    (mark! ui:num-topology-page-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getTopology" (topology-config id))
+    (let [user (get-user-name servlet-request)]
+      (json-response (topology-page id (:window m) (check-include-sys? (:sys m)) user (= scheme :https)) (:callback m))))
+  (GET "/api/v1/topology/:id/visualization-init" [:as {:keys [cookies servlet-request]} id & m]
+    (mark! ui:num-build-visualization-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getTopology" (topology-config id))
+    (json-response (build-visualization id (:window m) (check-include-sys? (:sys m))) (:callback m)))
   (GET "/api/v1/topology/:id/visualization" [:as {:keys [cookies servlet-request]} id & m]
-        (assert-authorized-user servlet-request "getTopology" (topology-config id))
-        (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m))) (:callback m)))
+    (mark! ui:num-mk-visualization-data-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getTopology" (topology-config id))
+    (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m))) (:callback m)))
   (GET "/api/v1/topology/:id/component/:component" [:as {:keys [cookies servlet-request scheme]} id component & m]
-       (let [user (.getUserName http-creds-handler servlet-request)]
-         (assert-authorized-user servlet-request "getTopology" (topology-config id))
-         (json-response
+    (mark! ui:num-component-page-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getTopology" (topology-config id))
+    (let [user (get-user-name servlet-request)]
+      (json-response
           (component-page id component (:window m) (check-include-sys? (:sys m)) user (= scheme :https))
           (:callback m))))
+  (GET "/api/v1/topology/:id/logconfig" [:as {:keys [cookies servlet-request]} id & m]
+    (mark! ui:num-log-config-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "getTopology" (topology-config id))
+       (json-response (log-config id) (:callback m)))
   (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id & m]
+    (mark! ui:num-activate-topology-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "activate" (topology-config id))
     (thrift/with-configured-nimbus-connection nimbus
-    (assert-authorized-user servlet-request "activate" (topology-config id))
-      (let [tplg (->> (doto
+       (let [tplg (->> (doto
                         (GetInfoOptions.)
                         (.set_num_err_choice NumErrorsChoice/NONE))
                       (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
@@ -1071,9 +999,11 @@
         (log-message "Activating topology '" name "'")))
     (json-response (topology-op-response id "activate") (m "callback")))
   (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id & m]
+    (mark! ui:num-deactivate-topology-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "deactivate" (topology-config id))
     (thrift/with-configured-nimbus-connection nimbus
-    (assert-authorized-user servlet-request "deactivate" (topology-config id))
-      (let [tplg (->> (doto
+        (let [tplg (->> (doto
                         (GetInfoOptions.)
                         (.set_num_err_choice NumErrorsChoice/NONE))
                       (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
@@ -1082,9 +1012,11 @@
         (log-message "Deactivating topology '" name "'")))
     (json-response (topology-op-response id "deactivate") (m "callback")))
   (POST "/api/v1/topology/:id/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id action spct & m]
-    (assert-authorized-user servlet-request "debug" (topology-config id))
+    (mark! ui:num-debug-topology-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "debug" (topology-config id))
     (thrift/with-configured-nimbus-connection nimbus
-      (let [tplg (->> (doto
+        (let [tplg (->> (doto
                         (GetInfoOptions.)
                         (.set_num_err_choice NumErrorsChoice/NONE))
                    (.getTopologyInfoWithOpts ^Nimbus$Client nimbus id))
@@ -1092,9 +1024,11 @@
             enable? (= "enable" action)]
         (.debug nimbus name "" enable? (Integer/parseInt spct))
         (log-message "Debug topology [" name "] action [" action "] sampling pct [" spct "]")))
-    (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
+     (json-response (topology-op-response id (str "debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/component/:component/debug/:action/:spct" [:as {:keys [cookies servlet-request]} id component action spct & m]
-    (assert-authorized-user servlet-request "debug" (topology-config id))
+    (mark! ui:num-component-op-response-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "debug" (topology-config id))
     (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (->> (doto
                         (GetInfoOptions.)
@@ -1106,8 +1040,10 @@
         (log-message "Debug topology [" name "] component [" component "] action [" action "] sampling pct [" spct "]")))
     (json-response (component-op-response id component (str "/debug/" action)) (m "callback")))
   (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
+    (mark! ui:num-topology-op-response-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "rebalance" (topology-config id))
     (thrift/with-configured-nimbus-connection nimbus
-    (assert-authorized-user servlet-request "rebalance" (topology-config id))
       (let [tplg (->> (doto
                         (GetInfoOptions.)
                         (.set_num_err_choice NumErrorsChoice/NONE))
@@ -1125,7 +1061,9 @@
         (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
     (json-response (topology-op-response id "rebalance") (m "callback")))
   (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time & m]
-    (assert-authorized-user servlet-request "killTopology" (topology-config id))
+    (mark! ui:num-topology-op-response-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "killTopology" (topology-config id))
     (thrift/with-configured-nimbus-connection nimbus
       (let [tplg (->> (doto
                         (GetInfoOptions.)
@@ -1137,19 +1075,161 @@
         (.killTopologyWithOpts nimbus name options)
         (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))
     (json-response (topology-op-response id "kill") (m "callback")))
+  (POST "/api/v1/topology/:id/logconfig" [:as {:keys [cookies servlet-request]} id namedLoggerLevels & m]
+    (mark! ui:num-topology-op-response-http-requests)
+    (populate-context! servlet-request)
+    (assert-authorized-user "setLogConfig" (topology-config id))
+    (thrift/with-configured-nimbus-connection
+      nimbus
+      (let [new-log-config (LogConfig.)]
+        (doseq [[key level] namedLoggerLevels]
+            (let [logger-name (str key)
+                  target-level (.get level "target_level")
+                  timeout (or (.get level "timeout") 0)
+                  named-logger-level (LogLevel.)]
+              ;; if target-level is nil, do not set it, user wants to clear
+              (log-message "The target level for " logger-name " is " target-level)
+              (if (nil? target-level)
+                (do
+                  (.set_action named-logger-level LogLevelAction/REMOVE)
+                  (.unset_target_log_level named-logger-level))
+                (do
+                  (.set_action named-logger-level LogLevelAction/UPDATE)
+                  ;; the toLevel here ensures the string we get is valid
+                  (.set_target_log_level named-logger-level (.name (Level/toLevel target-level)))
+                  (.set_reset_log_level_timeout_secs named-logger-level timeout)))
+              (log-message "Adding this " logger-name " " named-logger-level " to " new-log-config)
+              (.put_to_named_logger_level new-log-config logger-name named-logger-level)))
+        (log-message "Setting topology " id " log config " new-log-config)
+        (.setLogConfig nimbus id new-log-config)
+        (json-response (log-config id) (m "callback")))))
+
+  (GET "/api/v1/topology/:id/profiling/start/:host-port/:timeout"
+       [:as {:keys [servlet-request]} id host-port timeout & m]
+       (thrift/with-configured-nimbus-connection nimbus
+         (let [user (.getUserName http-creds-handler servlet-request)
+               topology-conf (from-json
+                              (.getTopologyConf ^Nimbus$Client nimbus id))]
+           (assert-authorized-user "setWorkerProfiler" (topology-config id))
+           (assert-authorized-profiler-action "start"))
+
+         (let [[host, port] (split host-port #":")
+               nodeinfo (NodeInfo. host (set [(Long. port)]))
+               timestamp (+ (System/currentTimeMillis) (* 60000 (Long. timeout)))
+               request (ProfileRequest. nodeinfo
+                                        ProfileAction/JPROFILE_STOP)]
+           (.set_time_stamp request timestamp)
+           (.setWorkerProfiler nimbus id request)
+           (json-response {"status" "ok"
+                           "id" host-port
+                           "timeout" timeout
+                           "dumplink" (worker-dump-link
+                                       host
+                                       port
+                                       id)}
+                          (m "callback")))))
+
+  (GET "/api/v1/topology/:id/profiling/stop/:host-port"
+       [:as {:keys [servlet-request]} id host-port & m]
+       (thrift/with-configured-nimbus-connection nimbus
+         (let [user (.getUserName http-creds-handler servlet-request)
+               topology-conf (from-json
+                              (.getTopologyConf ^Nimbus$Client nimbus id))]
+           (assert-authorized-user "setWorkerProfiler" (topology-config id))
+           (assert-authorized-profiler-action "stop"))
+         (let [[host, port] (split host-port #":")
+               nodeinfo (NodeInfo. host (set [(Long. port)]))
+               timestamp 0
+               request (ProfileRequest. nodeinfo
+                                        ProfileAction/JPROFILE_STOP)]
+           (.set_time_stamp request timestamp)
+           (.setWorkerProfiler nimbus id request)
+           (json-response {"status" "ok"
+                           "id" host-port}
+                          (m "callback")))))
+  
+  (GET "/api/v1/topology/:id/profiling/dumpprofile/:host-port"
+       [:as {:keys [servlet-request]} id host-port & m]
+       (thrift/with-configured-nimbus-connection nimbus
+         (let [user (.getUserName http-creds-handler servlet-request)
+               topology-conf (from-json
+                              (.getTopologyConf ^Nimbus$Client nimbus id))]
+           (assert-authorized-user "setWorkerProfiler" (topology-config id))
+           (assert-authorized-profiler-action "dumpprofile"))
+         (let [[host, port] (split host-port #":")
+               nodeinfo (NodeInfo. host (set [(Long. port)]))
+               timestamp (System/currentTimeMillis)
+               request (ProfileRequest. nodeinfo
+                                        ProfileAction/JPROFILE_DUMP)]
+           (.set_time_stamp request timestamp)
+           (.setWorkerProfiler nimbus id request)
+           (json-response {"status" "ok"
+                           "id" host-port}
+                          (m "callback")))))
+
+  (GET "/api/v1/topology/:id/profiling/dumpjstack/:host-port"
+       [:as {:keys [servlet-request]} id host-port & m]
+       (thrift/with-configured-nimbus-connection nimbus
+         (let [user (.getUserName http-creds-handler servlet-request)
+               topology-conf (from-json
+                              (.getTopologyConf ^Nimbus$Client nimbus id))]
+           (assert-authorized-user "setWorkerProfiler" (topology-config id))
+           (assert-authorized-profiler-action "dumpjstack"))
+         (let [[host, port] (split host-port #":")
+               nodeinfo (NodeInfo. host (set [(Long. port)]))
+               timestamp (System/currentTimeMillis)
+               request (ProfileRequest. nodeinfo
+                                        ProfileAction/JSTACK_DUMP)]
+           (.set_time_stamp request timestamp)
+           (.setWorkerProfiler nimbus id request)
+           (json-response {"status" "ok"
+                           "id" host-port}
+                          (m "callback")))))
+
+  (GET "/api/v1/topology/:id/profiling/restartworker/:host-port"
+       [:as {:keys [servlet-request]} id host-port & m]
+       (thrift/with-configured-nimbus-connection nimbus
+         (let [user (.getUserName http-creds-handler servlet-request)
+               topology-conf (from-json
+                              (.getTopologyConf ^Nimbus$Client nimbus id))]
+           (assert-authorized-user "setWorkerProfiler" (topology-config id))
+           (assert-authorized-profiler-action "restartworker"))
+         (let [[host, port] (split host-port #":")
+               nodeinfo (NodeInfo. host (set [(Long. port)]))
+               timestamp (System/currentTimeMillis)
+               request (ProfileRequest. nodeinfo
+                                        ProfileAction/JVM_RESTART)]
+           (.set_time_stamp request timestamp)
+           (.setWorkerProfiler nimbus id request)
+           (json-response {"status" "ok"
+                           "id" host-port}
+                          (m "callback")))))
+       
+  (GET "/api/v1/topology/:id/profiling/dumpheap/:host-port"
+       [:as {:keys [servlet-request]} id host-port & m]
+       (thrift/with-configured-nimbus-connection nimbus
+         (let [user (.getUserName http-creds-handler servlet-request)
+               topology-conf (from-json
+                              (.getTopologyConf ^Nimbus$Client nimbus id))]
+           (assert-authorized-user "setWorkerProfiler" (topology-config id))
+           (assert-authorized-profiler-action "dumpheap"))
+         (let [[host, port] (split host-port #":")
+               nodeinfo (NodeInfo. host (set [(Long. port)]))
+               timestamp (System/currentTimeMillis)
+               request (ProfileRequest. nodeinfo
+                                        ProfileAction/JMAP_DUMP)]
+           (.set_time_stamp request timestamp)
+           (.setWorkerProfiler nimbus id request)
+           (json-response {"status" "ok"
+                           "id" host-port}
+                          (m "callback")))))
+  
   (GET "/" [:as {cookies :cookies}]
-       (resp/redirect "/index.html"))
+    (mark! ui:num-main-page-http-requests)
+    (resp/redirect "/index.html"))
   (route/resources "/")
   (route/not-found "Page not found"))
 
-(defn exception->json
-  [ex]
-  {"error" "Internal Server Error"
-   "errorMessage"
-   (let [sw (java.io.StringWriter.)]
-     (.printStackTrace ex (java.io.PrintWriter. sw))
-     (.toString sw))})
-
 (defn catch-errors
   [handler]
   (fn [request]
@@ -1163,6 +1243,7 @@
                     (wrap-json-params)
                     (wrap-multipart-params)
                     (wrap-reload '[backtype.storm.ui.core])
+                    requests-middleware
                     catch-errors)))
 
 (defn start-server!
@@ -1182,6 +1263,7 @@
           https-ts-type (conf UI-HTTPS-TRUSTSTORE-TYPE)
           https-want-client-auth (conf UI-HTTPS-WANT-CLIENT-AUTH)
           https-need-client-auth (conf UI-HTTPS-NEED-CLIENT-AUTH)]
+      (start-metrics-reporters)
       (storm-run-jetty {:port (conf UI-PORT)
                         :host (conf UI-HOST)
                         :https-port https-port
diff --git a/storm-core/src/clj/backtype/storm/ui/helpers.clj b/storm-core/src/clj/backtype/storm/ui/helpers.clj
index 47e846f..e0db5c8 100644
--- a/storm-core/src/clj/backtype/storm/ui/helpers.clj
+++ b/storm-core/src/clj/backtype/storm/ui/helpers.clj
@@ -20,9 +20,10 @@
          [string :only [blank? join]]
          [walk :only [keywordize-keys]]])
   (:use [backtype.storm config log])
-  (:use [backtype.storm.util :only [clojurify-structure uuid defnk url-encode not-nil?]])
+  (:use [backtype.storm.util :only [clojurify-structure uuid defnk to-json url-encode not-nil?]])
   (:use [clj-time coerce format])
   (:import [backtype.storm.generated ExecutorInfo ExecutorSummary])
+  (:import [backtype.storm.logging.filters AccessLoggingFilter])
   (:import [java.util EnumSet])
   (:import [org.eclipse.jetty.server Server]
            [org.eclipse.jetty.server.nio SelectChannelConnector]
@@ -33,7 +34,16 @@
            [org.eclipse.jetty.servlets CrossOriginFilter])
   (:require [ring.util servlet])
   (:require [compojure.route :as route]
-            [compojure.handler :as handler]))
+            [compojure.handler :as handler])
+  (:require [metrics.meters :refer [defmeter mark!]]))
+
+(defmeter num-web-requests)
+(defn requests-middleware
+  "Coda Hale metric for counting the number of web requests."
+  [handler]
+  (fn [req]
+    (mark! num-web-requests)
+    (handler req)))
 
 (defn split-divide [val divider]
   [(Integer. (int (/ val divider))) (mod val divider)]
@@ -96,43 +106,18 @@
       )]
    ])
 
-(defn float-str [n]
-  (if n
-    (format "%.3f" (float n))
-    "0"
-    ))
-
-(defn swap-map-order [m]
-  (->> m
-       (map (fn [[k v]]
-              (into
-               {}
-               (for [[k2 v2] v]
-                 [k2 {k v2}]
-                 ))
-              ))
-       (apply merge-with merge)
-       ))
-
 (defn url-format [fmt & args]
   (String/format fmt
     (to-array (map #(url-encode (str %)) args))))
 
-(defn to-tasks [^ExecutorInfo e]
-  (let [start (.get_task_start e)
-        end (.get_task_end e)]
-    (range start (inc end))
-    ))
-
-(defn sum-tasks [executors]
-  (reduce + (->> executors
-                 (map #(.get_executor_info ^ExecutorSummary %))
-                 (map to-tasks)
-                 (map count))))
-
 (defn pretty-executor-info [^ExecutorInfo e]
   (str "[" (.get_task_start e) "-" (.get_task_end e) "]"))
 
+(defn unauthorized-user-json
+  [user]
+  {"error" "No Authorization"
+   "errorMessage" (str "User " user " is not authorized.")})
+
 (defn unauthorized-user-html [user]
   [[:h2 "User '" (escape-html user) "' is not authorized."]])
 
@@ -173,6 +158,9 @@
     (.setInitParameter CrossOriginFilter/ACCESS_CONTROL_ALLOW_ORIGIN_HEADER "*")
     ))
 
+(defn mk-access-logging-filter-handler []
+  (org.eclipse.jetty.servlet.FilterHolder. (AccessLoggingFilter.)))
+
 (defn config-filter [server handler filters-confs]
   (if filters-confs
     (let [servlet-holder (ServletHolder.
@@ -187,6 +175,7 @@
                                 (.setName (or filter-name filter-class))
                                 (.setInitParameters (or filter-params {})))]
             (.addFilter context filter-holder "/*" FilterMapping/ALL))))
+      (.addFilter context (mk-access-logging-filter-handler) "/*" (EnumSet/allOf DispatcherType))
       (.setHandler server context))))
 
 (defn ring-response-from-exception [ex]
@@ -225,3 +214,27 @@
         configurator (:configurator config)]
     (configurator s)
     (.start s)))
+
+(defn wrap-json-in-callback [callback response]
+  (str callback "(" response ");"))
+
+(defnk json-response
+  [data callback :serialize-fn to-json :status 200 :headers {}]
+  {:status status
+   :headers (merge {"Cache-Control" "no-cache, no-store"
+                    "Access-Control-Allow-Origin" "*"
+                    "Access-Control-Allow-Headers" "Content-Type, Access-Control-Allow-Headers, Access-Controler-Allow-Origin, X-Requested-By, X-Csrf-Token, Authorization, X-Requested-With"}
+              (if (not-nil? callback) {"Content-Type" "application/javascript;charset=utf-8"}
+                {"Content-Type" "application/json;charset=utf-8"})
+              headers)
+   :body (if (not-nil? callback)
+           (wrap-json-in-callback callback (serialize-fn data))
+           (serialize-fn data))})
+
+(defn exception->json
+  [ex]
+  {"error" "Internal Server Error"
+   "errorMessage"
+   (let [sw (java.io.StringWriter.)]
+     (.printStackTrace ex (java.io.PrintWriter. sw))
+     (.toString sw))})
diff --git a/storm-core/src/clj/backtype/storm/util.clj b/storm-core/src/clj/backtype/storm/util.clj
index c0f266a..9b22e70 100644
--- a/storm-core/src/clj/backtype/storm/util.clj
+++ b/storm-core/src/clj/backtype/storm/util.clj
@@ -26,11 +26,14 @@
   (:import [java.util.zip ZipFile])
   (:import [java.util.concurrent.locks ReentrantReadWriteLock])
   (:import [java.util.concurrent Semaphore])
+  (:import [java.nio.file Files Paths])
+  (:import [java.nio.file.attribute FileAttribute])
   (:import [java.io File FileOutputStream RandomAccessFile StringWriter
             PrintWriter BufferedReader InputStreamReader IOException])
   (:import [java.lang.management ManagementFactory])
   (:import [org.apache.commons.exec DefaultExecutor CommandLine])
   (:import [org.apache.commons.io FileUtils])
+  (:import [backtype.storm.logging ThriftAccessLogger])
   (:import [org.apache.commons.exec ExecuteException])
   (:import [org.json.simple JSONValue])
   (:import [org.yaml.snakeyaml Yaml]
@@ -514,18 +517,25 @@
     (map #(str \' (clojure.string/escape % {\' "'\"'\"'"}) \'))
       (clojure.string/join " ")))
 
+(defn script-file-path [dir]
+  (str dir file-path-separator "storm-worker-script.sh"))
+
+(defn container-file-path [dir]
+  (str dir file-path-separator "launch_container.sh"))
+
 (defnk write-script
   [dir command :environment {}]
   (let [script-src (str "#!/bin/bash\n" (clojure.string/join "" (map (fn [[k v]] (str (shell-cmd ["export" (str k "=" v)]) ";\n")) environment)) "\nexec " (shell-cmd command) ";")
-        script-path (str dir "/storm-worker-script.sh")
-        - (spit script-path script-src)]
+        script-path (script-file-path dir)
+        _ (spit script-path script-src)]
     script-path
   ))
 
 (defnk launch-process
-  [command :environment {} :log-prefix nil :exit-code-callback nil]
+  [command :environment {} :log-prefix nil :exit-code-callback nil :directory nil]
   (let [builder (ProcessBuilder. command)
         process-env (.environment builder)]
+    (when directory (.directory builder directory))
     (.redirectErrorStream builder true)
     (doseq [[k v] environment]
       (.put process-env k v))
@@ -578,6 +588,21 @@
     (when-not success?
       (throw (RuntimeException. (str "Failed to touch " path))))))
 
+(defn create-symlink!
+  "Create symlink is to the target"
+  ([path-dir target-dir file-name]
+    (create-symlink! path-dir target-dir file-name file-name))
+  ([path-dir target-dir from-file-name to-file-name]
+    (let [path (str path-dir file-path-separator from-file-name)
+          target (str target-dir file-path-separator to-file-name)
+          empty-array (make-array String 0)
+          attrs (make-array FileAttribute 0)
+          abs-path (.toAbsolutePath (Paths/get path empty-array))
+          abs-target (.toAbsolutePath (Paths/get target empty-array))]
+      (log-debug "Creating symlink [" abs-path "] to [" abs-target "]")
+      (if (not (.exists (.toFile abs-path)))
+        (Files/createSymbolicLink abs-path abs-target attrs)))))
+
 (defn read-dir-contents
   [dir]
   (if (exists-file? dir)
@@ -755,10 +780,6 @@
           rest-elems (apply interleave-all (map rest colls))]
       (concat my-elems rest-elems))))
 
-(defn update
-  [m k afn]
-  (assoc m k (afn (get m k))))
-
 (defn any-intersection
   [& sets]
   (let [elem->count (multi-set (apply concat sets))]
@@ -1018,30 +1039,19 @@
   ([x form & more] `(-<> (-<> ~x ~form) ~@more)))
 
 (def LOG-DIR
-  (.getCanonicalPath 
-                (clojure.java.io/file (System/getProperty "storm.home") "logs")))
+  (.getCanonicalPath
+    (clojure.java.io/file (or (System/getProperty "storm.log.dir") (str (System/getProperty "storm.home") "logs")))))
 
-(defn- logs-rootname
-  ([storm-id port] (logs-rootname storm-id port "-worker-"))
-  ([storm-id port type] (str storm-id type port)))
 
 (defn logs-filename
-  ([storm-id port] (str (logs-rootname storm-id port) ".log"))
-  ([storm-id port type] (str (logs-rootname storm-id port type) ".log")))
+  [storm-id port]
+  (str storm-id file-path-separator port file-path-separator "worker.log"))
 
-(defn event-logs-filename [storm-id port] (logs-filename storm-id port "-events-"))
+(def worker-log-filename-pattern #"^worker.log(.*)")
 
-(defn logs-metadata-filename [storm-id port]
-  (str (logs-rootname storm-id port) ".yaml"))
-
-(def worker-log-filename-pattern #"^((.*-\d+-\d+)-worker-(\d+))\.log")
-
-(defn get-log-metadata-file
-  ([fname]
-    (if-let [[_ _ id port] (re-matches worker-log-filename-pattern fname)]
-      (get-log-metadata-file id port)))
-  ([id port]
-    (clojure.java.io/file LOG-DIR "metadata" (logs-metadata-filename id port))))
+(defn event-logs-filename
+  [storm-id port]
+  (str storm-id file-path-separator port file-path-separator "events.log"))
 
 (defn clojure-from-yaml-file [yamlFile]
   (try
@@ -1053,6 +1063,22 @@
 (defn hashmap-to-persistent [^HashMap m]
   (zipmap (.keySet m) (.values m)))
 
+(defn retry-on-exception
+  "Retries specific function on exception based on retries count"
+  [retries task-description f & args]
+  (let [res (try {:value (apply f args)}
+              (catch Exception e
+                (if (<= 0 retries)
+                  (throw e)
+                  {:exception e})))]
+    (if (:exception res)
+      (do 
+        (log-error (:exception res) (str "Failed to " task-description ". Will make [" retries "] more attempts."))
+        (recur (dec retries) task-description f args))
+      (do 
+        (log-debug (str "Successful " task-description "."))
+        (:value res)))))
+
 (defn setup-default-uncaught-exception-handler
   "Set a default uncaught exception handler to handle exceptions not caught in other threads."
   []
@@ -1073,3 +1099,7 @@
     (assoc coll k (apply str (repeat (count (coll k)) "#")))
     coll))
 
+(defn log-thrift-access [request-id remoteAddress principal operation]
+  (doto
+    (ThriftAccessLogger.)
+    (.log (str "Request ID: " request-id " access from: " remoteAddress " principal: " principal " operation: " operation))))
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker.clj
new file mode 100644
index 0000000..588f6aa
--- /dev/null
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker.clj
@@ -0,0 +1,237 @@
+;; 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 org.apache.storm.pacemaker.pacemaker
+  (:import [org.apache.storm.pacemaker PacemakerServer IServerMessageHandler]
+           [java.util.concurrent ConcurrentHashMap]
+           [java.util.concurrent.atomic AtomicInteger]
+           [backtype.storm.generated HBNodes
+            HBServerMessageType HBMessage HBMessageData HBPulse])
+  (:use [clojure.string :only [replace-first split]]
+        [backtype.storm log config util])
+  (:require [clojure.java.jmx :as jmx])
+  (:gen-class))
+
+
+;; Stats Functions
+
+(def sleep-seconds 60)
+
+
+(defn- check-and-set-loop [stats key new & {:keys [compare new-fn]
+                                            :or {compare (fn [new old] true)
+                                                 new-fn (fn [new old] new)}}]
+  (loop []
+    (let [old (.get (key stats))
+          new (new-fn new old)]
+      (if (compare new old)
+        (if (.compareAndSet (key stats) old new)
+          nil
+          (recur))
+        nil))))
+
+(defn- set-average [stats size]
+  (check-and-set-loop
+   stats
+   :average-heartbeat-size
+   size
+   :new-fn (fn [new old]
+            (let [count (.get (:send-pulse-count stats))]
+                                        ; Weighted average
+              (/ (+ new (* count old)) (+ count 1))))))
+
+(defn- set-largest [stats size]
+  (check-and-set-loop
+   stats
+   :largest-heartbeat-size
+   size
+   :compare #'>))
+
+(defn- report-stats [heartbeats stats last-five-s]
+  (loop []
+    (let [send-count (.getAndSet (:send-pulse-count stats) 0)
+          received-size (.getAndSet (:total-received-size stats) 0)
+          get-count (.getAndSet (:get-pulse-count stats) 0)
+          sent-size (.getAndSet (:total-sent-size stats) 0)
+          largest (.getAndSet (:largest-heartbeat-size stats) 0)
+          average (.getAndSet (:average-heartbeat-size stats) 0)
+          total-keys (.size heartbeats)]
+      (log-debug "\nReceived " send-count " heartbeats totaling " received-size " bytes,\n"
+                 "Sent " get-count " heartbeats totaling " sent-size " bytes,\n"
+                 "The largest heartbeat was " largest " bytes,\n"
+                 "The average heartbeat was " average " bytes,\n"
+                 "Pacemaker contained " total-keys " total keys\n"
+                 "in the last " sleep-seconds " second(s)")
+      (dosync (ref-set last-five-s
+                       {:send-pulse-count send-count
+                        :total-received-size received-size
+                        :get-pulse-count get-count
+                        :total-sent-size sent-size
+                        :largest-heartbeat-size largest
+                        :average-heartbeat-size average
+                        :total-keys total-keys})))
+    (Thread/sleep (* 1000 sleep-seconds))
+    (recur)))
+
+;; JMX stuff
+(defn register [last-five-s]
+  (jmx/register-mbean
+    (jmx/create-bean
+      last-five-s)
+    "org.apache.storm.pacemaker.pacemaker:stats=Stats_Last_5_Seconds"))
+
+
+;; Pacemaker Functions
+
+(defn hb-data []
+  (ConcurrentHashMap.))
+
+(defn create-path [^String path heartbeats]
+  (HBMessage. HBServerMessageType/CREATE_PATH_RESPONSE nil))
+
+(defn exists [^String path heartbeats]
+  (let [it-does (.containsKey heartbeats path)]
+    (log-debug (str "Checking if path [" path "] exists..." it-does "."))
+    (HBMessage. HBServerMessageType/EXISTS_RESPONSE
+                (HBMessageData/boolval it-does))))
+
+(defn send-pulse [^HBPulse pulse heartbeats pacemaker-stats]
+  (let [id (.get_id pulse)
+        details (.get_details pulse)]
+    (log-debug (str "Saving Pulse for id [" id "] data [" + (str details) "]."))
+
+    (.incrementAndGet (:send-pulse-count pacemaker-stats))
+    (.addAndGet (:total-received-size pacemaker-stats) (alength details))
+    (set-largest pacemaker-stats (alength details))
+    (set-average pacemaker-stats (alength details))
+
+    (.put heartbeats id details)
+    (HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)))
+
+(defn get-all-pulse-for-path [^String path heartbeats]
+  (HBMessage. HBServerMessageType/GET_ALL_PULSE_FOR_PATH_RESPONSE nil))
+
+(defn get-all-nodes-for-path [^String path ^ConcurrentHashMap heartbeats]
+  (log-debug "List all nodes for path " path)
+  (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
+              (HBMessageData/nodes
+               (HBNodes. (distinct (for [k (.keySet heartbeats)
+                                         :let [trimmed-k (first
+                                                          (filter #(not (= "" %))
+                                                                  (split (replace-first k path "") #"/")))]
+                                         :when (and
+                                                (not (nil? trimmed-k))
+                                                (= (.indexOf k path) 0))]
+                                     trimmed-k))))))
+
+(defn get-pulse [^String path heartbeats pacemaker-stats]
+  (let [details (.get heartbeats path)]
+    (log-debug (str "Getting Pulse for path [" path "]...data " (str details) "]."))
+
+
+    (.incrementAndGet (:get-pulse-count pacemaker-stats))
+    (if details
+      (.addAndGet (:total-sent-size pacemaker-stats) (alength details)))
+
+    (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
+                (HBMessageData/pulse
+                 (doto (HBPulse. ) (.set_id path) (.set_details details))))))
+
+(defn delete-pulse-id [^String path heartbeats]
+  (log-debug (str "Deleting Pulse for id [" path "]."))
+  (.remove heartbeats path)
+  (HBMessage. HBServerMessageType/DELETE_PULSE_ID_RESPONSE nil))
+
+(defn delete-path [^String path heartbeats]
+  (let [prefix (if (= \/ (last path)) path (str path "/"))]
+    (doseq [k (.keySet heartbeats)
+            :when (= (.indexOf k prefix) 0)]
+      (delete-pulse-id k heartbeats)))
+  (HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil))
+
+(defn not-authorized []
+  (HBMessage. HBServerMessageType/NOT_AUTHORIZED nil))
+
+(defn mk-handler [conf]
+  (let [heartbeats ^ConcurrentHashMap (hb-data)
+        pacemaker-stats {:send-pulse-count (AtomicInteger.)
+                         :total-received-size (AtomicInteger.)
+                         :get-pulse-count (AtomicInteger.)
+                         :total-sent-size (AtomicInteger.)
+                         :largest-heartbeat-size (AtomicInteger.)
+                         :average-heartbeat-size (AtomicInteger.)}
+        last-five (ref {:send-pulse-count 0
+                        :total-received-size 0
+                        :get-pulse-count 0
+                        :total-sent-size 0
+                        :largest-heartbeat-size 0
+                        :average-heartbeat-size 0
+                        :total-keys 0})
+        stats-thread (Thread. (fn [] (report-stats heartbeats pacemaker-stats last-five)))]
+    (.setDaemon stats-thread true)
+    (.start stats-thread)
+    (register last-five)
+    (reify
+      IServerMessageHandler
+      (^HBMessage handleMessage [this ^HBMessage request ^boolean authenticated]
+        (let [response
+              (condp = (.get_type request)
+                HBServerMessageType/CREATE_PATH
+                (create-path (.get_path (.get_data request)) heartbeats)
+
+                HBServerMessageType/EXISTS
+                (if authenticated
+                  (exists (.get_path (.get_data request)) heartbeats)
+                  (not-authorized))
+
+                HBServerMessageType/SEND_PULSE
+                (send-pulse (.get_pulse (.get_data request)) heartbeats pacemaker-stats)
+
+                HBServerMessageType/GET_ALL_PULSE_FOR_PATH
+                (if authenticated
+                  (get-all-pulse-for-path (.get_path (.get_data request)) heartbeats)
+                  (not-authorized))
+
+                HBServerMessageType/GET_ALL_NODES_FOR_PATH
+                (if authenticated
+                  (get-all-nodes-for-path (.get_path (.get_data request)) heartbeats)
+                  (not-authorized))
+
+                HBServerMessageType/GET_PULSE
+                (if authenticated
+                  (get-pulse (.get_path (.get_data request)) heartbeats pacemaker-stats)
+                  (not-authorized))
+
+                HBServerMessageType/DELETE_PATH
+                (delete-path (.get_path (.get_data request)) heartbeats)
+
+                HBServerMessageType/DELETE_PULSE_ID
+                (delete-pulse-id (.get_path (.get_data request)) heartbeats)
+
+                ; Otherwise
+                (log-message "Got Unexpected Type: " (.get_type request)))]
+
+          (.set_message_id response (.get_message_id request))
+          response)))))
+
+(defn launch-server! []
+  (log-message "Starting Server.")
+  (let [conf (override-login-config-with-system-property (read-storm-config))]
+    (PacemakerServer. (mk-handler conf) conf)))
+
+(defn -main []
+  (redirect-stdio-to-slf4j!)
+  (launch-server!))
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
new file mode 100644
index 0000000..537397a
--- /dev/null
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -0,0 +1,124 @@
+;; 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 org.apache.storm.pacemaker.pacemaker-state-factory
+  (:require [org.apache.storm.pacemaker pacemaker]
+            [backtype.storm.cluster-state [zookeeper-state-factory :as zk-factory]]
+            [backtype.storm
+             [config :refer :all]
+             [cluster :refer :all]
+             [log :refer :all]
+             [util :as util]])
+  (:import [backtype.storm.generated
+            HBExecutionException HBServerMessageType HBMessage
+            HBMessageData HBPulse]
+           [backtype.storm.cluster_state zookeeper_state_factory]
+           [backtype.storm.cluster ClusterState]
+           [org.apache.storm.pacemaker PacemakerClient])
+  (:gen-class
+   :implements [backtype.storm.cluster.ClusterStateFactory]))
+
+;; So we can mock the client for testing
+(defn makeClient [conf]
+  (PacemakerClient. conf))
+
+(defn makeZKState [conf auth-conf acls context]
+  (.mkState (zookeeper_state_factory.) conf auth-conf acls context))
+
+(def max-retries 10)
+
+(defn -mkState [this conf auth-conf acls context]
+  (let [zk-state (makeZKState conf auth-conf acls context)
+        pacemaker-client (makeClient conf)]
+
+    (reify
+      ClusterState
+      ;; Let these pass through to the zk-state. We only want to handle heartbeats.
+      (register [this callback] (.register zk-state callback))
+      (unregister [this callback] (.unregister zk-state callback))
+      (set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
+      (create_sequential [this path data acls] (.create_sequential zk-state path data acls))
+      (set_data [this path data acls] (.set_data zk-state path data acls))
+      (delete_node [this path] (.delete_node zk-state path))
+      (get_data [this path watch?] (.get_data zk-state path watch?))
+      (get_data_with_version [this path watch?] (.get_data_with_version zk-state path watch?))
+      (get_version [this path watch?] (.get_version zk-state path watch?))
+      (get_children [this path watch?] (.get_children zk-state path watch?))
+      (mkdirs [this path acls] (.mkdirs zk-state path acls))
+      (node_exists [this path watch?] (.node_exists zk-state path watch?))
+      (add_listener [this listener] (.add_listener zk-state listener))
+      (sync_path [this path] (.sync_path zk-state path))
+      
+      (set_worker_hb [this path data acls]
+        (util/retry-on-exception
+         max-retries
+         "set_worker_hb"
+         #(let [response
+                (.send pacemaker-client
+                       (HBMessage. HBServerMessageType/SEND_PULSE
+                                   (HBMessageData/pulse
+                                    (doto (HBPulse.)
+                                      (.set_id path)
+                                      (.set_details data)))))]
+            (if (= (.get_type response) HBServerMessageType/SEND_PULSE_RESPONSE)
+              :ok
+              (throw (HBExecutionException. "Invalid Response Type"))))))
+
+      (delete_worker_hb [this path]
+        (util/retry-on-exception
+         max-retries
+         "delete_worker_hb"
+         #(let [response
+                (.send pacemaker-client
+                       (HBMessage. HBServerMessageType/DELETE_PATH
+                                   (HBMessageData/path path)))]
+            (if (= (.get_type response) HBServerMessageType/DELETE_PATH_RESPONSE)
+              :ok
+              (throw (HBExecutionException. "Invalid Response Type"))))))
+      
+      (get_worker_hb [this path watch?]
+        (util/retry-on-exception
+         max-retries
+         "get_worker_hb"
+         #(let [response
+                (.send pacemaker-client
+                       (HBMessage. HBServerMessageType/GET_PULSE
+                                   (HBMessageData/path path)))]
+            (if (= (.get_type response) HBServerMessageType/GET_PULSE_RESPONSE)
+              (try 
+                (.get_details (.get_pulse (.get_data response)))
+                (catch Exception e
+                  (throw (HBExecutionException. (.toString e)))))
+              (throw (HBExecutionException. "Invalid Response Type"))))))
+      
+      (get_worker_hb_children [this path watch?]
+        (util/retry-on-exception
+         max-retries
+         "get_worker_hb_children"
+         #(let [response
+                (.send pacemaker-client
+                       (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH
+                                   (HBMessageData/path path)))]
+            (if (= (.get_type response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE)
+              (try
+                (into [] (.get_pulseIds (.get_nodes (.get_data response))))
+                (catch Exception e
+                  (throw (HBExecutionException. (.toString e)))))
+              (throw (HBExecutionException. "Invalid Response Type"))))))
+      
+      (close [this]
+        (.close zk-state)
+        (.close pacemaker-client)))))
diff --git a/storm-core/src/dev/logviewer-search-context-tests.log b/storm-core/src/dev/logviewer-search-context-tests.log
new file mode 100644
index 0000000..6e4d4af
--- /dev/null
+++ b/storm-core/src/dev/logviewer-search-context-tests.log
@@ -0,0 +1 @@
+needle needle000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000needle needle
diff --git a/storm-core/src/dev/logviewer-search-context-tests.log.gz b/storm-core/src/dev/logviewer-search-context-tests.log.gz
new file mode 100644
index 0000000..5cf2a06
--- /dev/null
+++ b/storm-core/src/dev/logviewer-search-context-tests.log.gz
Binary files differ
diff --git a/storm-core/src/dev/small-worker.log b/storm-core/src/dev/small-worker.log
new file mode 100644
index 0000000..27d61d1
--- /dev/null
+++ b/storm-core/src/dev/small-worker.log
@@ -0,0 +1 @@
+000000 needle 000000
diff --git a/storm-core/src/dev/test-3072.log b/storm-core/src/dev/test-3072.log
new file mode 100644
index 0000000..56dc6f1
--- /dev/null
+++ b/storm-core/src/dev/test-3072.log
@@ -0,0 +1,3 @@
+This is a test log file of size 3072.
+
+...................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................needle
\ No newline at end of file
diff --git a/storm-core/src/dev/test-worker.log b/storm-core/src/dev/test-worker.log
new file mode 100644
index 0000000..8fb4c53
--- /dev/null
+++ b/storm-core/src/dev/test-worker.log
@@ -0,0 +1,380 @@
+Test needle is near the beginning of the file.
+This file assumes a buffer size of 2048 bytes, a max search string size of 1024 bytes, and a context length of 128 UTF-8 characters.
+The early match tests the case when we find a match too close to the start of the file to give the normal before context strings.
+
+padding 5
+padding 6
+padding 7
+padding 8
+padding 9
+padding 10
+padding 11
+padding 12
+padding 13
+padding 14
+padding 15
+padding 16
+padding 17
+padding 18
+padding 19
+padding 20
+padding 21
+padding 22
+padding 23
+padding 24
+padding 25
+padding 26
+padding 27
+padding 28
+padding 29
+padding 30
+padding 31
+padding 32
+padding 33
+padding 34
+padding 35
+padding 36
+padding 37
+padding 38
+padding 39
+padding 40
+padding 41
+padding 42
+padding 43
+padding 44
+padding 45
+padding 46
+padding 47
+padding 48
+padding 49
+padding 50
+padding 51
+padding 52
+padding 53
+padding 54
+padding 55
+padding 56
+padding 57
+padding 58
+padding 59
+padding 60
+padding 61
+padding 62
+padding 63
+padding 64
+padding 65
+padding 66
+padding 67
+padding 68
+padding 69
+padding 70
+padding 71
+padding 72
+padding 73
+padding 74
+padding 75
+padding 76
+padding 77
+padding 78
+padding 79
+padding 80
+padding 81
+padding 82
+padding 83
+padding 84
+padding 85
+padding 86
+padding 87
+padding 88
+padding 89
+padding 90
+padding 91
+padding 92
+padding 93
+padding 94
+padding 95
+padding 96
+padding 97
+padding 98
+padding 99
+padding 100
+padding 101
+padding 102
+padding 103
+padding 104
+padding 105
+padding 106
+padding 107
+padding 108
+padding 109
+padding 110
+padding 111
+padding 112
+padding 113
+padding 114
+padding 115
+padding 116
+padding 117
+padding 118
+padding 119
+padding 120
+padding 121
+padding 122
+padding 123
+padding 124
+padding 125
+padding 126
+padding 127
+padding 128
+padding 129
+padding 130
+padding 131
+padding 132
+padding 133
+padding 134
+padding 135
+padding 136
+padding 137
+padding 138
+padding 139
+padding 140
+padding 141
+padding 142
+padding 143
+padding 144
+padding 145
+padding 146
+padding 147
+padding 148
+padding 149
+padding 150
+padding 151
+padding 152
+padding 153
+Near the end of a 1024 byte block, a needle.
+A needle that straddles a 1024 byte boundary should also be detected.
+
+padding 157
+padding 158
+padding 159
+padding 160
+padding 161
+padding 162
+padding 163
+padding 164
+padding 165
+padding 166
+padding 167
+padding 168
+padding 169
+padding 170
+padding 171
+padding 172
+padding 173
+padding 174
+padding 175
+padding 176
+padding 177
+padding 178
+padding 179
+padding 180
+padding 181
+padding 182
+padding 183
+padding 184
+padding 185
+padding 186
+padding 187
+padding 188
+padding 189
+padding 190
+padding 191
+padding 192
+padding 193
+padding 194
+padding 195
+padding 196
+padding 197
+padding 198
+padding 199
+padding 200
+padding 201
+padding 202
+padding 203
+padding 204
+padding 205
+padding 206
+padding 207
+padding 208
+padding 209
+padding 210
+padding 211
+padding 212
+padding 213
+padding 214
+padding 215
+padding 216
+padding 217
+padding 218
+padding 219
+padding 220
+padding 221
+padding 222
+padding 223
+padding 224
+padding 225
+padding 226
+padding 227
+padding 228
+padding 229
+padding 230
+padding 231
+padding 232
+padding 233
+padding 234
+padding 235
+
+
+Here a needle occurs just after a 1024 byte boundary.  It should have the correct context.
+
+Text with two adjoining matches: needleneedle
+
+padding 243
+padding 244
+padding 245
+padding 246
+padding 247
+padding 248
+padding 249
+padding 250
+padding 251
+padding 252
+padding 253
+padding 254
+padding 255
+padding 256
+padding 257
+padding 258
+padding 259
+padding 260
+padding 261
+padding 262
+padding 263
+padding 264
+padding 265
+padding 266
+padding 267
+padding 268
+padding 269
+padding 270
+padding 271
+padding 272
+padding 273
+padding 274
+padding 275
+padding 276
+padding 277
+padding 278
+padding 279
+padding 280
+padding 281
+padding 282
+padding 283
+padding 284
+padding 285
+padding 286
+padding 287
+padding 288
+padding 289
+padding 290
+padding 291
+padding 292
+padding 293
+padding 294
+padding 295
+padding 296
+padding 297
+padding 298
+padding 299
+padding 300
+padding 301
+padding 302
+padding 303
+padding 304
+
+The following match of 1024 bytes completely fills half the byte buffer.  It is a search substring of the maximum size......
+
+XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
+The following max-size match straddles a 1024 byte buffer.
+XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
+
+Here are four non-ascii 1-byte UTF-8 characters: αβγδε
+
+needle
+
+Here are four printable 2-byte UTF-8 characters: ¡¢£¤¥
+
+needle
+
+
+
+Here are four printable 3-byte UTF-8 characters: ऄअआइई
+
+needle
+
+Here are four printable 4-byte UTF-8 characters: 𐄇𐄈𐄉𐄋𐄌
+
+needle
+
+Here are four of the same invalid UTF-8 characters: ö€€€ö€€€ö€€€ö€€€
+
+needle
+
+padding 332
+padding 333
+padding 334
+padding 335
+padding 336
+padding 337
+padding 338
+padding 339
+padding 340
+padding 341
+padding 342
+padding 343
+padding 344
+padding 345
+padding 346
+padding 347
+padding 348
+padding 349
+padding 350
+padding 351
+padding 352
+padding 353
+padding 354
+padding 355
+padding 356
+padding 357
+padding 358
+padding 359
+padding 360
+padding 361
+padding 362
+padding 363
+padding 364
+padding 365
+padding 366
+padding 367
+padding 368
+padding 369
+padding 370
+padding 371
+padding 372
+padding 373
+padding 374
+padding 375
+
+The following tests multibyte UTF-8 Characters straddling the byte boundary:   𐄀𐄁𐄂
+
+needle
\ No newline at end of file
diff --git a/storm-core/src/genthrift.sh b/storm-core/src/genthrift.sh
index 7e90576..54cd10a 100644
--- a/storm-core/src/genthrift.sh
+++ b/storm-core/src/genthrift.sh
@@ -17,7 +17,7 @@
 
 rm -rf gen-javabean gen-py py
 rm -rf jvm/backtype/storm/generated
-thrift --gen java:beans,hashcode,nocamel --gen py:utf8strings storm.thrift
+thrift --gen java:beans,hashcode,nocamel,generated_annotations=undated --gen py:utf8strings storm.thrift
 for file in gen-javabean/backtype/storm/generated/* ; do
   cat java_license_header.txt ${file} > ${file}.tmp
   mv -f ${file}.tmp ${file}
diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java
index 0eff51b..89422f6 100644
--- a/storm-core/src/jvm/backtype/storm/Config.java
+++ b/storm-core/src/jvm/backtype/storm/Config.java
@@ -19,7 +19,10 @@
 
 import backtype.storm.serialization.IKryoDecorator;
 import backtype.storm.serialization.IKryoFactory;
+import backtype.storm.validation.ConfigValidationAnnotations.*;
+import backtype.storm.validation.ConfigValidation.*;
 import com.esotericsoftware.kryo.Serializer;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -32,7 +35,7 @@
  * serializations.
  *
  * <p>This class also provides constants for all the configurations possible on
- * a Storm cluster and Storm topology. Each constant is paired with a schema
+ * a Storm cluster and Storm topology. Each constant is paired with an annotation
  * that defines the validity criterion of the corresponding field. Default
  * values for these configs can be found in defaults.yaml.</p>
  *
@@ -50,178 +53,213 @@
      * This is part of a temporary workaround to a ZK bug, it is the 'scheme:acl' for
      * the user Nimbus and Supervisors use to authenticate with ZK.
      */
+    @isString
     public static final String STORM_ZOOKEEPER_SUPERACL = "storm.zookeeper.superACL";
-    public static final Object STORM_ZOOKEEPER_SUPERACL_SCHEMA = String.class;
 
     /**
      * The transporter for communication among Storm tasks
      */
+    @isString
     public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport";
-    public static final Object STORM_MESSAGING_TRANSPORT_SCHEMA = String.class;
 
     /**
      * Netty based messaging: The buffer size for send/recv buffer
      */
+    @isInteger
+    @isPositiveNumber
     public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size";
-    public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Netty based messaging: Sets the backlog value to specify when the channel binds to a local address
      */
+    @isInteger
+    @isPositiveNumber
     public static final String STORM_MESSAGING_NETTY_SOCKET_BACKLOG = "storm.messaging.netty.socket.backlog";
-    public static final Object STORM_MESSAGING_NETTY_SOCKET_BACKLOG_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible
+     *@deprecated "Since netty clients should never stop reconnecting - this does not make sense anymore.
      */
+    @Deprecated
+    @isInteger
     public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries";
-    public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Netty based messaging: The min # of milliseconds that a peer will wait.
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms";
-    public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Netty based messaging: The max # of milliseconds that a peer will wait.
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms";
-    public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Netty based messaging: The # of worker threads for the server.
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads";
-    public static final Object STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Netty based messaging: The # of worker threads for the client.
      */
+    @isInteger
     public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads";
-    public static final Object STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of STORM_NETTY_MESSAGE_BATCH_SIZE bytes
      */
+    @isInteger
     public static final String STORM_NETTY_MESSAGE_BATCH_SIZE = "storm.messaging.netty.transfer.batch.size";
-    public static final Object STORM_NETTY_MESSAGE_BATCH_SIZE_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * We check with this interval that whether the Netty channel is writable and try to write pending messages
      */
+    @isInteger
     public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms";
-    public static final Object STORM_NETTY_FLUSH_CHECK_INTERVAL_MS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Netty based messaging: Is authentication required for Netty messaging from client worker process to server worker process.
      */
+    @isBoolean
     public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication";
-    public static final Object STORM_MESSAGING_NETTY_AUTHENTICATION_SCHEMA = Boolean.class;
 
     /**
      * The delegate for serializing metadata, should be used for serialized objects stored in zookeeper and on disk.
      * This is NOT used for compressing serialized tuples sent between topologies.
      */
+    @isString
     public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate";
-    public static final Object STORM_META_SERIALIZATION_DELEGATE_SCHEMA = String.class;
 
     /**
      * A list of hosts of ZooKeeper servers used to manage the cluster.
      */
+    @isStringList
     public static final String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers";
-    public static final Object STORM_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * The port Storm will use to connect to each of the ZooKeeper servers.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String STORM_ZOOKEEPER_PORT = "storm.zookeeper.port";
-    public static final Object STORM_ZOOKEEPER_PORT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * A directory on the local filesystem used by Storm for any local
      * filesystem usage it needs. The directory must exist and the Storm daemons must
      * have permission to read/write from this location.
      */
+    @isString
     public static final String STORM_LOCAL_DIR = "storm.local.dir";
-    public static final Object STORM_LOCAL_DIR_SCHEMA = String.class;
 
     /**
      * A directory that holds configuration files for log4j2.
+     * It can be either a relative or an absolute directory.
+     * If relative, it is relative to the storm's home directory.
      */
+    @isString
     public static final String STORM_LOG4J2_CONF_DIR = "storm.log4j2.conf.dir";
-    public static final Object STORM_LOG4J2_CONF_DIR_SCHEMA = String.class;
 
     /**
-     * A global task scheduler used to assign topologies's tasks to supervisors' wokers.
+     * A global task scheduler used to assign topologies's tasks to supervisors' workers.
      *
      * If this is not set, a default system scheduler will be used.
      */
+    @isString
     public static final String STORM_SCHEDULER = "storm.scheduler";
-    public static final Object STORM_SCHEDULER_SCHEMA = String.class;
+
+    /**
+     * Whether we want to display all the resource capacity and scheduled usage on the UI page.
+     * We suggest to have this variable set if you are using any kind of resource-related scheduler.
+     *
+     * If this is not set, we will not display resource capacity and usage on the UI.
+     */
+    @isBoolean
+    public static final String SCHEDULER_DISPLAY_RESOURCE = "scheduler.display.resource";
 
     /**
      * The mode this Storm cluster is running in. Either "distributed" or "local".
      */
+    @isString
     public static final String STORM_CLUSTER_MODE = "storm.cluster.mode";
-    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.
+     */
+    @isString
+    public static final String STORM_NETWORK_TOPOGRAPHY_PLUGIN = "storm.network.topography.plugin";
 
     /**
      * 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>.
      *
-     * You should set this config when you dont have a DNS which supervisors/workers
+     * You should set this config when you don't have a DNS which supervisors/workers
      * can utilize to find each other based on hostname got from calls to
      * <code>InetAddress.getLocalHost().getCanonicalHostName()</code>.
      */
+    @isString
     public static final String STORM_LOCAL_HOSTNAME = "storm.local.hostname";
-    public static final Object STORM_LOCAL_HOSTNAME_SCHEMA = String.class;
 
     /**
      * The plugin that will convert a principal to a local user.
      */
+    @isString
     public static final String STORM_PRINCIPAL_TO_LOCAL_PLUGIN = "storm.principal.tolocal";
-    public static final Object STORM_PRINCIPAL_TO_LOCAL_PLUGIN_SCHEMA = String.class;
 
     /**
      * The plugin that will provide user groups service
      */
+    @isString
     public static final String STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN = "storm.group.mapping.service";
-    public static final Object STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN_SCHEMA = String.class;
 
     /**
      * Max no.of seconds group mapping service will cache user groups
      */
+    @isInteger
     public static final String STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS = "storm.group.mapping.service.cache.duration.secs";
-    public static final Object STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS_SCHEMA = Number.class;
 
     /**
      * Initialization parameters for the group mapping service plugin.
      * Provides a way for a @link{STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN}
      * implementation to access optional settings.
      */
+    @isType(type=Map.class)
     public static final String STORM_GROUP_MAPPING_SERVICE_PARAMS = "storm.group.mapping.service.params";
-    public static final Object STORM_GROUP_MAPPING_SERVICE_PARAMS_SCHEMA = Map.class;
 
     /**
      * The default transport plug-in for Thrift client/server communication
      */
+    @isString
     public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport";
-    public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
      * The serializer class for ListDelegate (tuple payload).
      * The default serializer will be ListDelegateSerializer
      */
+    @isString
     public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer";
-    public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class;
+
+    /**
+     * Disable load aware grouping support.
+     */
+    @isBoolean
+    @NotNull
+    public static final String TOPOLOGY_DISABLE_LOADAWARE_MESSAGING = "topology.disable.loadaware.messaging";
 
     /**
      * Try to serialize all tuples, even for local transfers.  This should only be used
      * for testing, as a sanity check that all of your tuples are setup properly.
      */
+    @isBoolean
     public static final String TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE = "topology.testing.always.try.serialize";
-    public static final Object TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE_SCHEMA = Boolean.class;
 
     /**
      * Whether or not to use ZeroMQ for messaging in local mode. If this is set
@@ -231,50 +269,50 @@
      *
      * Defaults to false.
      */
+    @isBoolean
     public static final String STORM_LOCAL_MODE_ZMQ = "storm.local.mode.zmq";
-    public static final Object STORM_LOCAL_MODE_ZMQ_SCHEMA = Boolean.class;
 
     /**
      * The root location at which Storm stores data in ZooKeeper.
      */
+    @isString
     public static final String STORM_ZOOKEEPER_ROOT = "storm.zookeeper.root";
-    public static final Object STORM_ZOOKEEPER_ROOT_SCHEMA = String.class;
 
     /**
      * The session timeout for clients to ZooKeeper.
      */
+    @isInteger
     public static final String STORM_ZOOKEEPER_SESSION_TIMEOUT = "storm.zookeeper.session.timeout";
-    public static final Object STORM_ZOOKEEPER_SESSION_TIMEOUT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The connection timeout for clients to ZooKeeper.
      */
+    @isInteger
     public static final String STORM_ZOOKEEPER_CONNECTION_TIMEOUT = "storm.zookeeper.connection.timeout";
-    public static final Object STORM_ZOOKEEPER_CONNECTION_TIMEOUT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The number of times to retry a Zookeeper operation.
      */
+    @isInteger
     public static final String STORM_ZOOKEEPER_RETRY_TIMES="storm.zookeeper.retry.times";
-    public static final Object STORM_ZOOKEEPER_RETRY_TIMES_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The interval between retries of a Zookeeper operation.
      */
+    @isInteger
     public static final String STORM_ZOOKEEPER_RETRY_INTERVAL="storm.zookeeper.retry.interval";
-    public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The ceiling of the interval between retries of a Zookeeper operation.
      */
+    @isInteger
     public static final String STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING="storm.zookeeper.retry.intervalceiling.millis";
-    public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The cluster Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
      */
+    @isString
     public static final String STORM_ZOOKEEPER_AUTH_SCHEME="storm.zookeeper.auth.scheme";
-    public static final Object STORM_ZOOKEEPER_AUTH_SCHEME_SCHEMA = String.class;
 
     /**
      * A string representing the payload for cluster Zookeeper authentication.
@@ -285,144 +323,177 @@
      * This file storm-cluster-auth.yaml should then be protected with
      * appropriate permissions that deny access from workers.
      */
+    @isString
     public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD="storm.zookeeper.auth.payload";
-    public static final Object STORM_ZOOKEEPER_AUTH_PAYLOAD_SCHEMA = String.class;
 
     /**
      * The topology Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication.
      */
+    @isString
     public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME="storm.zookeeper.topology.auth.scheme";
-    public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME_SCHEMA = String.class;
 
     /**
      * A string representing the payload for topology Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication.
      */
+    @isString
     public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD="storm.zookeeper.topology.auth.payload";
-    public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD_SCHEMA = String.class;
 
     /**
      * The id assigned to a running topology. The id is the storm name with a unique nonce appended.
      */
+    @isString
     public static final String STORM_ID = "storm.id";
-    public static final Object STORM_ID_SCHEMA = String.class;
+
+    /**
+     * The directory where storm's health scripts go.
+     */
+    @isString
+    public static final String STORM_HEALTH_CHECK_DIR = "storm.health.check.dir";
+
+    /**
+     * The time to allow any given healthcheck script to run before it
+     * is marked failed due to timeout
+     */
+    @isNumber
+    public static final String STORM_HEALTH_CHECK_TIMEOUT_MS = "storm.health.check.timeout.ms";
 
     /**
      * The number of times to retry a Nimbus operation.
      */
+    @isNumber
     public static final String STORM_NIMBUS_RETRY_TIMES="storm.nimbus.retry.times";
-    public static final Object STORM_NIMBUS_RETRY_TIMES_SCHEMA = Number.class;
 
     /**
      * The starting interval between exponential backoff retries of a Nimbus operation.
      */
+    @isNumber
     public static final String STORM_NIMBUS_RETRY_INTERVAL="storm.nimbus.retry.interval.millis";
-    public static final Object STORM_NIMBUS_RETRY_INTERVAL_SCHEMA = Number.class;
 
     /**
      * The ceiling of the interval between retries of a client connect to Nimbus operation.
      */
+    @isNumber
     public static final String STORM_NIMBUS_RETRY_INTERVAL_CEILING="storm.nimbus.retry.intervalceiling.millis";
-    public static final Object STORM_NIMBUS_RETRY_INTERVAL_CEILING_SCHEMA = Number.class;
+
+    /**
+     * The ClusterState factory that worker will use to create a ClusterState
+     * to store state in. Defaults to ZooKeeper.
+     */
+    @isString
+    public static final String STORM_CLUSTER_STATE_STORE = "storm.cluster.state.store";
 
     /**
      * The Nimbus transport plug-in for Thrift client/server communication
      */
+    @isString
     public static final String NIMBUS_THRIFT_TRANSPORT_PLUGIN = "nimbus.thrift.transport";
-    public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
      * The host that the master server is running on, added only for backward compatibility,
      * the usage deprecated in favor of nimbus.seeds config.
      */
     @Deprecated
+    @isString
     public static final String NIMBUS_HOST = "nimbus.host";
-    public static final Object NIMBUS_HOST_SCHEMA = String.class;
 
     /**
      * List of seed nimbus hosts to use for leader nimbus discovery.
      */
+    @isStringList
     public static final String NIMBUS_SEEDS = "nimbus.seeds";
-    public static final Object NIMBUS_SEEDS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * Which port the Thrift interface of Nimbus should run on. Clients should
      * connect to this port to upload jars and submit topologies.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_THRIFT_PORT = "nimbus.thrift.port";
-    public static final Object NIMBUS_THRIFT_PORT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The number of threads that should be used by the nimbus thrift server.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_THRIFT_THREADS = "nimbus.thrift.threads";
-    public static final Object NIMBUS_THRIFT_THREADS_SCHEMA = Number.class;
 
     /**
      * A list of users that are cluster admins and can run any command.  To use this set
      * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
      */
+    @isStringList
     public static final String NIMBUS_ADMINS = "nimbus.admins";
-    public static final Object NIMBUS_ADMINS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * A list of users that are the only ones allowed to run user operation on storm cluster.
      * To use this set nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
      */
+    @isStringList
     public static final String NIMBUS_USERS = "nimbus.users";
-    public static final Object NIMBUS_USERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * A list of groups , users belong to these groups are the only ones allowed to run user operation on storm cluster.
      * To use this set nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
      */
+    @isStringList
     public static final String NIMBUS_GROUPS = "nimbus.groups";
-    public static final Object NIMBUS_GROUPS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * A list of users that run the supervisors and should be authorized to interact with
      * nimbus as a supervisor would.  To use this set
      * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
      */
+    @isStringList
     public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users";
-    public static final Object NIMBUS_SUPERVISOR_USERS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
+     * This is the user that the Nimbus daemon process is running as. May be used when security
+     * is enabled to authorize actions in the cluster.
+     */
+    @isString
+    public static final String NIMBUS_DAEMON_USER = "nimbus.daemon.user";
 
     /**
      * The maximum buffer size thrift should use when reading messages.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size";
-    public static final Object NIMBUS_THRIFT_MAX_BUFFER_SIZE_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * This parameter is used by the storm-deploy project to configure the
      * jvm options for the nimbus daemon.
      */
+    @isString
     public static final String NIMBUS_CHILDOPTS = "nimbus.childopts";
-    public static final Object NIMBUS_CHILDOPTS_SCHEMA = String.class;
 
 
     /**
      * How long without heartbeating a task can go before nimbus will consider the
      * task dead and reassign it to another location.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_TASK_TIMEOUT_SECS = "nimbus.task.timeout.secs";
-    public static final Object NIMBUS_TASK_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
 
     /**
      * How often nimbus should wake up to check heartbeats and do reassignments. Note
      * that if a machine ever goes down Nimbus will immediately wake up and take action.
      * This parameter is for checking for failures when there's no explicit event like that
-     * occuring.
+     * occurring.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_MONITOR_FREQ_SECS = "nimbus.monitor.freq.secs";
-    public static final Object NIMBUS_MONITOR_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How often nimbus should wake the cleanup thread to clean the inbox.
-     * @see NIMBUS_INBOX_JAR_EXPIRATION_SECS
+     * @see #NIMBUS_INBOX_JAR_EXPIRATION_SECS
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_CLEANUP_INBOX_FREQ_SECS = "nimbus.cleanup.inbox.freq.secs";
-    public static final Object NIMBUS_CLEANUP_INBOX_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The length of time a jar file lives in the inbox before being deleted by the cleanup thread.
@@ -431,17 +502,18 @@
      * Note that the time it takes to delete an inbox jar file is going to be somewhat more than
      * NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS (depending on how often NIMBUS_CLEANUP_FREQ_SECS
      * is set to).
-     * @see NIMBUS_CLEANUP_FREQ_SECS
+     * @see #NIMBUS_CLEANUP_INBOX_FREQ_SECS
      */
+    @isInteger
     public static final String NIMBUS_INBOX_JAR_EXPIRATION_SECS = "nimbus.inbox.jar.expiration.secs";
-    public static final Object NIMBUS_INBOX_JAR_EXPIRATION_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How long before a supervisor can go without heartbeating before nimbus considers it dead
      * and stops assigning new work to it.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_SUPERVISOR_TIMEOUT_SECS = "nimbus.supervisor.timeout.secs";
-    public static final Object NIMBUS_SUPERVISOR_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * A special timeout used when a task is initially launched. During launch, this is the timeout
@@ -450,369 +522,440 @@
      * <p>A separate timeout exists for launch because there can be quite a bit of overhead
      * to launching new JVM's and configuring them.</p>
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_TASK_LAUNCH_SECS = "nimbus.task.launch.secs";
-    public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = ConfigValidation.IntegerValidator;
-
-    /**
-     * Whether or not nimbus should reassign tasks if it detects that a task goes down.
-     * Defaults to true, and it's not recommended to change this value.
-     */
-    public static final String NIMBUS_REASSIGN = "nimbus.reassign";
-    public static final Object NIMBUS_REASSIGN_SCHEMA = Boolean.class;
 
     /**
      * During upload/download with the master, how long an upload or download connection is idle
      * before nimbus considers it dead and drops the connection.
      */
+    @isInteger
     public static final String NIMBUS_FILE_COPY_EXPIRATION_SECS = "nimbus.file.copy.expiration.secs";
-    public static final Object NIMBUS_FILE_COPY_EXPIRATION_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * A custom class that implements ITopologyValidator that is run whenever a
      * topology is submitted. Can be used to provide business-specific logic for
      * whether topologies are allowed to run or not.
      */
+    @isString
     public static final String NIMBUS_TOPOLOGY_VALIDATOR = "nimbus.topology.validator";
-    public static final Object NIMBUS_TOPOLOGY_VALIDATOR_SCHEMA = String.class;
 
     /**
      * Class name for authorization plugin for Nimbus
      */
+    @isString
     public static final String NIMBUS_AUTHORIZER = "nimbus.authorizer";
-    public static final Object NIMBUS_AUTHORIZER_SCHEMA = String.class;
-
 
     /**
      * Impersonation user ACL config entries.
      */
+    @isString
     public static final String NIMBUS_IMPERSONATION_AUTHORIZER = "nimbus.impersonation.authorizer";
-    public static final Object NIMBUS_IMPERSONATION_AUTHORIZER_SCHEMA = String.class;
-
 
     /**
      * Impersonation user ACL config entries.
      */
+    @isMapEntryCustom(keyValidatorClasses = {StringValidator.class}, valueValidatorClasses = {ImpersonationAclUserEntryValidator.class})
     public static final String NIMBUS_IMPERSONATION_ACL = "nimbus.impersonation.acl";
-    public static final Object NIMBUS_IMPERSONATION_ACL_SCHEMA = ConfigValidation.MapOfStringToMapValidator;
 
     /**
      * How often nimbus should wake up to renew credentials if needed.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String NIMBUS_CREDENTIAL_RENEW_FREQ_SECS = "nimbus.credential.renewers.freq.secs";
-    public static final Object NIMBUS_CREDENTIAL_RENEW_FREQ_SECS_SCHEMA = Number.class;
 
     /**
      * A list of credential renewers that nimbus should load.
      */
+    @isStringList
     public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes";
-    public static final Object NIMBUS_CREDENTIAL_RENEWERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * A list of plugins that nimbus should load during submit topology to populate
      * credentials on user's behalf.
      */
+    @isStringList
     public static final String NIMBUS_AUTO_CRED_PLUGINS = "nimbus.autocredential.plugins.classes";
-    public static final Object NIMBUS_AUTO_CRED_PLUGINS_SCHEMA = ConfigValidation.StringsValidator;
+
+    /**
+     * FQCN of a class that implements {@code ITopologyActionNotifierPlugin} @see backtype.storm.nimbus.ITopologyActionNotifierPlugin for details.
+     */
+    @isString
+    public static final String NIMBUS_TOPOLOGY_ACTION_NOTIFIER_PLUGIN = "nimbus.topology.action.notifier.plugin.class";
 
     /**
      * Storm UI binds to this host/interface.
      */
+    @isString
     public static final String UI_HOST = "ui.host";
-    public static final Object UI_HOST_SCHEMA = String.class;
 
     /**
      * Storm UI binds to this port.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String UI_PORT = "ui.port";
-    public static final Object UI_PORT_SCHEMA = ConfigValidation.IntegerValidator;
+
+    /**
+     * Storm UI Project BUGTRACKER Link for reporting issue.
+     */
+    @isString
+    public static final String UI_PROJECT_BUGTRACKER_URL = "ui.project.bugtracker.url";
+
+    /**
+     * Storm UI Central Logging URL.
+     */
+    @isString
+    public static final String UI_CENTRAL_LOGGING_URL = "ui.central.logging.url";
 
     /**
      * HTTP UI port for log viewer
      */
+    @isInteger
+    @isPositiveNumber
     public static final String LOGVIEWER_PORT = "logviewer.port";
-    public static final Object LOGVIEWER_PORT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Childopts for log viewer java process.
      */
+    @isString
     public static final String LOGVIEWER_CHILDOPTS = "logviewer.childopts";
-    public static final Object LOGVIEWER_CHILDOPTS_SCHEMA = String.class;
 
     /**
      * How often to clean up old log files
      */
+    @isInteger
+    @isPositiveNumber
     public static final String LOGVIEWER_CLEANUP_INTERVAL_SECS = "logviewer.cleanup.interval.secs";
-    public static final Object LOGVIEWER_CLEANUP_INTERVAL_SECS_SCHEMA = ConfigValidation.PositiveIntegerValidator;
 
     /**
      * How many minutes since a log was last modified for the log to be considered for clean-up
      */
+    @isInteger
+    @isPositiveNumber
     public static final String LOGVIEWER_CLEANUP_AGE_MINS = "logviewer.cleanup.age.mins";
-    public static final Object LOGVIEWER_CLEANUP_AGE_MINS_SCHEMA = ConfigValidation.PositiveIntegerValidator;
+
+    /**
+     * The maximum number of bytes all worker log files can take up in MB
+     */
+    @isPositiveNumber
+    public static final String LOGVIEWER_MAX_SUM_WORKER_LOGS_SIZE_MB = "logviewer.max.sum.worker.logs.size.mb";
+
+    /**
+     * The maximum number of bytes per worker's files can take up in MB
+     */
+    @isPositiveNumber
+    public static final String LOGVIEWER_MAX_PER_WORKER_LOGS_SIZE_MB = "logviewer.max.per.worker.logs.size.mb";
 
     /**
      * Storm Logviewer HTTPS port
      */
+    @isInteger
+    @isPositiveNumber
     public static final String LOGVIEWER_HTTPS_PORT = "logviewer.https.port";
-    public static final Object LOGVIEWER_HTTPS_PORT_SCHEMA = Number.class;
 
     /**
      * Path to the keystore containing the certs used by Storm Logviewer for HTTPS communications
      */
+    @isString
     public static final String LOGVIEWER_HTTPS_KEYSTORE_PATH = "logviewer.https.keystore.path";
-    public static final Object LOGVIEWER_HTTPS_KEYSTORE_PATH_SCHEMA = String.class;
 
     /**
      * Password for the keystore for HTTPS for Storm Logviewer
      */
+    @isString
     public static final String LOGVIEWER_HTTPS_KEYSTORE_PASSWORD = "logviewer.https.keystore.password";
-    public static final Object LOGVIEWER_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class;
 
     /**
      * Type of the keystore for HTTPS for Storm Logviewer.
      * see http://docs.oracle.com/javase/8/docs/api/java/security/KeyStore.html for more details.
      */
+    @isString
     public static final String LOGVIEWER_HTTPS_KEYSTORE_TYPE = "logviewer.https.keystore.type";
-    public static final Object LOGVIEWER_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class;
 
     /**
-     * Password to the private key in the keystore for settting up HTTPS (SSL).
+     * Password to the private key in the keystore for setting up HTTPS (SSL).
      */
+    @isString
     public static final String LOGVIEWER_HTTPS_KEY_PASSWORD = "logviewer.https.key.password";
-    public static final Object LOGVIEWER_HTTPS_KEY_PASSWORD_SCHEMA = String.class;
 
     /**
      * Path to the truststore containing the certs used by Storm Logviewer for HTTPS communications
      */
+    @isString
     public static final String LOGVIEWER_HTTPS_TRUSTSTORE_PATH = "logviewer.https.truststore.path";
-    public static final Object LOGVIEWER_HTTPS_TRUSTSTORE_PATH_SCHEMA = String.class;
 
     /**
      * Password for the truststore for HTTPS for Storm Logviewer
      */
+    @isString
     public static final String LOGVIEWER_HTTPS_TRUSTSTORE_PASSWORD = "logviewer.https.truststore.password";
-    public static final Object LOGVIEWER_HTTPS_TRUSTSTORE_PASSWORD_SCHEMA = String.class;
 
     /**
      * Type of the truststore for HTTPS for Storm Logviewer.
      * see http://docs.oracle.com/javase/8/docs/api/java/security/Truststore.html for more details.
      */
+    @isString
     public static final String LOGVIEWER_HTTPS_TRUSTSTORE_TYPE = "logviewer.https.truststore.type";
-    public static final Object LOGVIEWER_HTTPS_TRUSTSTORE_TYPE_SCHEMA = String.class;
 
     /**
-     * Password to the truststore used by Storm Logviewer settting up HTTPS (SSL).
+     * Password to the truststore used by Storm Logviewer setting up HTTPS (SSL).
      */
+    @isBoolean
     public static final String LOGVIEWER_HTTPS_WANT_CLIENT_AUTH = "logviewer.https.want.client.auth";
-    public static final Object LOGVIEWER_HTTPS_WANT_CLIENT_AUTH_SCHEMA = Boolean.class;
 
+    @isBoolean
     public static final String LOGVIEWER_HTTPS_NEED_CLIENT_AUTH = "logviewer.https.need.client.auth";
-    public static final Object LOGVIEWER_HTTPS_NEED_CLIENT_AUTH_SCHEMA = Boolean.class;
 
     /**
      * A list of users allowed to view logs via the Log Viewer
      */
+    @isStringList
     public static final String LOGS_USERS = "logs.users";
-    public static final Object LOGS_USERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * A list of groups allowed to view logs via the Log Viewer
      */
+    @isStringList
     public static final String LOGS_GROUPS = "logs.groups";
-    public static final Object LOGS_GROUPS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * Appender name used by log viewer to determine log directory.
      */
+    @isString
     public static final String LOGVIEWER_APPENDER_NAME = "logviewer.appender.name";
-    public static final Object LOGVIEWER_APPENDER_NAME_SCHEMA = String.class;
 
     /**
      * Childopts for Storm UI Java process.
      */
+    @isString
     public static final String UI_CHILDOPTS = "ui.childopts";
-    public static final Object UI_CHILDOPTS_SCHEMA = String.class;
 
     /**
      * A class implementing javax.servlet.Filter for authenticating/filtering UI requests
      */
+    @isString
     public static final String UI_FILTER = "ui.filter";
-    public static final Object UI_FILTER_SCHEMA = String.class;
 
     /**
      * Initialization parameters for the javax.servlet.Filter
      */
+    @isMapEntryType(keyType = String.class, valueType = String.class)
     public static final String UI_FILTER_PARAMS = "ui.filter.params";
-    public static final Object UI_FILTER_PARAMS_SCHEMA = Map.class;
 
     /**
      * The size of the header buffer for the UI in bytes
      */
+    @isInteger
+    @isPositiveNumber
     public static final String UI_HEADER_BUFFER_BYTES = "ui.header.buffer.bytes";
-    public static final Object UI_HEADER_BUFFER_BYTES_SCHEMA = Number.class;
 
     /**
      * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String UI_HTTPS_PORT = "ui.https.port";
-    public static final Object UI_HTTPS_PORT_SCHEMA = Number.class;
 
     /**
      * Path to the keystore used by Storm UI for setting up HTTPS (SSL).
      */
+    @isString
     public static final String UI_HTTPS_KEYSTORE_PATH = "ui.https.keystore.path";
-    public static final Object UI_HTTPS_KEYSTORE_PATH_SCHEMA = String.class;
 
     /**
      * Password to the keystore used by Storm UI for setting up HTTPS (SSL).
      */
+    @isString
     public static final String UI_HTTPS_KEYSTORE_PASSWORD = "ui.https.keystore.password";
-    public static final Object UI_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class;
 
     /**
      * Type of keystore used by Storm UI for setting up HTTPS (SSL).
      * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
      */
+    @isString
     public static final String UI_HTTPS_KEYSTORE_TYPE = "ui.https.keystore.type";
-    public static final Object UI_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class;
 
     /**
-     * Password to the private key in the keystore for settting up HTTPS (SSL).
+     * Password to the private key in the keystore for setting up HTTPS (SSL).
      */
+    @isString
     public static final String UI_HTTPS_KEY_PASSWORD = "ui.https.key.password";
-    public static final Object UI_HTTPS_KEY_PASSWORD_SCHEMA = String.class;
 
     /**
-     * Path to the truststore used by Storm UI settting up HTTPS (SSL).
+     * Path to the truststore used by Storm UI setting up HTTPS (SSL).
      */
+    @isString
     public static final String UI_HTTPS_TRUSTSTORE_PATH = "ui.https.truststore.path";
-    public static final Object UI_HTTPS_TRUSTSTORE_PATH_SCHEMA = String.class;
 
     /**
-     * Password to the truststore used by Storm UI settting up HTTPS (SSL).
+     * Password to the truststore used by Storm UI setting up HTTPS (SSL).
      */
+    @isString
     public static final String UI_HTTPS_TRUSTSTORE_PASSWORD = "ui.https.truststore.password";
-    public static final Object UI_HTTPS_TRUSTSTORE_PASSWORD_SCHEMA = String.class;
 
     /**
      * Type of truststore used by Storm UI for setting up HTTPS (SSL).
      * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
      */
+    @isString
     public static final String UI_HTTPS_TRUSTSTORE_TYPE = "ui.https.truststore.type";
-    public static final Object UI_HTTPS_TRUSTSTORE_TYPE_SCHEMA = String.class;
 
     /**
-     * Password to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     * Password to the truststore used by Storm DRPC setting up HTTPS (SSL).
      */
+    @isBoolean
     public static final String UI_HTTPS_WANT_CLIENT_AUTH = "ui.https.want.client.auth";
-    public static final Object UI_HTTPS_WANT_CLIENT_AUTH_SCHEMA = Boolean.class;
 
+    @isBoolean
     public static final String UI_HTTPS_NEED_CLIENT_AUTH = "ui.https.need.client.auth";
-    public static final Object UI_HTTPS_NEED_CLIENT_AUTH_SCHEMA = Boolean.class;
 
+    /**
+     * The host that Pacemaker is running on.
+     */
+    @isString
+    public static final String PACEMAKER_HOST = "pacemaker.host";
 
     /**
+     * The port Pacemaker should run on. Clients should
+     * connect to this port to submit or read heartbeats.
+     */
+    @isNumber
+    @isPositiveNumber
+    public static final String PACEMAKER_PORT = "pacemaker.port";
+
+    /**
+     * The maximum number of threads that should be used by the Pacemaker.
+     * When Pacemaker gets loaded it will spawn new threads, up to 
+     * this many total, to handle the load.
+     */
+    @isNumber
+    @isPositiveNumber
+    public static final String PACEMAKER_MAX_THREADS = "pacemaker.max.threads";
+
+    /**
+     * This parameter is used by the storm-deploy project to configure the
+     * jvm options for the pacemaker daemon.
+     */
+    @isString
+    public static final String PACEMAKER_CHILDOPTS = "pacemaker.childopts";
+
+    /**
+     * This should be one of "DIGEST", "KERBEROS", or "NONE"
+     * Determines the mode of authentication the pacemaker server and client use.
+     * The client must either match the server, or be NONE. In the case of NONE,
+     * no authentication is performed for the client, and if the server is running with
+     * DIGEST or KERBEROS, the client can only write to the server (no reads).
+     * This is intended to provide a primitive form of access-control.
+     */
+    @CustomValidator(validatorClass=PacemakerAuthTypeValidator.class)
+    public static final String PACEMAKER_AUTH_METHOD = "pacemaker.auth.method";
+    
+    /**
      * List of DRPC servers so that the DRPCSpout knows who to talk to.
      */
+    @isStringList
     public static final String DRPC_SERVERS = "drpc.servers";
-    public static final Object DRPC_SERVERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * This port is used by Storm DRPC for receiving HTTP DPRC requests from clients.
      */
+    @isInteger
     public static final String DRPC_HTTP_PORT = "drpc.http.port";
-    public static final Object DRPC_HTTP_PORT_SCHEMA = Number.class;
 
     /**
      * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients.
      */
+    @isInteger
     public static final String DRPC_HTTPS_PORT = "drpc.https.port";
-    public static final Object DRPC_HTTPS_PORT_SCHEMA = Number.class;
 
     /**
      * Path to the keystore used by Storm DRPC for setting up HTTPS (SSL).
      */
+    @isString
     public static final String DRPC_HTTPS_KEYSTORE_PATH = "drpc.https.keystore.path";
-    public static final Object DRPC_HTTPS_KEYSTORE_PATH_SCHEMA = String.class;
 
     /**
      * Password to the keystore used by Storm DRPC for setting up HTTPS (SSL).
      */
+    @isString
     public static final String DRPC_HTTPS_KEYSTORE_PASSWORD = "drpc.https.keystore.password";
-    public static final Object DRPC_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class;
 
     /**
      * Type of keystore used by Storm DRPC for setting up HTTPS (SSL).
      * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
      */
+    @isString
     public static final String DRPC_HTTPS_KEYSTORE_TYPE = "drpc.https.keystore.type";
-    public static final Object DRPC_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class;
 
     /**
-     * Password to the private key in the keystore for settting up HTTPS (SSL).
+     * Password to the private key in the keystore for setting up HTTPS (SSL).
      */
+    @isString
     public static final String DRPC_HTTPS_KEY_PASSWORD = "drpc.https.key.password";
-    public static final Object DRPC_HTTPS_KEY_PASSWORD_SCHEMA = String.class;
 
     /**
-     * Path to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     * Path to the truststore used by Storm DRPC setting up HTTPS (SSL).
      */
+    @isString
     public static final String DRPC_HTTPS_TRUSTSTORE_PATH = "drpc.https.truststore.path";
-    public static final Object DRPC_HTTPS_TRUSTSTORE_PATH_SCHEMA = String.class;
 
     /**
-     * Password to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     * Password to the truststore used by Storm DRPC setting up HTTPS (SSL).
      */
+    @isString
     public static final String DRPC_HTTPS_TRUSTSTORE_PASSWORD = "drpc.https.truststore.password";
-    public static final Object DRPC_HTTPS_TRUSTSTORE_PASSWORD_SCHEMA = String.class;
 
     /**
      * Type of truststore used by Storm DRPC for setting up HTTPS (SSL).
      * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details.
      */
+    @isString
     public static final String DRPC_HTTPS_TRUSTSTORE_TYPE = "drpc.https.truststore.type";
-    public static final Object DRPC_HTTPS_TRUSTSTORE_TYPE_SCHEMA = String.class;
 
     /**
-     * Password to the truststore used by Storm DRPC settting up HTTPS (SSL).
+     * Password to the truststore used by Storm DRPC setting up HTTPS (SSL).
      */
+    @isBoolean
     public static final String DRPC_HTTPS_WANT_CLIENT_AUTH = "drpc.https.want.client.auth";
-    public static final Object DRPC_HTTPS_WANT_CLIENT_AUTH_SCHEMA = Boolean.class;
 
+    @isBoolean
     public static final String DRPC_HTTPS_NEED_CLIENT_AUTH = "drpc.https.need.client.auth";
-    public static final Object DRPC_HTTPS_NEED_CLIENT_AUTH_SCHEMA = Boolean.class;
 
     /**
      * The DRPC transport plug-in for Thrift client/server communication
      */
+    @isString
     public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport";
-    public static final Object DRPC_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
      * This port is used by Storm DRPC for receiving DPRC requests from clients.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String DRPC_PORT = "drpc.port";
-    public static final Object DRPC_PORT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Class name for authorization plugin for DRPC client
      */
+    @isString
     public static final String DRPC_AUTHORIZER = "drpc.authorizer";
-    public static final Object DRPC_AUTHORIZER_SCHEMA = String.class;
 
     /**
      * The Access Control List for the DRPC Authorizer.
-     * @see DRPCSimpleAclAuthorizer
+     * @see backtype.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
      */
+    @isType(type=Map.class)
     public static final String DRPC_AUTHORIZER_ACL = "drpc.authorizer.acl";
-    public static final Object DRPC_AUTHORIZER_ACL_SCHEMA = Map.class;
 
     /**
      * File name of the DRPC Authorizer ACL.
-     * @see DRPCSimpleAclAuthorizer
+     * @see backtype.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
      */
+    @isString
     public static final String DRPC_AUTHORIZER_ACL_FILENAME = "drpc.authorizer.acl.filename";
-    public static final Object DRPC_AUTHORIZER_ACL_FILENAME_SCHEMA = String.class;
 
     /**
      * Whether the DRPCSimpleAclAuthorizer should deny requests for operations
@@ -821,130 +964,146 @@
      * permitted, which is appropriate for a development environment. When set
      * to true, explicit ACL entries are required for every DRPC function, and
      * any request for functions will be denied.
-     * @see DRPCSimpleAclAuthorizer
+     * @see backtype.storm.security.auth.authorizer.DRPCSimpleACLAuthorizer
      */
+    @isBoolean
     public static final String DRPC_AUTHORIZER_ACL_STRICT = "drpc.authorizer.acl.strict";
-    public static final Object DRPC_AUTHORIZER_ACL_STRICT_SCHEMA = Boolean.class;
 
     /**
      * DRPC thrift server worker threads
      */
+    @isInteger
+    @isPositiveNumber
     public static final String DRPC_WORKER_THREADS = "drpc.worker.threads";
-    public static final Object DRPC_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The maximum buffer size thrift should use when reading messages for DRPC.
      */
+    @isNumber
+    @isPositiveNumber
     public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size";
-    public static final Object DRPC_MAX_BUFFER_SIZE_SCHEMA = Number.class;
 
     /**
      * DRPC thrift server queue size
      */
+    @isInteger
+    @isPositiveNumber
     public static final String DRPC_QUEUE_SIZE = "drpc.queue.size";
-    public static final Object DRPC_QUEUE_SIZE_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The DRPC invocations transport plug-in for Thrift client/server communication
      */
+    @isString
     public static final String DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN = "drpc.invocations.thrift.transport";
-    public static final Object DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class;
 
     /**
      * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port";
-    public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * DRPC invocations thrift server worker threads
      */
+    @isInteger
+    @isPositiveNumber
     public static final String DRPC_INVOCATIONS_THREADS = "drpc.invocations.threads";
-    public static final Object DRPC_INVOCATIONS_THREADS_SCHEMA = Number.class;
 
     /**
      * The timeout on DRPC requests within the DRPC server. Defaults to 10 minutes. Note that requests can also
      * timeout based on the socket timeout on the DRPC client, and separately based on the topology message
      * timeout for the topology implementing the DRPC function.
      */
+
+    @isInteger
+    @isPositiveNumber
+    @NotNull
     public static final String DRPC_REQUEST_TIMEOUT_SECS  = "drpc.request.timeout.secs";
-    public static final Object DRPC_REQUEST_TIMEOUT_SECS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator;
 
     /**
      * Childopts for Storm DRPC Java process.
      */
+    @isString
     public static final String DRPC_CHILDOPTS = "drpc.childopts";
-    public static final Object DRPC_CHILDOPTS_SCHEMA = String.class;
 
     /**
      * Class name of the HTTP credentials plugin for the UI.
      */
+    @isString
     public static final String UI_HTTP_CREDS_PLUGIN = "ui.http.creds.plugin";
-    public static final Object UI_HTTP_CREDS_PLUGIN_SCHEMA = String.class;
 
     /**
      * Class name of the HTTP credentials plugin for DRPC.
      */
+    @isString
     public static final String DRPC_HTTP_CREDS_PLUGIN = "drpc.http.creds.plugin";
-    public static final Object DRPC_HTTP_CREDS_PLUGIN_SCHEMA = String.class;
 
     /**
      * the metadata configured on the supervisor
      */
+    @isType(type=Map.class)
     public static final String SUPERVISOR_SCHEDULER_META = "supervisor.scheduler.meta";
-    public static final Object SUPERVISOR_SCHEDULER_META_SCHEMA = Map.class;
+
     /**
      * A list of ports that can run workers on this supervisor. Each worker uses one port, and
      * the supervisor will only run one worker per port. Use this configuration to tune
      * how many workers run on each machine.
      */
+    @isNoDuplicateInList
+    @NotNull
+    @isListEntryCustom(entryValidatorClasses={IntegerValidator.class,PositiveNumberValidator.class})
     public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
-    public static final Object SUPERVISOR_SLOTS_PORTS_SCHEMA = ConfigValidation.NoDuplicateIntegersValidator;
 
     /**
      * A number representing the maximum number of workers any single topology can acquire.
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String NIMBUS_SLOTS_PER_TOPOLOGY = "nimbus.slots.perTopology";
-    public static final Object NIMBUS_SLOTS_PER_TOPOLOGY_SCHEMA = Number.class;
 
     /**
      * A class implementing javax.servlet.Filter for DRPC HTTP requests
      */
+    @isString
     public static final String DRPC_HTTP_FILTER = "drpc.http.filter";
-    public static final Object DRPC_HTTP_FILTER_SCHEMA = String.class;
 
     /**
      * Initialization parameters for the javax.servlet.Filter of the DRPC HTTP
      * service
      */
+    @isMapEntryType(keyType = String.class, valueType = String.class)
     public static final String DRPC_HTTP_FILTER_PARAMS = "drpc.http.filter.params";
-    public static final Object DRPC_HTTP_FILTER_PARAMS_SCHEMA = Map.class;
 
     /**
      * A number representing the maximum number of executors any single topology can acquire.
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String NIMBUS_EXECUTORS_PER_TOPOLOGY = "nimbus.executors.perTopology";
-    public static final Object NIMBUS_EXECUTORS_PER_TOPOLOGY_SCHEMA = Number.class;
 
     /**
      * This parameter is used by the storm-deploy project to configure the
      * jvm options for the supervisor daemon.
      */
+    @isString
     public static final String SUPERVISOR_CHILDOPTS = "supervisor.childopts";
-    public static final Object SUPERVISOR_CHILDOPTS_SCHEMA = String.class;
 
     /**
      * How long a worker can go without heartbeating before the supervisor tries to
      * restart the worker process.
      */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
     public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
-    public static final Object SUPERVISOR_WORKER_TIMEOUT_SECS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator;
 
     /**
      * How many seconds to sleep for before shutting down threads on worker
      */
+    @isInteger
+    @isPositiveNumber
     public static final String SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS = "supervisor.worker.shutdown.sleep.secs";
-    public static final Object SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How long a worker can go without heartbeating during the initial launch before
@@ -952,81 +1111,125 @@
      * supervisor.worker.timeout.secs during launch because there is additional
      * overhead to starting and configuring the JVM on launch.
      */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
     public static final String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs";
-    public static final Object SUPERVISOR_WORKER_START_TIMEOUT_SECS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator;
 
     /**
      * Whether or not the supervisor should launch workers assigned to it. Defaults
      * to true -- and you should probably never change this value. This configuration
      * is used in the Storm unit tests.
      */
+    @isBoolean
     public static final String SUPERVISOR_ENABLE = "supervisor.enable";
-    public static final Object SUPERVISOR_ENABLE_SCHEMA = Boolean.class;
 
     /**
      * how often the supervisor sends a heartbeat to the master.
      */
+    @isInteger
     public static final String SUPERVISOR_HEARTBEAT_FREQUENCY_SECS = "supervisor.heartbeat.frequency.secs";
-    public static final Object SUPERVISOR_HEARTBEAT_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
 
     /**
      * How often the supervisor checks the worker heartbeats to see if any of them
      * need to be restarted.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String SUPERVISOR_MONITOR_FREQUENCY_SECS = "supervisor.monitor.frequency.secs";
-    public static final Object SUPERVISOR_MONITOR_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Should the supervior try to run the worker as the lauching user or not.  Defaults to false.
      */
+    @isBoolean
     public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user";
-    public static final Object SUPERVISOR_RUN_WORKER_AS_USER_SCHEMA = Boolean.class;
 
     /**
      * Full path to the worker-laucher executable that will be used to lauch workers when
      * SUPERVISOR_RUN_WORKER_AS_USER is set to true.
      */
+    @isString
     public static final String SUPERVISOR_WORKER_LAUNCHER = "supervisor.worker.launcher";
-    public static final Object SUPERVISOR_WORKER_LAUNCHER_SCHEMA = String.class;
 
     /**
-     * The jvm opts provided to workers launched by this supervisor. All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%"
-     * and "%WORKER-PORT%" substrings are replaced with:
+     * 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
+     */
+    @isPositiveNumber
+    public static final String SUPERVISOR_MEMORY_CAPACITY_MB = "supervisor.memory.capacity.mb";
+
+    /**
+     * 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
+     */
+    @isPositiveNumber
+    public static final String SUPERVISOR_CPU_CAPACITY = "supervisor.cpu.capacity";
+
+    /**
+     * The jvm opts provided to workers launched by this supervisor.
+     * All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%",
+     * "%WORKER-PORT%" and "%HEAP-MEM%" substrings are replaced with:
      * %ID%          -> port (for backward compatibility),
      * %WORKER-ID%   -> worker-id,
      * %TOPOLOGY-ID%    -> topology-id,
      * %WORKER-PORT% -> port.
+     * %HEAP-MEM% -> mem-onheap.
      */
+    @isStringOrStringList
     public static final String WORKER_CHILDOPTS = "worker.childopts";
-    public static final Object WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
+
+    /**
+     * The default heap memory size in MB per worker, used in the jvm -Xmx opts for launching the worker
+      */
+    @isInteger
+    @isPositiveNumber
+    public static final String WORKER_HEAP_MEMORY_MB = "worker.heap.memory.mb";
+
+    /**
+     * The jvm profiler opts provided to workers launched by this supervisor.
+     */
+    @isStringOrStringList
+    public static final String WORKER_PROFILER_CHILDOPTS = "worker.profiler.childopts";
+
+    /**
+     * This configuration would enable or disable component page profiing and debugging for workers.
+     */
+    @isBoolean
+    public static final String WORKER_PROFILER_ENABLED = "worker.profiler.enabled";
+
+    /**
+     * The command launched supervisor with worker arguments
+     * pid, action and [target_directory]
+     * Where action is - start profile, stop profile, jstack, heapdump and kill against pid
+     *
+     */
+    @isString
+    public static final String WORKER_PROFILER_COMMAND = "worker.profiler.command";
 
     /**
      * The jvm opts provided to workers launched by this supervisor for GC. All "%ID%" substrings are replaced
      * with an identifier for this worker.  Because the JVM complains about multiple GC opts the topology
      * can override this default value by setting topology.worker.gc.childopts.
      */
+    @isStringOrStringList
     public static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts";
-    public static final Object WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
-
-    /**
-     * control how many worker receiver threads we need per worker
-     */
-    public static final String WORKER_RECEIVER_THREAD_COUNT = "topology.worker.receiver.thread.count";
-    public static final Object WORKER_RECEIVER_THREAD_COUNT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How often this worker should heartbeat to the supervisor.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs";
-    public static final Object WORKER_HEARTBEAT_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How often a task should heartbeat its status to the master.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs";
-    public static final Object TASK_HEARTBEAT_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator;
-
 
     /**
      * How often a task should sync its connections with other tasks (if a task is
@@ -1035,22 +1238,31 @@
      * almost immediately. This configuration is here just in case that notification doesn't
      * come through.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TASK_REFRESH_POLL_SECS = "task.refresh.poll.secs";
-    public static final Object TASK_REFRESH_POLL_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
+    /**
+     * How often a worker should check dynamic log level timeouts for expiration.
+     * For expired logger settings, the clean up polling task will reset the log levels
+     * to the original levels (detected at startup), and will clean up the timeout map
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String WORKER_LOG_LEVEL_RESET_POLL_SECS = "worker.log.level.reset.poll.secs";
 
     /**
      * How often a task should sync credentials, worst case.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TASK_CREDENTIALS_POLL_SECS = "task.credentials.poll.secs";
-    public static final Object TASK_CREDENTIALS_POLL_SECS_SCHEMA = Number.class;
-
 
     /**
      * Whether to enable backpressure in for a certain topology
      */
+    @isBoolean
     public static final String TOPOLOGY_BACKPRESSURE_ENABLE = "topology.backpressure.enable";
-    public static final Object TOPOLOGY_BACKPRESSURE_ENABLE_SCHEMA = Boolean.class;
 
     /**
      * This signifies the tuple congestion in a disruptor queue.
@@ -1058,50 +1270,50 @@
      * the backpressure scheme, if enabled, should slow down the tuple sending speed of
      * the spouts until reaching the low watermark.
      */
+    @isPositiveNumber
     public static final String BACKPRESSURE_DISRUPTOR_HIGH_WATERMARK="backpressure.disruptor.high.watermark";
-    public static final Object BACKPRESSURE_DISRUPTOR_HIGH_WATERMARK_SCHEMA =ConfigValidation.PositiveNumberValidator;
 
     /**
      * This signifies a state that a disruptor queue has left the congestion.
      * If the used ratio of a disruptor queue is lower than the low watermark,
      * it will unset the backpressure flag.
      */
+    @isPositiveNumber
     public static final String BACKPRESSURE_DISRUPTOR_LOW_WATERMARK="backpressure.disruptor.low.watermark";
-    public static final Object BACKPRESSURE_DISRUPTOR_LOW_WATERMARK_SCHEMA =ConfigValidation.PositiveNumberValidator;
 
     /**
      * A list of users that are allowed to interact with the topology.  To use this set
      * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
      */
+    @isStringList
     public static final String TOPOLOGY_USERS = "topology.users";
-    public static final Object TOPOLOGY_USERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * A list of groups that are allowed to interact with the topology.  To use this set
      * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer
      */
+    @isStringList
     public static final String TOPOLOGY_GROUPS = "topology.groups";
-    public static final Object TOPOLOGY_GROUPS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * True if Storm should timeout messages or not. Defaults to true. This is meant to be used
      * in unit tests to prevent tuples from being accidentally timed out during the test.
      */
+    @isBoolean
     public static final String TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS = "topology.enable.message.timeouts";
-    public static final Object TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS_SCHEMA = Boolean.class;
 
     /**
      * When set to true, Storm will log every message that's emitted.
      */
+    @isBoolean
     public static final String TOPOLOGY_DEBUG = "topology.debug";
-    public static final Object TOPOLOGY_DEBUG_SCHEMA = Boolean.class;
 
     /**
      * The serializer for communication between shell components and non-JVM
      * processes
      */
+    @isString
     public static final String TOPOLOGY_MULTILANG_SERIALIZER = "topology.multilang.serializer";
-    public static final Object TOPOLOGY_MULTILANG_SERIALIZER_SCHEMA = String.class;
 
     /**
      * How many processes should be spawned around the cluster to execute this
@@ -1109,8 +1321,9 @@
      * them. This parameter should be used in conjunction with the parallelism hints
      * on each component in the topology to tune the performance of a topology.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TOPOLOGY_WORKERS = "topology.workers";
-    public static final Object TOPOLOGY_WORKERS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How many instances to create for a spout/bolt. A task runs on a thread with zero or more
@@ -1120,8 +1333,37 @@
      * without redeploying the topology or violating the constraints of Storm (such as a fields grouping
      * guaranteeing that the same value goes to the same task).
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String TOPOLOGY_TASKS = "topology.tasks";
-    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
+     */
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_COMPONENT_RESOURCES_ONHEAP_MEMORY_MB = "topology.component.resources.onheap.memory.mb";
+
+    /**
+     * 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
+     */
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_COMPONENT_RESOURCES_OFFHEAP_MEMORY_MB = "topology.component.resources.offheap.memory.mb";
+
+    /**
+     * 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
+     */
+    @isPositiveNumber(includeZero = true)
+    public static final String TOPOLOGY_COMPONENT_CPU_PCORE_PERCENT = "topology.component.cpu.pcore.percent";
+
+    /**
+     * A per topology config that specifies the maximum amount of memory a worker can use for that specific topology
+     */
+    @isPositiveNumber
+    public static final String TOPOLOGY_WORKER_MAX_HEAP_SIZE_MB = "topology.worker.max.heap.size.mb";
 
     /**
      * How many executors to spawn for ackers.
@@ -1130,8 +1372,9 @@
      * to be equal to the number of workers configured for this topology. If this variable is set to 0,
      * then Storm will immediately ack tuples as soon as they come off the spout, effectively disabling reliability.</p>
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String TOPOLOGY_ACKER_EXECUTORS = "topology.acker.executors";
-    public static final Object TOPOLOGY_ACKER_EXECUTORS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How many executors to spawn for event logger.
@@ -1140,8 +1383,9 @@
      * to be equal to the number of workers configured for this topology. If this variable is set to 0,
      * event logging will be disabled.</p>
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String TOPOLOGY_EVENTLOGGER_EXECUTORS = "topology.eventlogger.executors";
-    public static final Object TOPOLOGY_EVENTLOGGER_EXECUTORS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The maximum amount of time given to the topology to fully process a message
@@ -1149,8 +1393,10 @@
      * will fail the message on the spout. Some spouts implementations will then replay
      * the message at a later time.
      */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
     public static final String TOPOLOGY_MESSAGE_TIMEOUT_SECS = "topology.message.timeout.secs";
-    public static final Object TOPOLOGY_MESSAGE_TIMEOUT_SECS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator;
 
     /**
      * A list of serialization registrations for Kryo ( http://code.google.com/p/kryo/ ),
@@ -1160,8 +1406,8 @@
      *
      * See Kryo's documentation for more information about writing custom serializers.
      */
+    @isKryoReg
     public static final String TOPOLOGY_KRYO_REGISTER = "topology.kryo.register";
-    public static final Object TOPOLOGY_KRYO_REGISTER_SCHEMA = ConfigValidation.KryoRegValidator;
 
     /**
      * A list of classes that customize storm's kryo instance during start-up.
@@ -1169,17 +1415,16 @@
      * listed class is instantiated with 0 arguments, then its 'decorate' method
      * is called with storm's kryo instance as the only argument.
      */
+    @isStringList
     public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators";
-    public static final Object TOPOLOGY_KRYO_DECORATORS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * Class that specifies how to create a Kryo instance for serialization. Storm will then apply
      * topology.kryo.register and topology.kryo.decorators on top of this. The default implementation
      * implements topology.fall.back.on.java.serialization and turns references off.
      */
+    @isString
     public static final String TOPOLOGY_KRYO_FACTORY = "topology.kryo.factory";
-    public static final Object TOPOLOGY_KRYO_FACTORY_SCHEMA = String.class;
-
 
     /**
      * Whether or not Storm should skip the loading of kryo registrations for which it
@@ -1191,36 +1436,37 @@
      * By setting this config to true, Storm will ignore that it doesn't have those other serializations
      * rather than throw an error.
      */
+    @isBoolean
     public static final String TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS= "topology.skip.missing.kryo.registrations";
-    public static final Object TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS_SCHEMA = Boolean.class;
 
     /**
      * A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format).
      * Each listed class will be routed all the metrics data generated by the storm metrics API.
      * Each listed class maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable.
      */
+
+    @isListEntryCustom(entryValidatorClasses={MetricRegistryValidator.class})
     public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register";
-    public static final Object TOPOLOGY_METRICS_CONSUMER_REGISTER_SCHEMA = ConfigValidation.MapsValidator;
 
     /**
      * A map of metric name to class name implementing IMetric that will be created once per worker JVM
      */
+    @isMapEntryType(keyType = String.class, valueType = String.class)
     public static final String TOPOLOGY_WORKER_METRICS = "topology.worker.metrics";
-    public static final Object TOPOLOGY_WORKER_METRICS_SCHEMA = Map.class;
 
     /**
      * A map of metric name to class name implementing IMetric that will be created once per worker JVM
      */
+    @isMapEntryType(keyType = String.class, valueType = String.class)
     public static final String WORKER_METRICS = "worker.metrics";
-    public static final Object WORKER_METRICS_SCHEMA = Map.class;
 
     /**
      * The maximum parallelism allowed for a component in this topology. This configuration is
      * typically used in testing to limit the number of threads spawned in local mode.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TOPOLOGY_MAX_TASK_PARALLELISM="topology.max.task.parallelism";
-    public static final Object TOPOLOGY_MAX_TASK_PARALLELISM_SCHEMA = ConfigValidation.IntegerValidator;
-
 
     /**
      * The maximum number of tuples that can be pending on a spout task at any given time.
@@ -1230,8 +1476,9 @@
      * Note that this config parameter has no effect for unreliable spouts that don't tag
      * their tuples with a message id.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending";
-    public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * A class that implements a strategy for what to do when a spout needs to wait. Waiting is
@@ -1240,70 +1487,73 @@
      * 1. nextTuple emits no tuples
      * 2. The spout has hit maxSpoutPending and can't emit any more tuples
      */
+    @isString
     public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy";
-    public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class;
 
     /**
      * The amount of milliseconds the SleepEmptyEmitStrategy should sleep for.
      */
+    @isInteger
+    @isPositiveNumber(includeZero = true)
     public static final String TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS="topology.sleep.spout.wait.strategy.time.ms";
-    public static final Object TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The maximum amount of time a component gives a source of state to synchronize before it requests
      * synchronization again.
      */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
     public static final String TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS="topology.state.synchronization.timeout.secs";
-    public static final Object TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator;
 
     /**
      * The percentage of tuples to sample to produce stats for a task.
      */
+    @isPositiveNumber
     public static final String TOPOLOGY_STATS_SAMPLE_RATE="topology.stats.sample.rate";
-    public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA =ConfigValidation.PositiveNumberValidator;
 
     /**
      * The time period that builtin metrics data in bucketed into.
      */
+    @isInteger
     public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs";
-    public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Whether or not to use Java serialization in a topology.
      */
+    @isBoolean
     public static final String TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION="topology.fall.back.on.java.serialization";
-    public static final Object TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION_SCHEMA = Boolean.class;
 
     /**
      * Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS.
      */
+    @isStringOrStringList
     public static final String TOPOLOGY_WORKER_CHILDOPTS="topology.worker.childopts";
-    public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
 
     /**
      * Topology-specific options GC for the worker child process. This overrides WORKER_GC_CHILDOPTS.
      */
+    @isStringOrStringList
     public static final String TOPOLOGY_WORKER_GC_CHILDOPTS="topology.worker.gc.childopts";
-    public static final Object TOPOLOGY_WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
 
     /**
      * Topology-specific options for the logwriter process of a worker.
      */
+    @isStringOrStringList
     public static final String TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS="topology.worker.logwriter.childopts";
-    public static final Object TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator;
 
     /**
      * Topology-specific classpath for the worker child process. This is combined to the usual classpath.
      */
+    @isStringOrStringList
     public static final String TOPOLOGY_CLASSPATH="topology.classpath";
-    public static final Object TOPOLOGY_CLASSPATH_SCHEMA = ConfigValidation.StringOrStringListValidator;
 
     /**
      * Topology-specific environment variables for the worker child process.
      * This is added to the existing environment (that of the supervisor)
      */
-     public static final String TOPOLOGY_ENVIRONMENT="topology.environment";
-     public static final Object TOPOLOGY_ENVIRONMENT_SCHEMA = Map.class;
+    @isMapEntryType(keyType = String.class, valueType = String.class)
+    public static final String TOPOLOGY_ENVIRONMENT="topology.environment";
 
     /*
      * Topology-specific option to disable/enable bolt's outgoing overflow buffer.
@@ -1312,122 +1562,162 @@
      * The overflow buffer can fill degrading the performance gradually,
      * eventually running out of memory.
      */
+    @isBoolean
     public static final String TOPOLOGY_BOLTS_OUTGOING_OVERFLOW_BUFFER_ENABLE="topology.bolts.outgoing.overflow.buffer.enable";
-    public static final Object TOPOLOGY_BOLTS_OUTGOING_OVERFLOW_BUFFER_ENABLE_SCHEMA = Boolean.class;
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the window length as a count of number of tuples
+     * in the window.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT = "topology.bolts.window.length.count";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specify the window length in time duration.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS = "topology.bolts.window.length.duration.ms";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specifiy the sliding interval as a count of number of tuples.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT = "topology.bolts.window.sliding.interval.count";
+
+    /*
+     * Bolt-specific configuration for windowed bolts to specifiy the sliding interval in time duration.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS = "topology.bolts.window.sliding.interval.duration.ms";
 
     /**
      * This config is available for TransactionalSpouts, and contains the id ( a String) for
      * the transactional topology. This id is used to store the state of the transactional
      * topology in Zookeeper.
      */
+    @isString
     public static final String TOPOLOGY_TRANSACTIONAL_ID="topology.transactional.id";
-    public static final Object TOPOLOGY_TRANSACTIONAL_ID_SCHEMA = String.class;
 
     /**
      * A list of task hooks that are automatically added to every spout and bolt in the topology. An example
      * of when you'd do this is to add a hook that integrates with your internal
      * monitoring system. These hooks are instantiated using the zero-arg constructor.
      */
+    @isStringList
     public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks";
-    public static final Object TOPOLOGY_AUTO_TASK_HOOKS_SCHEMA = ConfigValidation.StringsValidator;
-
 
     /**
      * The size of the Disruptor receive queue for each executor. Must be a power of 2.
      */
+    @isPowerOf2
     public static final String TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE="topology.executor.receive.buffer.size";
-    public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator;
 
     /**
      * The size of the Disruptor send queue for each executor. Must be a power of 2.
      */
+    @isPowerOf2
     public static final String TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE="topology.executor.send.buffer.size";
-    public static final Object TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator;
 
     /**
      * The size of the Disruptor transfer queue for each worker.
      */
+    @isInteger
+    @isPowerOf2
     public static final String TOPOLOGY_TRANSFER_BUFFER_SIZE="topology.transfer.buffer.size";
-    public static final Object TOPOLOGY_TRANSFER_BUFFER_SIZE_SCHEMA = ConfigValidation.IntegerValidator;
 
-   /**
-    * How often a tick tuple from the "__system" component and "__tick" stream should be sent
-    * to tasks. Meant to be used as a component-specific configuration.
-    */
+    /**
+     * How often a tick tuple from the "__system" component and "__tick" stream should be sent
+     * to tasks. Meant to be used as a component-specific configuration.
+     */
+    @isInteger
     public static final String TOPOLOGY_TICK_TUPLE_FREQ_SECS="topology.tick.tuple.freq.secs";
-    public static final Object TOPOLOGY_TICK_TUPLE_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator;
-
 
    /**
+    * @deprecated this is no longer supported
     * Configure the wait strategy used for internal queuing. Can be used to tradeoff latency
     * vs. throughput
     */
+    @Deprecated
+    @isString
     public static final String TOPOLOGY_DISRUPTOR_WAIT_STRATEGY="topology.disruptor.wait.strategy";
-    public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class;
 
-   /**
-    * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
-    * via the TopologyContext.
-    */
+    /**
+     * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
+     * via the TopologyContext.
+     */
+    @isInteger
     public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size";
-    public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
      * an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
      * reported to Zookeeper per task for every 10 second interval of time.
      */
+    @isInteger
     public static final String TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS="topology.error.throttle.interval.secs";
-    public static final Object TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * See doc for TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL="topology.max.error.report.per.interval";
-    public static final Object TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL_SCHEMA = ConfigValidation.IntegerValidator;
-
 
     /**
      * How often a batch can be emitted in a Trident topology.
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS="topology.trident.batch.emit.interval.millis";
-    public static final Object TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * Name of the topology. This config is automatically set by Storm when the topology is submitted.
      */
+    @isString
     public final static String TOPOLOGY_NAME="topology.name";
-    public static final Object TOPOLOGY_NAME_SCHEMA = String.class;
 
     /**
      * The principal who submitted a topology
      */
+    @isString
     public final static String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal";
-    public static final Object TOPOLOGY_SUBMITTER_PRINCIPAL_SCHEMA = String.class;
 
     /**
      * The local user name of the user who submitted a topology.
      */
+    @isString
     public static final String TOPOLOGY_SUBMITTER_USER = "topology.submitter.user";
-    public static final Object TOPOLOGY_SUBMITTER_USER_SCHEMA = String.class;
 
     /**
      * Array of components that scheduler should try to place on separate hosts.
      */
+    @isStringList
     public static final String TOPOLOGY_SPREAD_COMPONENTS = "topology.spread.components";
-    public static final Object TOPOLOGY_SPREAD_COMPONENTS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * A list of IAutoCredentials that the topology should load and use.
      */
+    @isStringList
     public static final String TOPOLOGY_AUTO_CREDENTIALS = "topology.auto-credentials";
-    public static final Object TOPOLOGY_AUTO_CREDENTIALS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * Max pending tuples in one ShellBolt
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TOPOLOGY_SHELLBOLT_MAX_PENDING="topology.shellbolt.max.pending";
-    public static final Object TOPOLOGY_SHELLBOLT_MAX_PENDING_SCHEMA = ConfigValidation.IntegerValidator;
+
+    /**
+     * How long a subprocess can go without heartbeating before the ShellSpout/ShellBolt tries to
+     * suicide itself.
+     */
+    @isInteger
+    @isPositiveNumber
+    public static final String TOPOLOGY_SUBPROCESS_TIMEOUT_SECS = "topology.subprocess.timeout.secs";
 
     /**
      * Topology central logging sensitivity to determine who has access to logs in central logging system.
@@ -1437,55 +1727,59 @@
      *   S2 - Confidential
      *   S3 - Secret (default.)
      */
+    @isString(acceptedValues = {"S0", "S1", "S2", "S3"})
     public static final String TOPOLOGY_LOGGING_SENSITIVITY="topology.logging.sensitivity";
-    public static final Object TOPOLOGY_LOGGING_SENSITIVITY_SCHEMA = String.class;
 
     /**
      * The root directory in ZooKeeper for metadata about TransactionalSpouts.
      */
+    @isString
     public static final String TRANSACTIONAL_ZOOKEEPER_ROOT="transactional.zookeeper.root";
-    public static final Object TRANSACTIONAL_ZOOKEEPER_ROOT_SCHEMA = String.class;
 
     /**
      * The list of zookeeper servers in which to keep the transactional state. If null (which is default),
      * will use storm.zookeeper.servers
      */
+    @isStringList
     public static final String TRANSACTIONAL_ZOOKEEPER_SERVERS="transactional.zookeeper.servers";
-    public static final Object TRANSACTIONAL_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator;
 
     /**
      * The port to use to connect to the transactional zookeeper servers. If null (which is default),
      * will use storm.zookeeper.port
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TRANSACTIONAL_ZOOKEEPER_PORT="transactional.zookeeper.port";
-    public static final Object TRANSACTIONAL_ZOOKEEPER_PORT_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The user as which the nimbus client should be acquired to perform the operation.
      */
+    @isString
     public static final String STORM_DO_AS_USER="storm.doAsUser";
-    public static final Object STORM_DO_AS_USER_SCHEMA = String.class;
 
     /**
      * The number of threads that should be used by the zeromq context in each worker process.
      */
+    @Deprecated
+    @isInteger
     public static final String ZMQ_THREADS = "zmq.threads";
-    public static final Object ZMQ_THREADS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * How long a connection should retry sending messages to a target host when
      * the connection is closed. This is an advanced configuration and can almost
      * certainly be ignored.
      */
+    @Deprecated
+    @isInteger
     public static final String ZMQ_LINGER_MILLIS = "zmq.linger.millis";
-    public static final Object ZMQ_LINGER_MILLIS_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * The high water for the ZeroMQ push sockets used for networking. Use this config to prevent buffer explosion
      * on the networking layer.
      */
+    @Deprecated
+    @isInteger
     public static final String ZMQ_HWM = "zmq.hwm";
-    public static final Object ZMQ_HWM_SCHEMA = ConfigValidation.IntegerValidator;
 
     /**
      * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
@@ -1493,59 +1787,79 @@
      * to look for native libraries. It is necessary to set this config correctly since
      * Storm uses the ZeroMQ and JZMQ native libs.
      */
+    @isString
     public static final String JAVA_LIBRARY_PATH = "java.library.path";
-    public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class;
 
     /**
      * The path to use as the zookeeper dir when running a zookeeper server via
      * "storm dev-zookeeper". This zookeeper instance is only intended for development;
      * it is not a production grade zookeeper setup.
      */
+    @isString
     public static final String DEV_ZOOKEEPER_PATH = "dev.zookeeper.path";
-    public static final Object DEV_ZOOKEEPER_PATH_SCHEMA = String.class;
 
     /**
      * A map from topology name to the number of machines that should be dedicated for that topology. Set storm.scheduler
      * to backtype.storm.scheduler.IsolationScheduler to make use of the isolation scheduler.
      */
+    @isMapEntryType(keyType = String.class, valueType = Number.class)
     public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines";
-    public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = ConfigValidation.MapOfStringToNumberValidator;
 
     /**
      * A map from the user name to the number of machines that should that user is allowed to use. Set storm.scheduler
      * to backtype.storm.scheduler.multitenant.MultitenantScheduler
      */
+    @isMapEntryType(keyType = String.class, valueType = Number.class)
     public static final String MULTITENANT_SCHEDULER_USER_POOLS = "multitenant.scheduler.user.pools";
-    public static final Object MULTITENANT_SCHEDULER_USER_POOLS_SCHEMA = ConfigValidation.MapOfStringToNumberValidator;
 
     /**
      * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler
      * to backtype.storm.scheduler.multitenant.MultitenantScheduler
      */
+    @isInteger
+    @isPositiveNumber
     public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines";
-    public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class;
 
     /**
      * Configure timeout milliseconds used for disruptor queue wait strategy. Can be used to tradeoff latency
      * vs. CPU usage
      */
+    @isInteger
+    @NotNull
     public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis";
-    public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator;
+
+    /**
+     * The number of tuples to batch before sending to the next thread.  This number is just an initial suggestion and
+     * the code may adjust it as your topology runs.
+     */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
+    public static final String TOPOLOGY_DISRUPTOR_BATCH_SIZE="topology.disruptor.batch.size";
+
+    /**
+     * The maximum age in milliseconds a batch can be before being sent to the next thread.  This number is just an
+     * initial suggestion and the code may adjust it as your topology runs.
+     */
+    @isInteger
+    @isPositiveNumber
+    @NotNull
+    public static final String TOPOLOGY_DISRUPTOR_BATCH_TIMEOUT_MILLIS="topology.disruptor.batch.timeout.millis";
 
     /**
      * Which implementation of {@link backtype.storm.codedistributor.ICodeDistributor} should be used by storm for code
      * distribution.
      */
+    @isString
     public static final String STORM_CODE_DISTRIBUTOR_CLASS = "storm.codedistributor.class";
-    public static final Object STORM_CODE_DISTRIBUTOR_CLASS_SCHEMA = String.class;
 
     /**
      * Minimum number of nimbus hosts where the code must be replicated before leader nimbus
      * is allowed to perform topology activation tasks like setting up heartbeats/assignments
      * and marking the topology as active. default is 0.
      */
+    @isNumber
     public static final String TOPOLOGY_MIN_REPLICATION_COUNT = "topology.min.replication.count";
-    public static final Object TOPOLOGY_MIN_REPLICATION_COUNT_SCHEMA = Number.class;
 
     /**
      * Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
@@ -1553,14 +1867,14 @@
      * if required nimbus.min.replication.count is not achieved. The default is 0 seconds, a value of
      * -1 indicates to wait for ever.
      */
+    @isNumber
     public static final String TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC = "topology.max.replication.wait.time.sec";
-    public static final Object TOPOLOGY_MAX_REPLICATION_WAIT_TIME_SEC_SCHEMA = Number.class;
 
     /**
      * How often nimbus's background thread to sync code for missing topologies should run.
      */
+    @isInteger
     public static final String NIMBUS_CODE_SYNC_FREQ_SECS = "nimbus.code.sync.freq.secs";
-    public static final Object NIMBUS_CODE_SYNC_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator;
 
     public static void setClasspath(Map conf, String cp) {
         conf.put(Config.TOPOLOGY_CLASSPATH, cp);
@@ -1650,7 +1964,7 @@
     }
 
     public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) {
-       registerMetricsConsumer(this, klass, argument, parallelismHint);
+        registerMetricsConsumer(this, klass, argument, parallelismHint);
     }
 
     public static void registerMetricsConsumer(Map conf, Class klass, long parallelismHint) {
@@ -1690,7 +2004,7 @@
     }
 
     public void setSkipMissingKryoRegistrations(boolean skip) {
-       setSkipMissingKryoRegistrations(this, skip);
+        setSkipMissingKryoRegistrations(this, skip);
     }
 
     public static void setMaxTaskParallelism(Map conf, int max) {
@@ -1746,4 +2060,14 @@
         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);
+        }
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/ConfigValidation.java
deleted file mode 100644
index fd9dae7..0000000
--- a/storm-core/src/jvm/backtype/storm/ConfigValidation.java
+++ /dev/null
@@ -1,375 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package backtype.storm;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Provides functionality for validating configuration fields.
- */
-public class ConfigValidation {
-
-    /**
-     * Declares methods for validating configuration values.
-     */
-    public static interface FieldValidator {
-        /**
-         * Validates the given field.
-         * @param name the name of the field.
-         * @param field The field to be validated.
-         * @throws IllegalArgumentException if the field fails validation.
-         */
-        public void validateField(String name, Object field) throws IllegalArgumentException;
-    }
-    
-    /**
-     * Declares a method for validating configuration values that is nestable.
-     */
-    public static abstract class NestableFieldValidator implements FieldValidator {
-        @Override
-        public void validateField(String name, Object field) throws IllegalArgumentException {
-            validateField(null, name, field);
-        }
-        
-        /**
-         * Validates the given field.
-         * @param pd describes the parent wrapping this validator.
-         * @param name the name of the field.
-         * @param field The field to be validated.
-         * @throws IllegalArgumentException if the field fails validation.
-         */
-        public abstract void validateField(String pd, String name, Object field) throws IllegalArgumentException;
-    }
-
-    /**
-     * Returns a new NestableFieldValidator for a given class.
-     * @param cls the Class the field should be a type of
-     * @param nullAllowed whether or not a value of null is valid
-     * @return a NestableFieldValidator for that class
-     */
-    public static NestableFieldValidator fv(final Class cls, final boolean nullAllowed) {
-        return new NestableFieldValidator() {
-            @Override
-            public void validateField(String pd, String name, Object field)
-                    throws IllegalArgumentException {
-                if (nullAllowed && field == null) {
-                    return;
-                }
-                if (! cls.isInstance(field)) {
-                    throw new IllegalArgumentException(
-                        pd + name + " must be a " + cls.getName() + ". ("+field+")");
-                }
-            }
-        };
-    }
-    
-    /**
-     * Returns a new NestableFieldValidator for a List of the given Class.
-     * @param cls the Class of elements composing the list
-     * @param nullAllowed whether or not a value of null is valid
-     * @return a NestableFieldValidator for a list of the given class
-     */
-    public static NestableFieldValidator listFv(Class cls, boolean nullAllowed) {
-      return listFv(fv(cls, false), nullAllowed);
-    }
-    
-    /**
-     * Returns a new NestableFieldValidator for a List where each item is validated by validator.
-     * @param validator used to validate each item in the list
-     * @param nullAllowed whether or not a value of null is valid
-     * @return a NestableFieldValidator for a list with each item validated by a different validator.
-     */
-    public static NestableFieldValidator listFv(final NestableFieldValidator validator, 
-            final boolean nullAllowed) {
-        return new NestableFieldValidator() {
-            @Override
-            public void validateField(String pd, String name, Object field)
-                    throws IllegalArgumentException {
-                if (nullAllowed && field == null) {
-                    return;
-                }
-                if (field instanceof Iterable) {
-                    for (Object e : (Iterable)field) {
-                        validator.validateField(pd + "Each element of the list ", name, e);
-                    }
-                    return;
-                }
-                throw new IllegalArgumentException(
-                        "Field " + name + " must be an Iterable but was " +
-                        ((field == null) ? "null" :  ("a " + field.getClass())));
-            }
-        };
-    }
-
-    /**
-     * Returns a new NestableFieldValidator for a Map of key to val.
-     * @param key the Class of keys in the map
-     * @param val the Class of values in the map
-     * @param nullAllowed whether or not a value of null is valid
-     * @return a NestableFieldValidator for a Map of key to val
-     */
-    public static NestableFieldValidator mapFv(Class key, Class val, 
-            boolean nullAllowed) {
-        return mapFv(fv(key, false), fv(val, false), nullAllowed);
-    }
- 
-    /**
-     * Returns a new NestableFieldValidator for a Map.
-     * @param key a validator for the keys in the map
-     * @param val a validator for the values in the map
-     * @param nullAllowed whether or not a value of null is valid
-     * @return a NestableFieldValidator for a Map
-     */   
-    public static NestableFieldValidator mapFv(final NestableFieldValidator key, 
-            final NestableFieldValidator val, final boolean nullAllowed) {
-        return new NestableFieldValidator() {
-            @SuppressWarnings("unchecked")
-            @Override
-            public void validateField(String pd, String name, Object field)
-                    throws IllegalArgumentException {
-                if (nullAllowed && field == null) {
-                    return;
-                }
-                if (field instanceof Map) {
-                    for (Map.Entry<Object, Object> entry: ((Map<Object, Object>)field).entrySet()) {
-                      key.validateField("Each key of the map ", name, entry.getKey());
-                      val.validateField("Each value in the map ", name, entry.getValue());
-                    }
-                    return;
-                }
-                throw new IllegalArgumentException(
-                        "Field " + name + " must be a Map");
-            }
-        };
-    }
-    
-    /**
-     * Validates a list of Numbers.
-     */
-    public static Object NumbersValidator = listFv(Number.class, true);
-
-    /**
-     * Validates a list of Strings.
-     */
-    public static Object StringsValidator = listFv(String.class, true);
-
-    /**
-     * Validates a map of Strings to Numbers.
-     */
-    public static Object MapOfStringToNumberValidator = mapFv(String.class, Number.class, true);
-
-    /**
-     * Validates a map of Strings to a map of Strings to a list.
-     * {str -> {str -> [str,str]}
-     */
-    public static Object MapOfStringToMapValidator = mapFv(fv(String.class, false), mapFv(fv(String.class, false), listFv(String.class, false), false), true);
-
-    /**
-     * Validates is a list of Maps.
-     */
-    public static Object MapsValidator = listFv(Map.class, true);
-
-    /**
-     * Validates a non null Integer > 0
-     */
-    public static Object NotNullPosIntegerValidator = new FieldValidator() {
-        @Override
-        public void validateField(String name, Object o) throws IllegalArgumentException {
-            final long i;
-            if (o instanceof Number &&
-                    (i = ((Number)o).longValue()) == ((Number)o).doubleValue()) {
-                if (i <= Integer.MAX_VALUE && i > 0) {
-                    return;
-                }
-            }
-
-            throw new IllegalArgumentException("Field " + name + " must be an Integer > 0");
-        }
-    };
-
-    /**
-     * Validates a Integer.
-     */
-    public static Object IntegerValidator = new FieldValidator() {
-        @Override
-        public void validateField(String name, Object o) throws IllegalArgumentException {
-            if (o == null) {
-                // A null value is acceptable.
-                return;
-            }
-            final long i;
-            if (o instanceof Number &&
-                    (i = ((Number)o).longValue()) == ((Number)o).doubleValue()) {
-                if (i <= Integer.MAX_VALUE && i >= Integer.MIN_VALUE) {
-                    return;
-                }
-            }
-
-            throw new IllegalArgumentException("Field " + name + " must be an Integer within type range.");
-        }
-    };
-
-    /**
-     * Validates is a list of Integers.
-     */
-    public static Object NoDuplicateIntegersValidator = new FieldValidator() {
-        @Override
-        public void validateField(String name, Object field)
-                throws IllegalArgumentException {
-            if (field == null) {
-                // A null value is acceptable.
-                return;
-            }
-            int size = 0;
-            Set<Number> integerSet = new HashSet<Number>();
-            if (field instanceof Iterable) {
-                for (Object o : (Iterable)field) {
-                    size++;
-                    final long i;
-                    if (o instanceof Number &&
-                            ((i = ((Number)o).longValue()) == ((Number)o).doubleValue()) &&
-                            (i <= Integer.MAX_VALUE && i >= Integer.MIN_VALUE)) {
-                        // pass the test
-                        integerSet.add((Number) o);
-                    } else {
-                        throw new IllegalArgumentException(
-                                "Each element of the list " + name + " must be an Integer within type range.");
-                    }
-                }
-                if (size != integerSet.size()) {
-                       throw new IllegalArgumentException(name + " should contain no duplicate elements");
-                }
-                return;
-            }
-        }
-    };
-
-    /**
-     * Validates a Positive Number
-     */
-    public static Object PositiveNumberValidator = new FieldValidator() {
-        @Override
-        public void validateField(String name, Object o) throws IllegalArgumentException {
-            if (o == null) {
-                // A null value is acceptable.
-                return;
-            }
-            if(o instanceof Number) {
-                if(((Number)o).doubleValue() > 0.0) {
-                    return;
-                }
-            }
-            throw new IllegalArgumentException("Field " + name + " must be a Positive Number");
-        }
-    };
-
-    /**
-     * Validates a power of 2.
-     */
-    public static Object PowerOf2Validator = new FieldValidator() {
-        @Override
-        public void validateField(String name, Object o) throws IllegalArgumentException {
-            if (o == null) {
-                // A null value is acceptable.
-                return;
-            }
-            final long i;
-            if (o instanceof Number &&
-                    (i = ((Number)o).longValue()) == ((Number)o).doubleValue())
-            {
-                // Test whether the integer is a power of 2.
-                if (i > 0 && (i & (i-1)) == 0) {
-                    return;
-                }
-            }
-            throw new IllegalArgumentException("Field " + name + " must be a power of 2.");
-        }
-    };
-
-    /**
-     * Validates a positive integer.
-     */
-    public static Object PositiveIntegerValidator = new FieldValidator() {
-        @Override
-        public void validateField(String name, Object o) throws IllegalArgumentException {
-            if (o == null) {
-                // A null value is acceptable.
-                return;
-            }
-            final long i;
-            if (o instanceof Number &&
-                    (i = ((Number)o).longValue()) == ((Number)o).doubleValue())
-            {
-                if (i > 0) {
-                    return;
-                }
-            }
-            throw new IllegalArgumentException("Field " + name + " must be a positive integer.");
-        }
-    };
-
-    /**
-     * Validates Kryo Registration
-     */
-    public static Object KryoRegValidator = new FieldValidator() {
-        @Override
-        public void validateField(String name, Object o) throws IllegalArgumentException {
-            if (o == null) {
-                // A null value is acceptable.
-                return;
-            }
-            if (o instanceof Iterable) {
-                for (Object e : (Iterable)o) {
-                    if (e instanceof Map) {
-                        for (Map.Entry<Object,Object> entry: ((Map<Object,Object>)e).entrySet()) {
-                            if (!(entry.getKey() instanceof String) ||
-                                !(entry.getValue() instanceof String)) {
-                                throw new IllegalArgumentException(
-                                    "Each element of the list " + name + " must be a String or a Map of Strings");
-                            }
-                        }
-                    } else if (!(e instanceof String)) {
-                        throw new IllegalArgumentException(
-                                "Each element of the list " + name + " must be a String or a Map of Strings");
-                    }
-                }
-                return;
-            }
-            throw new IllegalArgumentException(
-                    "Field " + name + " must be an Iterable containing only Strings or Maps of Strings");
-        }
-    };
-
-    /**
-     * Validates a String or a list of Strings
-     */
-    public static Object StringOrStringListValidator = new FieldValidator() {
-
-        private FieldValidator fv = listFv(String.class, false);
-
-        @Override
-        public void validateField(String name, Object o) throws IllegalArgumentException {
-            if (o == null || o instanceof String) {
-                // A null value or a String value is acceptable
-                return;
-            }
-            this.fv.validateField(name, o);
-        }
-    };
-}
diff --git a/storm-core/src/jvm/backtype/storm/LogWriter.java b/storm-core/src/jvm/backtype/storm/LogWriter.java
index b0857e8..849f5ca 100644
--- a/storm-core/src/jvm/backtype/storm/LogWriter.java
+++ b/storm-core/src/jvm/backtype/storm/LogWriter.java
@@ -41,7 +41,7 @@
     public void run() {
         Logger logger = this.logger;
         BufferedReader in = this.in;
-        String line = null;
+        String line;
         try {
             while ((line = in.readLine()) != null) {
                 logger.info(line);
diff --git a/storm-core/src/jvm/backtype/storm/StormSubmitter.java b/storm-core/src/jvm/backtype/storm/StormSubmitter.java
index a4ccf5f..1d7a1f3 100644
--- a/storm-core/src/jvm/backtype/storm/StormSubmitter.java
+++ b/storm-core/src/jvm/backtype/storm/StormSubmitter.java
@@ -24,6 +24,8 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import backtype.storm.scheduler.resource.ResourceUtils;
+import backtype.storm.validation.ConfigValidation;
 import org.apache.commons.lang.StringUtils;
 import org.apache.thrift.TException;
 import org.json.simple.JSONValue;
@@ -43,16 +45,12 @@
  * submit your topologies.
  */
 public class StormSubmitter {
-    public static Logger LOG = LoggerFactory.getLogger(StormSubmitter.class);    
+    public static final Logger LOG = LoggerFactory.getLogger(StormSubmitter.class);
 
     private static final int THRIFT_CHUNK_SIZE_BYTES = 307200;
     
     private static ILocalCluster localNimbus = null;
 
-    public static void setLocalNimbus(ILocalCluster localNimbusHandler) {
-        StormSubmitter.localNimbus = localNimbusHandler;
-    }
-
     private static String generateZookeeperDigestSecretPayload() {
         return Utils.secureRandomLong() + ":" + Utils.secureRandomLong();
     }
@@ -89,7 +87,7 @@
     }
 
     private static Map<String,String> populateCredentials(Map conf, Map<String, String> creds) {
-        Map<String,String> ret = new HashMap<String,String>();
+        Map<String,String> ret = new HashMap<>();
         for (IAutoCredentials autoCred: AuthUtils.GetAutoCredentials(conf)) {
             LOG.info("Running "+autoCred);
             autoCred.populateCredentials(ret);
@@ -152,7 +150,7 @@
      * @throws InvalidTopologyException if an invalid topology was submitted
      * @throws AuthorizationException if authorization is failed
      */
-    public static void submitTopology(String name, Map stormConf, StormTopology topology) 
+    public static void submitTopology(String name, Map stormConf, StormTopology topology)
             throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
         submitTopology(name, stormConf, topology, null, null);
     }    
@@ -169,7 +167,7 @@
      * @throws InvalidTopologyException if an invalid topology was submitted
      * @throws AuthorizationException if authorization is failed
      */
-    public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts) 
+    public static void submitTopology(String name, Map stormConf, StormTopology topology, SubmitOptions opts)
             throws AlreadyAliveException, InvalidTopologyException, AuthorizationException {
         submitTopology(name, stormConf, topology, opts, null);
     }
@@ -185,9 +183,10 @@
      * @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,7 +196,9 @@
         conf.putAll(stormConf);
         stormConf.putAll(prepareZookeeperAuthentication(conf));
 
-        Map<String,String> passedCreds = new HashMap<String, String>();
+        validateConfs(conf, topology);
+
+        Map<String,String> passedCreds = new HashMap<>();
         if (opts != null) {
             Credentials tmpCreds = opts.get_creds();
             if (tmpCreds != null) {
@@ -442,4 +443,38 @@
          */
         public void onCompleted(String srcFile, String targetFile, long totalBytes);
     }
+
+    private static void validateConfs(Map stormConf, StormTopology topology) throws IllegalArgumentException {
+        ConfigValidation.validateFields(stormConf);
+        validateTopologyWorkerMaxHeapSizeMBConfigs(stormConf, topology);
+    }
+
+    private static void validateTopologyWorkerMaxHeapSizeMBConfigs(Map stormConf, StormTopology topology) {
+        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;
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java b/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java
new file mode 100644
index 0000000..1bcc645
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/cluster/ClusterState.java
@@ -0,0 +1,208 @@
+/**
+ * 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.cluster;
+
+import clojure.lang.APersistentMap;
+import clojure.lang.IFn;
+import java.util.List;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * ClusterState provides the API for the pluggable state store used by the
+ * Storm daemons. Data is stored in path/value format, and the store supports
+ * listing sub-paths at a given path.
+ * All data should be available across all nodes with eventual consistency.
+ *
+ * IMPORTANT NOTE:
+ * Heartbeats have different api calls used to interact with them. The root
+ * path (/) may or may not be the same as the root path for the other api calls.
+ *
+ * For example, performing these two calls:
+ *     set_data("/path", data, acls);
+ *     void set_worker_hb("/path", heartbeat, acls);
+ * may or may not cause a collision in "/path".
+ * Never use the same paths with the *_hb* methods as you do with the others.
+ */
+public interface ClusterState {
+
+    /**
+     * Registers a callback function that gets called when CuratorEvents happen.
+     * @param callback is a clojure IFn that accepts the type - translated to
+     * clojure keyword as in zookeeper.clj - and the path: (callback type path)
+     * @return is an id that can be passed to unregister(...) to unregister the
+     * callback.
+     */
+    String register(IFn callback);
+
+    /**
+     * Unregisters a callback function that was registered with register(...).
+     * @param id is the String id that was returned from register(...).
+     */
+    void unregister(String id);
+
+    /**
+     * Path will be appended with a monotonically increasing integer, a new node
+     * will be created there, and data will be put at that node.
+     * @param path The path that the monotonically increasing integer suffix will
+     * be added to.
+     * @param data The data that will be written at the suffixed path's node.
+     * @param acls The acls to apply to the path. May be null.
+     * @return The path with the integer suffix appended.
+     */
+    String create_sequential(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Creates nodes for path and all its parents. Path elements are separated by
+     * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
+     * @param path The path to create, along with all its parents.
+     * @param acls The acls to apply to the path. May be null.
+     * @return path
+     */
+    String mkdirs(String path, List<ACL> acls);
+
+    /**
+     * Deletes the node at a given path, and any child nodes that may exist.
+     * @param path The path to delete
+     */
+    void delete_node(String path);
+
+    /**
+     * Creates an ephemeral node at path. Ephemeral nodes are destroyed
+     * by the store when the client disconnects.
+     * @param path The path where a node will be created.
+     * @param data The data to be written at the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Gets the 'version' of the node at a path. Optionally sets a watch
+     * on that node. The version should increase whenever a write happens.
+     * @param path The path to get the version of.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The integer version of this node.
+     */
+    Integer get_version(String path, boolean watch);
+
+    /**
+     * Check if a node exists and optionally set a watch on the path.
+     * @param path The path to check for the existence of a node.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return Whether or not a node exists at path.
+     */
+    boolean node_exists(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_children(String path, boolean watch);
+
+    /**
+     * Close the connection to the data store.
+     */
+    void close();
+
+    /**
+     * Set the value of the node at path to data.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_data(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the data from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The data at the node.
+     */
+    byte[] get_data(String path, boolean watch);
+
+    /**
+     * Get the data at the node along with its version. Data is returned
+     * in an APersistentMap with clojure keyword keys :data and :version.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return An APersistentMap in the form {:data data :version version}
+     */
+    APersistentMap get_data_with_version(String path, boolean watch);
+
+    /**
+     * Write a worker heartbeat at the path.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_worker_hb(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the heartbeat from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The heartbeat at the node.
+     */
+    byte[] get_worker_hb(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path. This is similar to get_children, but must be used for
+     * any nodes
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_worker_hb_children(String path, boolean watch);
+
+    /**
+     * Deletes the heartbeat at a given path, and any child nodes that may exist.
+     * @param path The path to delete.
+     */
+    void delete_worker_hb(String path);
+
+    /**
+     * Add a ClusterStateListener to the connection.
+     * @param listener A ClusterStateListener to handle changing cluster state
+     * events.
+     */
+    void add_listener(ClusterStateListener listener);
+
+    /**
+     * Force consistency on a path. Any writes committed on the path before
+     * this call will be completely propagated when it returns.
+     * @param path The path to synchronize.
+     */
+    void sync_path(String path);
+}
diff --git a/storm-core/src/jvm/backtype/storm/cluster/ClusterStateContext.java b/storm-core/src/jvm/backtype/storm/cluster/ClusterStateContext.java
new file mode 100644
index 0000000..5ccde23
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/cluster/ClusterStateContext.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package backtype.storm.cluster;
+
+/**
+ * This class is intended to provide runtime-context to ClusterStateFactory
+ * implementors, giving information such as what daemon is creating it.
+ */
+public class ClusterStateContext {
+    
+    private DaemonType daemonType;
+
+    public ClusterStateContext() {
+        daemonType = DaemonType.UNKNOWN;
+    }
+    
+    public ClusterStateContext(DaemonType daemonType) {
+        this.daemonType = daemonType;
+    }
+    
+    public DaemonType getDaemonType() {
+        return daemonType;
+    }
+    
+}
diff --git a/storm-core/src/jvm/backtype/storm/cluster/ClusterStateFactory.java b/storm-core/src/jvm/backtype/storm/cluster/ClusterStateFactory.java
new file mode 100644
index 0000000..1f946ee
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/cluster/ClusterStateFactory.java
@@ -0,0 +1,28 @@
+/**
+ * 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.cluster;
+
+import clojure.lang.APersistentMap;
+import java.util.List;
+import org.apache.zookeeper.data.ACL;
+
+public interface ClusterStateFactory {
+    
+    ClusterState mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
+
+}
diff --git a/storm-core/src/jvm/backtype/storm/cluster/ClusterStateListener.java b/storm-core/src/jvm/backtype/storm/cluster/ClusterStateListener.java
new file mode 100644
index 0000000..22693f8
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/cluster/ClusterStateListener.java
@@ -0,0 +1,22 @@
+/**
+ * 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.cluster;
+
+public interface ClusterStateListener {
+    void stateChanged(ConnectionState newState);
+}
diff --git a/storm-core/src/jvm/backtype/storm/cluster/ConnectionState.java b/storm-core/src/jvm/backtype/storm/cluster/ConnectionState.java
new file mode 100644
index 0000000..d6887da
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/cluster/ConnectionState.java
@@ -0,0 +1,24 @@
+/**
+ * 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.cluster;
+
+public enum ConnectionState {
+    CONNECTED,
+    RECONNECTED,
+    LOST
+}
diff --git a/storm-core/src/jvm/backtype/storm/cluster/DaemonType.java b/storm-core/src/jvm/backtype/storm/cluster/DaemonType.java
new file mode 100644
index 0000000..684d0ef
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/cluster/DaemonType.java
@@ -0,0 +1,27 @@
+/**
+ * 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.cluster;
+
+public enum DaemonType {
+    SUPERVISOR,
+    NIMBUS,
+    WORKER,
+    PACEMAKER,
+    UNKNOWN
+}
diff --git a/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java b/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
index 25e578b..c46688f 100644
--- a/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
+++ b/storm-core/src/jvm/backtype/storm/codedistributor/ICodeDistributor.java
@@ -1,3 +1,20 @@
+/**
+ * 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.codedistributor;
 
 
diff --git a/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java b/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
index 02d5e2d..76993e2 100644
--- a/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
+++ b/storm-core/src/jvm/backtype/storm/codedistributor/LocalFileSystemCodeDistributor.java
@@ -1,3 +1,20 @@
+/**
+ * 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.codedistributor;
 
 import backtype.storm.nimbus.NimbusInfo;
diff --git a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
index 8653010..55590d0 100644
--- a/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/coordination/BatchBoltExecutor.java
@@ -32,7 +32,7 @@
 import org.slf4j.LoggerFactory;
 
 public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCallback {
-    public static Logger LOG = LoggerFactory.getLogger(BatchBoltExecutor.class);    
+    public static final Logger LOG = LoggerFactory.getLogger(BatchBoltExecutor.class);
 
     byte[] _boltSer;
     Map<Object, IBatchBolt> _openTransactions;
@@ -49,7 +49,7 @@
         _conf = conf;
         _context = context;
         _collector = new BatchOutputCollectorImpl(collector);
-        _openTransactions = new HashMap<Object, IBatchBolt>();
+        _openTransactions = new HashMap<>();
     }
 
     @Override
diff --git a/storm-core/src/jvm/backtype/storm/coordination/CoordinatedBolt.java b/storm-core/src/jvm/backtype/storm/coordination/CoordinatedBolt.java
index 6f337a6..c3a428c 100644
--- a/storm-core/src/jvm/backtype/storm/coordination/CoordinatedBolt.java
+++ b/storm-core/src/jvm/backtype/storm/coordination/CoordinatedBolt.java
@@ -49,7 +49,7 @@
  * in the case of retries.
  */
 public class CoordinatedBolt implements IRichBolt {
-    public static Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class);
+    public static final Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class);
 
     public static interface FinishedCallback {
         void finishedId(Object id);
@@ -148,7 +148,7 @@
     private IdStreamSpec _idStreamSpec;
     private IRichBolt _delegate;
     private Integer _numSourceReports;
-    private List<Integer> _countOutTasks = new ArrayList<Integer>();;
+    private List<Integer> _countOutTasks = new ArrayList<>();
     private OutputCollector _collector;
     private TimeCacheMap<Object, TrackingInfo> _tracked;
 
@@ -157,10 +157,10 @@
         int expectedTupleCount = 0;
         int receivedTuples = 0;
         boolean failed = false;
-        Map<Integer, Integer> taskEmittedTuples = new HashMap<Integer, Integer>();
+        Map<Integer, Integer> taskEmittedTuples = new HashMap<>();
         boolean receivedId = false;
         boolean finished = false;
-        List<Tuple> ackTuples = new ArrayList<Tuple>();
+        List<Tuple> ackTuples = new ArrayList<>();
         
         @Override
         public String toString() {
@@ -199,7 +199,7 @@
     
     public CoordinatedBolt(IRichBolt delegate, Map<String, SourceArgs> sourceArgs, IdStreamSpec idStreamSpec) {
         _sourceArgs = sourceArgs;
-        if(_sourceArgs==null) _sourceArgs = new HashMap<String, SourceArgs>();
+        if(_sourceArgs==null) _sourceArgs = new HashMap<>();
         _delegate = delegate;
         _idStreamSpec = idStreamSpec;
     }
@@ -209,7 +209,7 @@
         if(_delegate instanceof TimeoutCallback) {
             callback = new TimeoutItems();
         }
-        _tracked = new TimeCacheMap<Object, TrackingInfo>(context.maxTopologyMessageTimeout(), callback);
+        _tracked = new TimeCacheMap<>(context.maxTopologyMessageTimeout(), callback);
         _collector = collector;
         _delegate.prepare(config, context, new OutputCollector(new CoordinatedOutputCollector(collector)));
         for(String component: Utils.get(context.getThisTargets(),
@@ -343,7 +343,7 @@
     }
     
     private static Map<String, SourceArgs> singleSourceArgs(String sourceComponent, SourceArgs sourceArgs) {
-        Map<String, SourceArgs> ret = new HashMap<String, SourceArgs>();
+        Map<String, SourceArgs> ret = new HashMap<>();
         ret.put(sourceComponent, sourceArgs);
         return ret;
     }
@@ -364,7 +364,7 @@
     
     private TupleType getTupleType(Tuple tuple) {
         if(_idStreamSpec!=null
-                && tuple.getSourceGlobalStreamid().equals(_idStreamSpec._id)) {
+                && tuple.getSourceGlobalStreamId().equals(_idStreamSpec._id)) {
             return TupleType.ID;
         } else if(!_sourceArgs.isEmpty()
                 && tuple.getSourceStreamId().equals(Constants.COORDINATED_STREAM_ID)) {
diff --git a/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java b/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
index 624db3e..78e8d9b 100644
--- a/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
+++ b/storm-core/src/jvm/backtype/storm/drpc/DRPCInvocationsClient.java
@@ -31,9 +31,8 @@
 import org.slf4j.LoggerFactory;
 
 public class DRPCInvocationsClient extends ThriftClient implements DistributedRPCInvocations.Iface {
-    public static Logger LOG = LoggerFactory.getLogger(DRPCInvocationsClient.class);
-    private final AtomicReference<DistributedRPCInvocations.Client> client =
-       new AtomicReference<DistributedRPCInvocations.Client>();
+    public static final Logger LOG = LoggerFactory.getLogger(DRPCInvocationsClient.class);
+    private final AtomicReference<DistributedRPCInvocations.Client> client = new AtomicReference<>();
     private String host;
     private int port;
 
diff --git a/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java b/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
index ce233ed..4ed15c0 100644
--- a/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
+++ b/storm-core/src/jvm/backtype/storm/drpc/DRPCSpout.java
@@ -37,7 +37,6 @@
 import java.util.List;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.concurrent.Executors;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.Callable;
@@ -47,17 +46,16 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 import org.json.simple.JSONValue;
 
 public class DRPCSpout extends BaseRichSpout {
     //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS
     static final long serialVersionUID = 2387848310969237877L;
 
-    public static Logger LOG = LoggerFactory.getLogger(DRPCSpout.class);
+    public static final Logger LOG = LoggerFactory.getLogger(DRPCSpout.class);
     
     SpoutOutputCollector _collector;
-    List<DRPCInvocationsClient> _clients = new ArrayList<DRPCInvocationsClient>();
+    List<DRPCInvocationsClient> _clients = new ArrayList<>();
     transient LinkedList<Future<Void>> _futures = null;
     transient ExecutorService _backround = null;
     String _function;
@@ -140,7 +138,7 @@
             _backround = new ExtendedThreadPoolExecutor(0, Integer.MAX_VALUE,
                 60L, TimeUnit.SECONDS,
                 new SynchronousQueue<Runnable>());
-            _futures = new LinkedList<Future<Void>>();
+            _futures = new LinkedList<>();
 
             int numTasks = context.getComponentTasks(context.getThisComponentId()).size();
             int index = context.getThisTaskIndex();
@@ -174,7 +172,7 @@
     public void nextTuple() {
         boolean gotRequest = false;
         if(_local_drpc_id==null) {
-            int size = 0;
+            int size;
             synchronized (_clients) {
                 size = _clients.size(); //This will only ever grow, so no need to worry about falling off the end
             }
diff --git a/storm-core/src/jvm/backtype/storm/drpc/JoinResult.java b/storm-core/src/jvm/backtype/storm/drpc/JoinResult.java
index b74b97e..2c416ed 100644
--- a/storm-core/src/jvm/backtype/storm/drpc/JoinResult.java
+++ b/storm-core/src/jvm/backtype/storm/drpc/JoinResult.java
@@ -33,11 +33,11 @@
 
 
 public class JoinResult extends BaseRichBolt {
-    public static Logger LOG = LoggerFactory.getLogger(JoinResult.class);
+    public static final Logger LOG = LoggerFactory.getLogger(JoinResult.class);
 
     String returnComponent;
-    Map<Object, Tuple> returns = new HashMap<Object, Tuple>();
-    Map<Object, Tuple> results = new HashMap<Object, Tuple>();
+    Map<Object, Tuple> returns = new HashMap<>();
+    Map<Object, Tuple> results = new HashMap<>();
     OutputCollector _collector;
 
     public JoinResult(String returnComponent) {
@@ -60,7 +60,7 @@
             Tuple result = results.remove(requestId);
             Tuple returner = returns.remove(requestId);
             LOG.debug(result.getValue(1).toString());
-            List<Tuple> anchors = new ArrayList<Tuple>();
+            List<Tuple> anchors = new ArrayList<>();
             anchors.add(result);
             anchors.add(returner);            
             _collector.emit(anchors, new Values(""+result.getValue(1), returner.getValue(1)));
diff --git a/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java b/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
index 62647c3..eecf044 100644
--- a/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/AlreadyAliveException.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class AlreadyAliveException extends TException implements org.apache.thrift.TBase<AlreadyAliveException, AlreadyAliveException._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyAliveException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyAliveException");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/Assignment.java b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
index f576c84..cc9bb19 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Assignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Assignment.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class Assignment implements org.apache.thrift.TBase<Assignment, Assignment._Fields>, java.io.Serializable, Cloneable, Comparable<Assignment> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Assignment");
 
@@ -59,6 +59,7 @@
   private static final org.apache.thrift.protocol.TField NODE_HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("node_host", org.apache.thrift.protocol.TType.MAP, (short)2);
   private static final org.apache.thrift.protocol.TField EXECUTOR_NODE_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_node_port", org.apache.thrift.protocol.TType.MAP, (short)3);
   private static final org.apache.thrift.protocol.TField EXECUTOR_START_TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("executor_start_time_secs", org.apache.thrift.protocol.TType.MAP, (short)4);
+  private static final org.apache.thrift.protocol.TField WORKER_RESOURCES_FIELD_DESC = new org.apache.thrift.protocol.TField("worker_resources", org.apache.thrift.protocol.TType.MAP, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -70,13 +71,15 @@
   private Map<String,String> node_host; // optional
   private Map<List<Long>,NodeInfo> executor_node_port; // optional
   private Map<List<Long>,Long> executor_start_time_secs; // optional
+  private Map<NodeInfo,WorkerResources> worker_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 {
     MASTER_CODE_DIR((short)1, "master_code_dir"),
     NODE_HOST((short)2, "node_host"),
     EXECUTOR_NODE_PORT((short)3, "executor_node_port"),
-    EXECUTOR_START_TIME_SECS((short)4, "executor_start_time_secs");
+    EXECUTOR_START_TIME_SECS((short)4, "executor_start_time_secs"),
+    WORKER_RESOURCES((short)5, "worker_resources");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -99,6 +102,8 @@
           return EXECUTOR_NODE_PORT;
         case 4: // EXECUTOR_START_TIME_SECS
           return EXECUTOR_START_TIME_SECS;
+        case 5: // WORKER_RESOURCES
+          return WORKER_RESOURCES;
         default:
           return null;
       }
@@ -139,7 +144,7 @@
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.NODE_HOST,_Fields.EXECUTOR_NODE_PORT,_Fields.EXECUTOR_START_TIME_SECS};
+  private static final _Fields optionals[] = {_Fields.NODE_HOST,_Fields.EXECUTOR_NODE_PORT,_Fields.EXECUTOR_START_TIME_SECS,_Fields.WORKER_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);
@@ -159,6 +164,10 @@
             new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
                 new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WORKER_RESOURCES, new org.apache.thrift.meta_data.FieldMetaData("worker_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.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NodeInfo.class), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerResources.class))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Assignment.class, metaDataMap);
   }
@@ -170,6 +179,8 @@
 
     this.executor_start_time_secs = new HashMap<List<Long>,Long>();
 
+    this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+
   }
 
   public Assignment(
@@ -220,6 +231,21 @@
       }
       this.executor_start_time_secs = __this__executor_start_time_secs;
     }
+    if (other.is_set_worker_resources()) {
+      Map<NodeInfo,WorkerResources> __this__worker_resources = new HashMap<NodeInfo,WorkerResources>(other.worker_resources.size());
+      for (Map.Entry<NodeInfo, WorkerResources> other_element : other.worker_resources.entrySet()) {
+
+        NodeInfo other_element_key = other_element.getKey();
+        WorkerResources other_element_value = other_element.getValue();
+
+        NodeInfo __this__worker_resources_copy_key = new NodeInfo(other_element_key);
+
+        WorkerResources __this__worker_resources_copy_value = new WorkerResources(other_element_value);
+
+        __this__worker_resources.put(__this__worker_resources_copy_key, __this__worker_resources_copy_value);
+      }
+      this.worker_resources = __this__worker_resources;
+    }
   }
 
   public Assignment deepCopy() {
@@ -235,6 +261,8 @@
 
     this.executor_start_time_secs = new HashMap<List<Long>,Long>();
 
+    this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+
   }
 
   public String get_master_code_dir() {
@@ -362,6 +390,40 @@
     }
   }
 
+  public int get_worker_resources_size() {
+    return (this.worker_resources == null) ? 0 : this.worker_resources.size();
+  }
+
+  public void put_to_worker_resources(NodeInfo key, WorkerResources val) {
+    if (this.worker_resources == null) {
+      this.worker_resources = new HashMap<NodeInfo,WorkerResources>();
+    }
+    this.worker_resources.put(key, val);
+  }
+
+  public Map<NodeInfo,WorkerResources> get_worker_resources() {
+    return this.worker_resources;
+  }
+
+  public void set_worker_resources(Map<NodeInfo,WorkerResources> worker_resources) {
+    this.worker_resources = worker_resources;
+  }
+
+  public void unset_worker_resources() {
+    this.worker_resources = null;
+  }
+
+  /** Returns true if field worker_resources is set (has been assigned a value) and false otherwise */
+  public boolean is_set_worker_resources() {
+    return this.worker_resources != null;
+  }
+
+  public void set_worker_resources_isSet(boolean value) {
+    if (!value) {
+      this.worker_resources = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case MASTER_CODE_DIR:
@@ -396,6 +458,14 @@
       }
       break;
 
+    case WORKER_RESOURCES:
+      if (value == null) {
+        unset_worker_resources();
+      } else {
+        set_worker_resources((Map<NodeInfo,WorkerResources>)value);
+      }
+      break;
+
     }
   }
 
@@ -413,6 +483,9 @@
     case EXECUTOR_START_TIME_SECS:
       return get_executor_start_time_secs();
 
+    case WORKER_RESOURCES:
+      return get_worker_resources();
+
     }
     throw new IllegalStateException();
   }
@@ -432,6 +505,8 @@
       return is_set_executor_node_port();
     case EXECUTOR_START_TIME_SECS:
       return is_set_executor_start_time_secs();
+    case WORKER_RESOURCES:
+      return is_set_worker_resources();
     }
     throw new IllegalStateException();
   }
@@ -485,6 +560,15 @@
         return false;
     }
 
+    boolean this_present_worker_resources = true && this.is_set_worker_resources();
+    boolean that_present_worker_resources = true && that.is_set_worker_resources();
+    if (this_present_worker_resources || that_present_worker_resources) {
+      if (!(this_present_worker_resources && that_present_worker_resources))
+        return false;
+      if (!this.worker_resources.equals(that.worker_resources))
+        return false;
+    }
+
     return true;
   }
 
@@ -512,6 +596,11 @@
     if (present_executor_start_time_secs)
       list.add(executor_start_time_secs);
 
+    boolean present_worker_resources = true && (is_set_worker_resources());
+    list.add(present_worker_resources);
+    if (present_worker_resources)
+      list.add(worker_resources);
+
     return list.hashCode();
   }
 
@@ -563,6 +652,16 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_worker_resources()).compareTo(other.is_set_worker_resources());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_worker_resources()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.worker_resources, other.worker_resources);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -620,6 +719,16 @@
       }
       first = false;
     }
+    if (is_set_worker_resources()) {
+      if (!first) sb.append(", ");
+      sb.append("worker_resources:");
+      if (this.worker_resources == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.worker_resources);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -678,15 +787,15 @@
           case 2: // NODE_HOST
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map388 = iprot.readMapBegin();
-                struct.node_host = new HashMap<String,String>(2*_map388.size);
-                String _key389;
-                String _val390;
-                for (int _i391 = 0; _i391 < _map388.size; ++_i391)
+                org.apache.thrift.protocol.TMap _map532 = iprot.readMapBegin();
+                struct.node_host = new HashMap<String,String>(2*_map532.size);
+                String _key533;
+                String _val534;
+                for (int _i535 = 0; _i535 < _map532.size; ++_i535)
                 {
-                  _key389 = iprot.readString();
-                  _val390 = iprot.readString();
-                  struct.node_host.put(_key389, _val390);
+                  _key533 = iprot.readString();
+                  _val534 = iprot.readString();
+                  struct.node_host.put(_key533, _val534);
                 }
                 iprot.readMapEnd();
               }
@@ -698,26 +807,26 @@
           case 3: // EXECUTOR_NODE_PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map392 = iprot.readMapBegin();
-                struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map392.size);
-                List<Long> _key393;
-                NodeInfo _val394;
-                for (int _i395 = 0; _i395 < _map392.size; ++_i395)
+                org.apache.thrift.protocol.TMap _map536 = iprot.readMapBegin();
+                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.TList _list396 = iprot.readListBegin();
-                    _key393 = new ArrayList<Long>(_list396.size);
-                    long _elem397;
-                    for (int _i398 = 0; _i398 < _list396.size; ++_i398)
+                    org.apache.thrift.protocol.TList _list540 = iprot.readListBegin();
+                    _key537 = new ArrayList<Long>(_list540.size);
+                    long _elem541;
+                    for (int _i542 = 0; _i542 < _list540.size; ++_i542)
                     {
-                      _elem397 = iprot.readI64();
-                      _key393.add(_elem397);
+                      _elem541 = iprot.readI64();
+                      _key537.add(_elem541);
                     }
                     iprot.readListEnd();
                   }
-                  _val394 = new NodeInfo();
-                  _val394.read(iprot);
-                  struct.executor_node_port.put(_key393, _val394);
+                  _val538 = new NodeInfo();
+                  _val538.read(iprot);
+                  struct.executor_node_port.put(_key537, _val538);
                 }
                 iprot.readMapEnd();
               }
@@ -729,25 +838,25 @@
           case 4: // EXECUTOR_START_TIME_SECS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map399 = iprot.readMapBegin();
-                struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map399.size);
-                List<Long> _key400;
-                long _val401;
-                for (int _i402 = 0; _i402 < _map399.size; ++_i402)
+                org.apache.thrift.protocol.TMap _map543 = iprot.readMapBegin();
+                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.TList _list403 = iprot.readListBegin();
-                    _key400 = new ArrayList<Long>(_list403.size);
-                    long _elem404;
-                    for (int _i405 = 0; _i405 < _list403.size; ++_i405)
+                    org.apache.thrift.protocol.TList _list547 = iprot.readListBegin();
+                    _key544 = new ArrayList<Long>(_list547.size);
+                    long _elem548;
+                    for (int _i549 = 0; _i549 < _list547.size; ++_i549)
                     {
-                      _elem404 = iprot.readI64();
-                      _key400.add(_elem404);
+                      _elem548 = iprot.readI64();
+                      _key544.add(_elem548);
                     }
                     iprot.readListEnd();
                   }
-                  _val401 = iprot.readI64();
-                  struct.executor_start_time_secs.put(_key400, _val401);
+                  _val545 = iprot.readI64();
+                  struct.executor_start_time_secs.put(_key544, _val545);
                 }
                 iprot.readMapEnd();
               }
@@ -756,6 +865,28 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // WORKER_RESOURCES
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map550 = iprot.readMapBegin();
+                struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map550.size);
+                NodeInfo _key551;
+                WorkerResources _val552;
+                for (int _i553 = 0; _i553 < _map550.size; ++_i553)
+                {
+                  _key551 = new NodeInfo();
+                  _key551.read(iprot);
+                  _val552 = new WorkerResources();
+                  _val552.read(iprot);
+                  struct.worker_resources.put(_key551, _val552);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_worker_resources_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -779,10 +910,10 @@
           oprot.writeFieldBegin(NODE_HOST_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.node_host.size()));
-            for (Map.Entry<String, String> _iter406 : struct.node_host.entrySet())
+            for (Map.Entry<String, String> _iter554 : struct.node_host.entrySet())
             {
-              oprot.writeString(_iter406.getKey());
-              oprot.writeString(_iter406.getValue());
+              oprot.writeString(_iter554.getKey());
+              oprot.writeString(_iter554.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -794,17 +925,17 @@
           oprot.writeFieldBegin(EXECUTOR_NODE_PORT_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, struct.executor_node_port.size()));
-            for (Map.Entry<List<Long>, NodeInfo> _iter407 : struct.executor_node_port.entrySet())
+            for (Map.Entry<List<Long>, NodeInfo> _iter555 : struct.executor_node_port.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter407.getKey().size()));
-                for (long _iter408 : _iter407.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter555.getKey().size()));
+                for (long _iter556 : _iter555.getKey())
                 {
-                  oprot.writeI64(_iter408);
+                  oprot.writeI64(_iter556);
                 }
                 oprot.writeListEnd();
               }
-              _iter407.getValue().write(oprot);
+              _iter555.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -816,17 +947,32 @@
           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> _iter409 : struct.executor_start_time_secs.entrySet())
+            for (Map.Entry<List<Long>, Long> _iter557 : struct.executor_start_time_secs.entrySet())
             {
               {
-                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter409.getKey().size()));
-                for (long _iter410 : _iter409.getKey())
+                oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, _iter557.getKey().size()));
+                for (long _iter558 : _iter557.getKey())
                 {
-                  oprot.writeI64(_iter410);
+                  oprot.writeI64(_iter558);
                 }
                 oprot.writeListEnd();
               }
-              oprot.writeI64(_iter409.getValue());
+              oprot.writeI64(_iter557.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.worker_resources != null) {
+        if (struct.is_set_worker_resources()) {
+          oprot.writeFieldBegin(WORKER_RESOURCES_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.worker_resources.size()));
+            for (Map.Entry<NodeInfo, WorkerResources> _iter559 : struct.worker_resources.entrySet())
+            {
+              _iter559.getKey().write(oprot);
+              _iter559.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -861,46 +1007,59 @@
       if (struct.is_set_executor_start_time_secs()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_worker_resources()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
       if (struct.is_set_node_host()) {
         {
           oprot.writeI32(struct.node_host.size());
-          for (Map.Entry<String, String> _iter411 : struct.node_host.entrySet())
+          for (Map.Entry<String, String> _iter560 : struct.node_host.entrySet())
           {
-            oprot.writeString(_iter411.getKey());
-            oprot.writeString(_iter411.getValue());
+            oprot.writeString(_iter560.getKey());
+            oprot.writeString(_iter560.getValue());
           }
         }
       }
       if (struct.is_set_executor_node_port()) {
         {
           oprot.writeI32(struct.executor_node_port.size());
-          for (Map.Entry<List<Long>, NodeInfo> _iter412 : struct.executor_node_port.entrySet())
+          for (Map.Entry<List<Long>, NodeInfo> _iter561 : struct.executor_node_port.entrySet())
           {
             {
-              oprot.writeI32(_iter412.getKey().size());
-              for (long _iter413 : _iter412.getKey())
+              oprot.writeI32(_iter561.getKey().size());
+              for (long _iter562 : _iter561.getKey())
               {
-                oprot.writeI64(_iter413);
+                oprot.writeI64(_iter562);
               }
             }
-            _iter412.getValue().write(oprot);
+            _iter561.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> _iter414 : struct.executor_start_time_secs.entrySet())
+          for (Map.Entry<List<Long>, Long> _iter563 : struct.executor_start_time_secs.entrySet())
           {
             {
-              oprot.writeI32(_iter414.getKey().size());
-              for (long _iter415 : _iter414.getKey())
+              oprot.writeI32(_iter563.getKey().size());
+              for (long _iter564 : _iter563.getKey())
               {
-                oprot.writeI64(_iter415);
+                oprot.writeI64(_iter564);
               }
             }
-            oprot.writeI64(_iter414.getValue());
+            oprot.writeI64(_iter563.getValue());
+          }
+        }
+      }
+      if (struct.is_set_worker_resources()) {
+        {
+          oprot.writeI32(struct.worker_resources.size());
+          for (Map.Entry<NodeInfo, WorkerResources> _iter565 : struct.worker_resources.entrySet())
+          {
+            _iter565.getKey().write(oprot);
+            _iter565.getValue().write(oprot);
           }
         }
       }
@@ -911,71 +1070,88 @@
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.master_code_dir = iprot.readString();
       struct.set_master_code_dir_isSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map416 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.node_host = new HashMap<String,String>(2*_map416.size);
-          String _key417;
-          String _val418;
-          for (int _i419 = 0; _i419 < _map416.size; ++_i419)
+          org.apache.thrift.protocol.TMap _map566 = 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*_map566.size);
+          String _key567;
+          String _val568;
+          for (int _i569 = 0; _i569 < _map566.size; ++_i569)
           {
-            _key417 = iprot.readString();
-            _val418 = iprot.readString();
-            struct.node_host.put(_key417, _val418);
+            _key567 = iprot.readString();
+            _val568 = iprot.readString();
+            struct.node_host.put(_key567, _val568);
           }
         }
         struct.set_node_host_isSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TMap _map420 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.executor_node_port = new HashMap<List<Long>,NodeInfo>(2*_map420.size);
-          List<Long> _key421;
-          NodeInfo _val422;
-          for (int _i423 = 0; _i423 < _map420.size; ++_i423)
+          org.apache.thrift.protocol.TMap _map570 = 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*_map570.size);
+          List<Long> _key571;
+          NodeInfo _val572;
+          for (int _i573 = 0; _i573 < _map570.size; ++_i573)
           {
             {
-              org.apache.thrift.protocol.TList _list424 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key421 = new ArrayList<Long>(_list424.size);
-              long _elem425;
-              for (int _i426 = 0; _i426 < _list424.size; ++_i426)
+              org.apache.thrift.protocol.TList _list574 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key571 = new ArrayList<Long>(_list574.size);
+              long _elem575;
+              for (int _i576 = 0; _i576 < _list574.size; ++_i576)
               {
-                _elem425 = iprot.readI64();
-                _key421.add(_elem425);
+                _elem575 = iprot.readI64();
+                _key571.add(_elem575);
               }
             }
-            _val422 = new NodeInfo();
-            _val422.read(iprot);
-            struct.executor_node_port.put(_key421, _val422);
+            _val572 = new NodeInfo();
+            _val572.read(iprot);
+            struct.executor_node_port.put(_key571, _val572);
           }
         }
         struct.set_executor_node_port_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map427 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.LIST, org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.executor_start_time_secs = new HashMap<List<Long>,Long>(2*_map427.size);
-          List<Long> _key428;
-          long _val429;
-          for (int _i430 = 0; _i430 < _map427.size; ++_i430)
+          org.apache.thrift.protocol.TMap _map577 = 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*_map577.size);
+          List<Long> _key578;
+          long _val579;
+          for (int _i580 = 0; _i580 < _map577.size; ++_i580)
           {
             {
-              org.apache.thrift.protocol.TList _list431 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-              _key428 = new ArrayList<Long>(_list431.size);
-              long _elem432;
-              for (int _i433 = 0; _i433 < _list431.size; ++_i433)
+              org.apache.thrift.protocol.TList _list581 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+              _key578 = new ArrayList<Long>(_list581.size);
+              long _elem582;
+              for (int _i583 = 0; _i583 < _list581.size; ++_i583)
               {
-                _elem432 = iprot.readI64();
-                _key428.add(_elem432);
+                _elem582 = iprot.readI64();
+                _key578.add(_elem582);
               }
             }
-            _val429 = iprot.readI64();
-            struct.executor_start_time_secs.put(_key428, _val429);
+            _val579 = iprot.readI64();
+            struct.executor_start_time_secs.put(_key578, _val579);
           }
         }
         struct.set_executor_start_time_secs_isSet(true);
       }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TMap _map584 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.worker_resources = new HashMap<NodeInfo,WorkerResources>(2*_map584.size);
+          NodeInfo _key585;
+          WorkerResources _val586;
+          for (int _i587 = 0; _i587 < _map584.size; ++_i587)
+          {
+            _key585 = new NodeInfo();
+            _key585.read(iprot);
+            _val586 = new WorkerResources();
+            _val586.read(iprot);
+            struct.worker_resources.put(_key585, _val586);
+          }
+        }
+        struct.set_worker_resources_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
index a179751..c4bf053 100644
--- a/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/AuthorizationException.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class AuthorizationException extends TException implements org.apache.thrift.TBase<AuthorizationException, AuthorizationException._Fields>, java.io.Serializable, Cloneable, Comparable<AuthorizationException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AuthorizationException");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/Bolt.java b/storm-core/src/jvm/backtype/storm/generated/Bolt.java
index 861226c..df1a007 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Bolt.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Bolt.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class Bolt implements org.apache.thrift.TBase<Bolt, Bolt._Fields>, java.io.Serializable, Cloneable, Comparable<Bolt> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Bolt");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
new file mode 100644
index 0000000..061f3fb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/BoltAggregateStats.java
@@ -0,0 +1,704 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField EXECUTE_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("execute_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)1);
+  private static final org.apache.thrift.protocol.TField PROCESS_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("process_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)2);
+  private static final org.apache.thrift.protocol.TField EXECUTED_FIELD_DESC = new org.apache.thrift.protocol.TField("executed", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField CAPACITY_FIELD_DESC = new org.apache.thrift.protocol.TField("capacity", org.apache.thrift.protocol.TType.DOUBLE, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new BoltAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new BoltAggregateStatsTupleSchemeFactory());
+  }
+
+  private double execute_latency_ms; // optional
+  private double process_latency_ms; // optional
+  private long executed; // optional
+  private double capacity; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXECUTE_LATENCY_MS((short)1, "execute_latency_ms"),
+    PROCESS_LATENCY_MS((short)2, "process_latency_ms"),
+    EXECUTED((short)3, "executed"),
+    CAPACITY((short)4, "capacity");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXECUTE_LATENCY_MS
+          return EXECUTE_LATENCY_MS;
+        case 2: // PROCESS_LATENCY_MS
+          return PROCESS_LATENCY_MS;
+        case 3: // EXECUTED
+          return EXECUTED;
+        case 4: // CAPACITY
+          return CAPACITY;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __EXECUTE_LATENCY_MS_ISSET_ID = 0;
+  private static final int __PROCESS_LATENCY_MS_ISSET_ID = 1;
+  private static final int __EXECUTED_ISSET_ID = 2;
+  private static final int __CAPACITY_ISSET_ID = 3;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.EXECUTE_LATENCY_MS,_Fields.PROCESS_LATENCY_MS,_Fields.EXECUTED,_Fields.CAPACITY};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXECUTE_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("execute_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.PROCESS_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("process_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.EXECUTED, new org.apache.thrift.meta_data.FieldMetaData("executed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.CAPACITY, new org.apache.thrift.meta_data.FieldMetaData("capacity", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(BoltAggregateStats.class, metaDataMap);
+  }
+
+  public BoltAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public BoltAggregateStats(BoltAggregateStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.execute_latency_ms = other.execute_latency_ms;
+    this.process_latency_ms = other.process_latency_ms;
+    this.executed = other.executed;
+    this.capacity = other.capacity;
+  }
+
+  public BoltAggregateStats deepCopy() {
+    return new BoltAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    set_execute_latency_ms_isSet(false);
+    this.execute_latency_ms = 0.0;
+    set_process_latency_ms_isSet(false);
+    this.process_latency_ms = 0.0;
+    set_executed_isSet(false);
+    this.executed = 0;
+    set_capacity_isSet(false);
+    this.capacity = 0.0;
+  }
+
+  public double get_execute_latency_ms() {
+    return this.execute_latency_ms;
+  }
+
+  public void set_execute_latency_ms(double execute_latency_ms) {
+    this.execute_latency_ms = execute_latency_ms;
+    set_execute_latency_ms_isSet(true);
+  }
+
+  public void unset_execute_latency_ms() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID);
+  }
+
+  /** Returns true if field execute_latency_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_execute_latency_ms() {
+    return EncodingUtils.testBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID);
+  }
+
+  public void set_execute_latency_ms_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXECUTE_LATENCY_MS_ISSET_ID, value);
+  }
+
+  public double get_process_latency_ms() {
+    return this.process_latency_ms;
+  }
+
+  public void set_process_latency_ms(double process_latency_ms) {
+    this.process_latency_ms = process_latency_ms;
+    set_process_latency_ms_isSet(true);
+  }
+
+  public void unset_process_latency_ms() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID);
+  }
+
+  /** Returns true if field process_latency_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_process_latency_ms() {
+    return EncodingUtils.testBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID);
+  }
+
+  public void set_process_latency_ms_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROCESS_LATENCY_MS_ISSET_ID, value);
+  }
+
+  public long get_executed() {
+    return this.executed;
+  }
+
+  public void set_executed(long executed) {
+    this.executed = executed;
+    set_executed_isSet(true);
+  }
+
+  public void unset_executed() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EXECUTED_ISSET_ID);
+  }
+
+  /** Returns true if field executed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_executed() {
+    return EncodingUtils.testBit(__isset_bitfield, __EXECUTED_ISSET_ID);
+  }
+
+  public void set_executed_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EXECUTED_ISSET_ID, value);
+  }
+
+  public double get_capacity() {
+    return this.capacity;
+  }
+
+  public void set_capacity(double capacity) {
+    this.capacity = capacity;
+    set_capacity_isSet(true);
+  }
+
+  public void unset_capacity() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CAPACITY_ISSET_ID);
+  }
+
+  /** Returns true if field capacity is set (has been assigned a value) and false otherwise */
+  public boolean is_set_capacity() {
+    return EncodingUtils.testBit(__isset_bitfield, __CAPACITY_ISSET_ID);
+  }
+
+  public void set_capacity_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CAPACITY_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXECUTE_LATENCY_MS:
+      if (value == null) {
+        unset_execute_latency_ms();
+      } else {
+        set_execute_latency_ms((Double)value);
+      }
+      break;
+
+    case PROCESS_LATENCY_MS:
+      if (value == null) {
+        unset_process_latency_ms();
+      } else {
+        set_process_latency_ms((Double)value);
+      }
+      break;
+
+    case EXECUTED:
+      if (value == null) {
+        unset_executed();
+      } else {
+        set_executed((Long)value);
+      }
+      break;
+
+    case CAPACITY:
+      if (value == null) {
+        unset_capacity();
+      } else {
+        set_capacity((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXECUTE_LATENCY_MS:
+      return get_execute_latency_ms();
+
+    case PROCESS_LATENCY_MS:
+      return get_process_latency_ms();
+
+    case EXECUTED:
+      return get_executed();
+
+    case CAPACITY:
+      return get_capacity();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXECUTE_LATENCY_MS:
+      return is_set_execute_latency_ms();
+    case PROCESS_LATENCY_MS:
+      return is_set_process_latency_ms();
+    case EXECUTED:
+      return is_set_executed();
+    case CAPACITY:
+      return is_set_capacity();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof BoltAggregateStats)
+      return this.equals((BoltAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(BoltAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_execute_latency_ms = true && this.is_set_execute_latency_ms();
+    boolean that_present_execute_latency_ms = true && that.is_set_execute_latency_ms();
+    if (this_present_execute_latency_ms || that_present_execute_latency_ms) {
+      if (!(this_present_execute_latency_ms && that_present_execute_latency_ms))
+        return false;
+      if (this.execute_latency_ms != that.execute_latency_ms)
+        return false;
+    }
+
+    boolean this_present_process_latency_ms = true && this.is_set_process_latency_ms();
+    boolean that_present_process_latency_ms = true && that.is_set_process_latency_ms();
+    if (this_present_process_latency_ms || that_present_process_latency_ms) {
+      if (!(this_present_process_latency_ms && that_present_process_latency_ms))
+        return false;
+      if (this.process_latency_ms != that.process_latency_ms)
+        return false;
+    }
+
+    boolean this_present_executed = true && this.is_set_executed();
+    boolean that_present_executed = true && that.is_set_executed();
+    if (this_present_executed || that_present_executed) {
+      if (!(this_present_executed && that_present_executed))
+        return false;
+      if (this.executed != that.executed)
+        return false;
+    }
+
+    boolean this_present_capacity = true && this.is_set_capacity();
+    boolean that_present_capacity = true && that.is_set_capacity();
+    if (this_present_capacity || that_present_capacity) {
+      if (!(this_present_capacity && that_present_capacity))
+        return false;
+      if (this.capacity != that.capacity)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_execute_latency_ms = true && (is_set_execute_latency_ms());
+    list.add(present_execute_latency_ms);
+    if (present_execute_latency_ms)
+      list.add(execute_latency_ms);
+
+    boolean present_process_latency_ms = true && (is_set_process_latency_ms());
+    list.add(present_process_latency_ms);
+    if (present_process_latency_ms)
+      list.add(process_latency_ms);
+
+    boolean present_executed = true && (is_set_executed());
+    list.add(present_executed);
+    if (present_executed)
+      list.add(executed);
+
+    boolean present_capacity = true && (is_set_capacity());
+    list.add(present_capacity);
+    if (present_capacity)
+      list.add(capacity);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(BoltAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_execute_latency_ms()).compareTo(other.is_set_execute_latency_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_execute_latency_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.execute_latency_ms, other.execute_latency_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_process_latency_ms()).compareTo(other.is_set_process_latency_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_process_latency_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.process_latency_ms, other.process_latency_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_executed()).compareTo(other.is_set_executed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_executed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.executed, other.executed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_capacity()).compareTo(other.is_set_capacity());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_capacity()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.capacity, other.capacity);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("BoltAggregateStats(");
+    boolean first = true;
+
+    if (is_set_execute_latency_ms()) {
+      sb.append("execute_latency_ms:");
+      sb.append(this.execute_latency_ms);
+      first = false;
+    }
+    if (is_set_process_latency_ms()) {
+      if (!first) sb.append(", ");
+      sb.append("process_latency_ms:");
+      sb.append(this.process_latency_ms);
+      first = false;
+    }
+    if (is_set_executed()) {
+      if (!first) sb.append(", ");
+      sb.append("executed:");
+      sb.append(this.executed);
+      first = false;
+    }
+    if (is_set_capacity()) {
+      if (!first) sb.append(", ");
+      sb.append("capacity:");
+      sb.append(this.capacity);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class BoltAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public BoltAggregateStatsStandardScheme getScheme() {
+      return new BoltAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class BoltAggregateStatsStandardScheme extends StandardScheme<BoltAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXECUTE_LATENCY_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.execute_latency_ms = iprot.readDouble();
+              struct.set_execute_latency_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // PROCESS_LATENCY_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.process_latency_ms = iprot.readDouble();
+              struct.set_process_latency_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EXECUTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.executed = iprot.readI64();
+              struct.set_executed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // CAPACITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.capacity = iprot.readDouble();
+              struct.set_capacity_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_execute_latency_ms()) {
+        oprot.writeFieldBegin(EXECUTE_LATENCY_MS_FIELD_DESC);
+        oprot.writeDouble(struct.execute_latency_ms);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_process_latency_ms()) {
+        oprot.writeFieldBegin(PROCESS_LATENCY_MS_FIELD_DESC);
+        oprot.writeDouble(struct.process_latency_ms);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_executed()) {
+        oprot.writeFieldBegin(EXECUTED_FIELD_DESC);
+        oprot.writeI64(struct.executed);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_capacity()) {
+        oprot.writeFieldBegin(CAPACITY_FIELD_DESC);
+        oprot.writeDouble(struct.capacity);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class BoltAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public BoltAggregateStatsTupleScheme getScheme() {
+      return new BoltAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class BoltAggregateStatsTupleScheme extends TupleScheme<BoltAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_execute_latency_ms()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_process_latency_ms()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_executed()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_capacity()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_execute_latency_ms()) {
+        oprot.writeDouble(struct.execute_latency_ms);
+      }
+      if (struct.is_set_process_latency_ms()) {
+        oprot.writeDouble(struct.process_latency_ms);
+      }
+      if (struct.is_set_executed()) {
+        oprot.writeI64(struct.executed);
+      }
+      if (struct.is_set_capacity()) {
+        oprot.writeDouble(struct.capacity);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, BoltAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.execute_latency_ms = iprot.readDouble();
+        struct.set_execute_latency_ms_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.process_latency_ms = iprot.readDouble();
+        struct.set_process_latency_ms_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.executed = iprot.readI64();
+        struct.set_executed_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.capacity = iprot.readDouble();
+        struct.set_capacity_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/BoltStats.java b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
index 26ef5d8..cbadd32 100644
--- a/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/BoltStats.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class BoltStats implements org.apache.thrift.TBase<BoltStats, BoltStats._Fields>, java.io.Serializable, Cloneable, Comparable<BoltStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("BoltStats");
 
@@ -881,28 +881,28 @@
           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 _map116 = iprot.readMapBegin();
+                struct.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map116.size);
+                String _key117;
+                Map<GlobalStreamId,Long> _val118;
+                for (int _i119 = 0; _i119 < _map116.size; ++_i119)
                 {
-                  _key99 = iprot.readString();
+                  _key117 = 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 _map120 = iprot.readMapBegin();
+                    _val118 = new HashMap<GlobalStreamId,Long>(2*_map120.size);
+                    GlobalStreamId _key121;
+                    long _val122;
+                    for (int _i123 = 0; _i123 < _map120.size; ++_i123)
                     {
-                      _key103 = new GlobalStreamId();
-                      _key103.read(iprot);
-                      _val104 = iprot.readI64();
-                      _val100.put(_key103, _val104);
+                      _key121 = new GlobalStreamId();
+                      _key121.read(iprot);
+                      _val122 = iprot.readI64();
+                      _val118.put(_key121, _val122);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.acked.put(_key99, _val100);
+                  struct.acked.put(_key117, _val118);
                 }
                 iprot.readMapEnd();
               }
@@ -914,28 +914,28 @@
           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 _map124 = iprot.readMapBegin();
+                struct.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map124.size);
+                String _key125;
+                Map<GlobalStreamId,Long> _val126;
+                for (int _i127 = 0; _i127 < _map124.size; ++_i127)
                 {
-                  _key107 = iprot.readString();
+                  _key125 = 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 _map128 = iprot.readMapBegin();
+                    _val126 = new HashMap<GlobalStreamId,Long>(2*_map128.size);
+                    GlobalStreamId _key129;
+                    long _val130;
+                    for (int _i131 = 0; _i131 < _map128.size; ++_i131)
                     {
-                      _key111 = new GlobalStreamId();
-                      _key111.read(iprot);
-                      _val112 = iprot.readI64();
-                      _val108.put(_key111, _val112);
+                      _key129 = new GlobalStreamId();
+                      _key129.read(iprot);
+                      _val130 = iprot.readI64();
+                      _val126.put(_key129, _val130);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.failed.put(_key107, _val108);
+                  struct.failed.put(_key125, _val126);
                 }
                 iprot.readMapEnd();
               }
@@ -947,28 +947,28 @@
           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 _map132 = iprot.readMapBegin();
+                struct.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map132.size);
+                String _key133;
+                Map<GlobalStreamId,Double> _val134;
+                for (int _i135 = 0; _i135 < _map132.size; ++_i135)
                 {
-                  _key115 = iprot.readString();
+                  _key133 = 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 _map136 = iprot.readMapBegin();
+                    _val134 = new HashMap<GlobalStreamId,Double>(2*_map136.size);
+                    GlobalStreamId _key137;
+                    double _val138;
+                    for (int _i139 = 0; _i139 < _map136.size; ++_i139)
                     {
-                      _key119 = new GlobalStreamId();
-                      _key119.read(iprot);
-                      _val120 = iprot.readDouble();
-                      _val116.put(_key119, _val120);
+                      _key137 = new GlobalStreamId();
+                      _key137.read(iprot);
+                      _val138 = iprot.readDouble();
+                      _val134.put(_key137, _val138);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.process_ms_avg.put(_key115, _val116);
+                  struct.process_ms_avg.put(_key133, _val134);
                 }
                 iprot.readMapEnd();
               }
@@ -980,28 +980,28 @@
           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 _map140 = iprot.readMapBegin();
+                struct.executed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map140.size);
+                String _key141;
+                Map<GlobalStreamId,Long> _val142;
+                for (int _i143 = 0; _i143 < _map140.size; ++_i143)
                 {
-                  _key123 = iprot.readString();
+                  _key141 = 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 _map144 = iprot.readMapBegin();
+                    _val142 = new HashMap<GlobalStreamId,Long>(2*_map144.size);
+                    GlobalStreamId _key145;
+                    long _val146;
+                    for (int _i147 = 0; _i147 < _map144.size; ++_i147)
                     {
-                      _key127 = new GlobalStreamId();
-                      _key127.read(iprot);
-                      _val128 = iprot.readI64();
-                      _val124.put(_key127, _val128);
+                      _key145 = new GlobalStreamId();
+                      _key145.read(iprot);
+                      _val146 = iprot.readI64();
+                      _val142.put(_key145, _val146);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.executed.put(_key123, _val124);
+                  struct.executed.put(_key141, _val142);
                 }
                 iprot.readMapEnd();
               }
@@ -1013,28 +1013,28 @@
           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 _map148 = iprot.readMapBegin();
+                struct.execute_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map148.size);
+                String _key149;
+                Map<GlobalStreamId,Double> _val150;
+                for (int _i151 = 0; _i151 < _map148.size; ++_i151)
                 {
-                  _key131 = iprot.readString();
+                  _key149 = 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 _map152 = iprot.readMapBegin();
+                    _val150 = new HashMap<GlobalStreamId,Double>(2*_map152.size);
+                    GlobalStreamId _key153;
+                    double _val154;
+                    for (int _i155 = 0; _i155 < _map152.size; ++_i155)
                     {
-                      _key135 = new GlobalStreamId();
-                      _key135.read(iprot);
-                      _val136 = iprot.readDouble();
-                      _val132.put(_key135, _val136);
+                      _key153 = new GlobalStreamId();
+                      _key153.read(iprot);
+                      _val154 = iprot.readDouble();
+                      _val150.put(_key153, _val154);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.execute_ms_avg.put(_key131, _val132);
+                  struct.execute_ms_avg.put(_key149, _val150);
                 }
                 iprot.readMapEnd();
               }
@@ -1060,15 +1060,15 @@
         oprot.writeFieldBegin(ACKED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter138 : struct.acked.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter156 : struct.acked.entrySet())
           {
-            oprot.writeString(_iter138.getKey());
+            oprot.writeString(_iter156.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, _iter156.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter157 : _iter156.getValue().entrySet())
               {
-                _iter139.getKey().write(oprot);
-                oprot.writeI64(_iter139.getValue());
+                _iter157.getKey().write(oprot);
+                oprot.writeI64(_iter157.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1081,15 +1081,15 @@
         oprot.writeFieldBegin(FAILED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter140 : struct.failed.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter158 : struct.failed.entrySet())
           {
-            oprot.writeString(_iter140.getKey());
+            oprot.writeString(_iter158.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, _iter158.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter159 : _iter158.getValue().entrySet())
               {
-                _iter141.getKey().write(oprot);
-                oprot.writeI64(_iter141.getValue());
+                _iter159.getKey().write(oprot);
+                oprot.writeI64(_iter159.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1102,15 +1102,15 @@
         oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.process_ms_avg.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter142 : struct.process_ms_avg.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter160 : struct.process_ms_avg.entrySet())
           {
-            oprot.writeString(_iter142.getKey());
+            oprot.writeString(_iter160.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, _iter160.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Double> _iter161 : _iter160.getValue().entrySet())
               {
-                _iter143.getKey().write(oprot);
-                oprot.writeDouble(_iter143.getValue());
+                _iter161.getKey().write(oprot);
+                oprot.writeDouble(_iter161.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1123,15 +1123,15 @@
         oprot.writeFieldBegin(EXECUTED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.executed.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter144 : struct.executed.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter162 : struct.executed.entrySet())
           {
-            oprot.writeString(_iter144.getKey());
+            oprot.writeString(_iter162.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, _iter162.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Long> _iter163 : _iter162.getValue().entrySet())
               {
-                _iter145.getKey().write(oprot);
-                oprot.writeI64(_iter145.getValue());
+                _iter163.getKey().write(oprot);
+                oprot.writeI64(_iter163.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1144,15 +1144,15 @@
         oprot.writeFieldBegin(EXECUTE_MS_AVG_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.execute_ms_avg.size()));
-          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter146 : struct.execute_ms_avg.entrySet())
+          for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter164 : struct.execute_ms_avg.entrySet())
           {
-            oprot.writeString(_iter146.getKey());
+            oprot.writeString(_iter164.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, _iter164.getValue().size()));
+              for (Map.Entry<GlobalStreamId, Double> _iter165 : _iter164.getValue().entrySet())
               {
-                _iter147.getKey().write(oprot);
-                oprot.writeDouble(_iter147.getValue());
+                _iter165.getKey().write(oprot);
+                oprot.writeDouble(_iter165.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -1180,75 +1180,75 @@
       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>> _iter166 : struct.acked.entrySet())
         {
-          oprot.writeString(_iter148.getKey());
+          oprot.writeString(_iter166.getKey());
           {
-            oprot.writeI32(_iter148.getValue().size());
-            for (Map.Entry<GlobalStreamId, Long> _iter149 : _iter148.getValue().entrySet())
+            oprot.writeI32(_iter166.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter167 : _iter166.getValue().entrySet())
             {
-              _iter149.getKey().write(oprot);
-              oprot.writeI64(_iter149.getValue());
+              _iter167.getKey().write(oprot);
+              oprot.writeI64(_iter167.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.failed.size());
-        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter150 : struct.failed.entrySet())
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter168 : struct.failed.entrySet())
         {
-          oprot.writeString(_iter150.getKey());
+          oprot.writeString(_iter168.getKey());
           {
-            oprot.writeI32(_iter150.getValue().size());
-            for (Map.Entry<GlobalStreamId, Long> _iter151 : _iter150.getValue().entrySet())
+            oprot.writeI32(_iter168.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter169 : _iter168.getValue().entrySet())
             {
-              _iter151.getKey().write(oprot);
-              oprot.writeI64(_iter151.getValue());
+              _iter169.getKey().write(oprot);
+              oprot.writeI64(_iter169.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>> _iter170 : struct.process_ms_avg.entrySet())
         {
-          oprot.writeString(_iter152.getKey());
+          oprot.writeString(_iter170.getKey());
           {
-            oprot.writeI32(_iter152.getValue().size());
-            for (Map.Entry<GlobalStreamId, Double> _iter153 : _iter152.getValue().entrySet())
+            oprot.writeI32(_iter170.getValue().size());
+            for (Map.Entry<GlobalStreamId, Double> _iter171 : _iter170.getValue().entrySet())
             {
-              _iter153.getKey().write(oprot);
-              oprot.writeDouble(_iter153.getValue());
+              _iter171.getKey().write(oprot);
+              oprot.writeDouble(_iter171.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.executed.size());
-        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter154 : struct.executed.entrySet())
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter172 : struct.executed.entrySet())
         {
-          oprot.writeString(_iter154.getKey());
+          oprot.writeString(_iter172.getKey());
           {
-            oprot.writeI32(_iter154.getValue().size());
-            for (Map.Entry<GlobalStreamId, Long> _iter155 : _iter154.getValue().entrySet())
+            oprot.writeI32(_iter172.getValue().size());
+            for (Map.Entry<GlobalStreamId, Long> _iter173 : _iter172.getValue().entrySet())
             {
-              _iter155.getKey().write(oprot);
-              oprot.writeI64(_iter155.getValue());
+              _iter173.getKey().write(oprot);
+              oprot.writeI64(_iter173.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>> _iter174 : struct.execute_ms_avg.entrySet())
         {
-          oprot.writeString(_iter156.getKey());
+          oprot.writeString(_iter174.getKey());
           {
-            oprot.writeI32(_iter156.getValue().size());
-            for (Map.Entry<GlobalStreamId, Double> _iter157 : _iter156.getValue().entrySet())
+            oprot.writeI32(_iter174.getValue().size());
+            for (Map.Entry<GlobalStreamId, Double> _iter175 : _iter174.getValue().entrySet())
             {
-              _iter157.getKey().write(oprot);
-              oprot.writeDouble(_iter157.getValue());
+              _iter175.getKey().write(oprot);
+              oprot.writeDouble(_iter175.getValue());
             }
           }
         }
@@ -1259,127 +1259,127 @@
     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 _map176 = 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*_map176.size);
+        String _key177;
+        Map<GlobalStreamId,Long> _val178;
+        for (int _i179 = 0; _i179 < _map176.size; ++_i179)
         {
-          _key159 = iprot.readString();
+          _key177 = 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 _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)
             {
-              _key163 = new GlobalStreamId();
-              _key163.read(iprot);
-              _val164 = iprot.readI64();
-              _val160.put(_key163, _val164);
+              _key181 = new GlobalStreamId();
+              _key181.read(iprot);
+              _val182 = iprot.readI64();
+              _val178.put(_key181, _val182);
             }
           }
-          struct.acked.put(_key159, _val160);
+          struct.acked.put(_key177, _val178);
         }
       }
       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 _map184 = 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*_map184.size);
+        String _key185;
+        Map<GlobalStreamId,Long> _val186;
+        for (int _i187 = 0; _i187 < _map184.size; ++_i187)
         {
-          _key167 = iprot.readString();
+          _key185 = 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 _map188 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val186 = new HashMap<GlobalStreamId,Long>(2*_map188.size);
+            GlobalStreamId _key189;
+            long _val190;
+            for (int _i191 = 0; _i191 < _map188.size; ++_i191)
             {
-              _key171 = new GlobalStreamId();
-              _key171.read(iprot);
-              _val172 = iprot.readI64();
-              _val168.put(_key171, _val172);
+              _key189 = new GlobalStreamId();
+              _key189.read(iprot);
+              _val190 = iprot.readI64();
+              _val186.put(_key189, _val190);
             }
           }
-          struct.failed.put(_key167, _val168);
+          struct.failed.put(_key185, _val186);
         }
       }
       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 _map192 = 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*_map192.size);
+        String _key193;
+        Map<GlobalStreamId,Double> _val194;
+        for (int _i195 = 0; _i195 < _map192.size; ++_i195)
         {
-          _key175 = iprot.readString();
+          _key193 = 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 _map196 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val194 = new HashMap<GlobalStreamId,Double>(2*_map196.size);
+            GlobalStreamId _key197;
+            double _val198;
+            for (int _i199 = 0; _i199 < _map196.size; ++_i199)
             {
-              _key179 = new GlobalStreamId();
-              _key179.read(iprot);
-              _val180 = iprot.readDouble();
-              _val176.put(_key179, _val180);
+              _key197 = new GlobalStreamId();
+              _key197.read(iprot);
+              _val198 = iprot.readDouble();
+              _val194.put(_key197, _val198);
             }
           }
-          struct.process_ms_avg.put(_key175, _val176);
+          struct.process_ms_avg.put(_key193, _val194);
         }
       }
       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 _map200 = 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*_map200.size);
+        String _key201;
+        Map<GlobalStreamId,Long> _val202;
+        for (int _i203 = 0; _i203 < _map200.size; ++_i203)
         {
-          _key183 = iprot.readString();
+          _key201 = 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 _map204 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val202 = new HashMap<GlobalStreamId,Long>(2*_map204.size);
+            GlobalStreamId _key205;
+            long _val206;
+            for (int _i207 = 0; _i207 < _map204.size; ++_i207)
             {
-              _key187 = new GlobalStreamId();
-              _key187.read(iprot);
-              _val188 = iprot.readI64();
-              _val184.put(_key187, _val188);
+              _key205 = new GlobalStreamId();
+              _key205.read(iprot);
+              _val206 = iprot.readI64();
+              _val202.put(_key205, _val206);
             }
           }
-          struct.executed.put(_key183, _val184);
+          struct.executed.put(_key201, _val202);
         }
       }
       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 _map208 = 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*_map208.size);
+        String _key209;
+        Map<GlobalStreamId,Double> _val210;
+        for (int _i211 = 0; _i211 < _map208.size; ++_i211)
         {
-          _key191 = iprot.readString();
+          _key209 = 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 _map212 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val210 = new HashMap<GlobalStreamId,Double>(2*_map212.size);
+            GlobalStreamId _key213;
+            double _val214;
+            for (int _i215 = 0; _i215 < _map212.size; ++_i215)
             {
-              _key195 = new GlobalStreamId();
-              _key195.read(iprot);
-              _val196 = iprot.readDouble();
-              _val192.put(_key195, _val196);
+              _key213 = new GlobalStreamId();
+              _key213.read(iprot);
+              _val214 = iprot.readDouble();
+              _val210.put(_key213, _val214);
             }
           }
-          struct.execute_ms_avg.put(_key191, _val192);
+          struct.execute_ms_avg.put(_key209, _val210);
         }
       }
       struct.set_execute_ms_avg_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
index e5651df..9c42427 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterSummary.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,11 +51,12 @@
 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.3)")
 public class ClusterSummary implements org.apache.thrift.TBase<ClusterSummary, ClusterSummary._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterSummary");
 
   private static final org.apache.thrift.protocol.TField SUPERVISORS_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisors", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField NIMBUS_UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus_uptime_secs", org.apache.thrift.protocol.TType.I32, (short)2);
   private static final org.apache.thrift.protocol.TField TOPOLOGIES_FIELD_DESC = new org.apache.thrift.protocol.TField("topologies", org.apache.thrift.protocol.TType.LIST, (short)3);
   private static final org.apache.thrift.protocol.TField NIMBUSES_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbuses", org.apache.thrift.protocol.TType.LIST, (short)4);
 
@@ -66,12 +67,14 @@
   }
 
   private List<SupervisorSummary> supervisors; // required
+  private int nimbus_uptime_secs; // optional
   private List<TopologySummary> topologies; // required
   private List<NimbusSummary> nimbuses; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     SUPERVISORS((short)1, "supervisors"),
+    NIMBUS_UPTIME_SECS((short)2, "nimbus_uptime_secs"),
     TOPOLOGIES((short)3, "topologies"),
     NIMBUSES((short)4, "nimbuses");
 
@@ -90,6 +93,8 @@
       switch(fieldId) {
         case 1: // SUPERVISORS
           return SUPERVISORS;
+        case 2: // NIMBUS_UPTIME_SECS
+          return NIMBUS_UPTIME_SECS;
         case 3: // TOPOLOGIES
           return TOPOLOGIES;
         case 4: // NIMBUSES
@@ -134,12 +139,17 @@
   }
 
   // isset id assignments
+  private static final int __NIMBUS_UPTIME_SECS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NIMBUS_UPTIME_SECS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
     tmpMap.put(_Fields.SUPERVISORS, new org.apache.thrift.meta_data.FieldMetaData("supervisors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorSummary.class))));
+    tmpMap.put(_Fields.NIMBUS_UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("nimbus_uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     tmpMap.put(_Fields.TOPOLOGIES, new org.apache.thrift.meta_data.FieldMetaData("topologies", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologySummary.class))));
@@ -151,6 +161,8 @@
   }
 
   public ClusterSummary() {
+    this.nimbus_uptime_secs = 0;
+
   }
 
   public ClusterSummary(
@@ -168,6 +180,7 @@
    * Performs a deep copy on <i>other</i>.
    */
   public ClusterSummary(ClusterSummary other) {
+    __isset_bitfield = other.__isset_bitfield;
     if (other.is_set_supervisors()) {
       List<SupervisorSummary> __this__supervisors = new ArrayList<SupervisorSummary>(other.supervisors.size());
       for (SupervisorSummary other_element : other.supervisors) {
@@ -175,6 +188,7 @@
       }
       this.supervisors = __this__supervisors;
     }
+    this.nimbus_uptime_secs = other.nimbus_uptime_secs;
     if (other.is_set_topologies()) {
       List<TopologySummary> __this__topologies = new ArrayList<TopologySummary>(other.topologies.size());
       for (TopologySummary other_element : other.topologies) {
@@ -198,6 +212,8 @@
   @Override
   public void clear() {
     this.supervisors = null;
+    this.nimbus_uptime_secs = 0;
+
     this.topologies = null;
     this.nimbuses = null;
   }
@@ -240,6 +256,28 @@
     }
   }
 
+  public int get_nimbus_uptime_secs() {
+    return this.nimbus_uptime_secs;
+  }
+
+  public void set_nimbus_uptime_secs(int nimbus_uptime_secs) {
+    this.nimbus_uptime_secs = nimbus_uptime_secs;
+    set_nimbus_uptime_secs_isSet(true);
+  }
+
+  public void unset_nimbus_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field nimbus_uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_nimbus_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_nimbus_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NIMBUS_UPTIME_SECS_ISSET_ID, value);
+  }
+
   public int get_topologies_size() {
     return (this.topologies == null) ? 0 : this.topologies.size();
   }
@@ -326,6 +364,14 @@
       }
       break;
 
+    case NIMBUS_UPTIME_SECS:
+      if (value == null) {
+        unset_nimbus_uptime_secs();
+      } else {
+        set_nimbus_uptime_secs((Integer)value);
+      }
+      break;
+
     case TOPOLOGIES:
       if (value == null) {
         unset_topologies();
@@ -350,6 +396,9 @@
     case SUPERVISORS:
       return get_supervisors();
 
+    case NIMBUS_UPTIME_SECS:
+      return get_nimbus_uptime_secs();
+
     case TOPOLOGIES:
       return get_topologies();
 
@@ -369,6 +418,8 @@
     switch (field) {
     case SUPERVISORS:
       return is_set_supervisors();
+    case NIMBUS_UPTIME_SECS:
+      return is_set_nimbus_uptime_secs();
     case TOPOLOGIES:
       return is_set_topologies();
     case NIMBUSES:
@@ -399,6 +450,15 @@
         return false;
     }
 
+    boolean this_present_nimbus_uptime_secs = true && this.is_set_nimbus_uptime_secs();
+    boolean that_present_nimbus_uptime_secs = true && that.is_set_nimbus_uptime_secs();
+    if (this_present_nimbus_uptime_secs || that_present_nimbus_uptime_secs) {
+      if (!(this_present_nimbus_uptime_secs && that_present_nimbus_uptime_secs))
+        return false;
+      if (this.nimbus_uptime_secs != that.nimbus_uptime_secs)
+        return false;
+    }
+
     boolean this_present_topologies = true && this.is_set_topologies();
     boolean that_present_topologies = true && that.is_set_topologies();
     if (this_present_topologies || that_present_topologies) {
@@ -429,6 +489,11 @@
     if (present_supervisors)
       list.add(supervisors);
 
+    boolean present_nimbus_uptime_secs = true && (is_set_nimbus_uptime_secs());
+    list.add(present_nimbus_uptime_secs);
+    if (present_nimbus_uptime_secs)
+      list.add(nimbus_uptime_secs);
+
     boolean present_topologies = true && (is_set_topologies());
     list.add(present_topologies);
     if (present_topologies)
@@ -460,6 +525,16 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_nimbus_uptime_secs()).compareTo(other.is_set_nimbus_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_nimbus_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus_uptime_secs, other.nimbus_uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(other.is_set_topologies());
     if (lastComparison != 0) {
       return lastComparison;
@@ -507,6 +582,12 @@
       sb.append(this.supervisors);
     }
     first = false;
+    if (is_set_nimbus_uptime_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("nimbus_uptime_secs:");
+      sb.append(this.nimbus_uptime_secs);
+      first = false;
+    }
     if (!first) sb.append(", ");
     sb.append("topologies:");
     if (this.topologies == null) {
@@ -554,6 +635,8 @@
 
   private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
     try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
       read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
     } catch (org.apache.thrift.TException te) {
       throw new java.io.IOException(te);
@@ -581,14 +664,14 @@
           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 _list92 = iprot.readListBegin();
+                struct.supervisors = new ArrayList<SupervisorSummary>(_list92.size);
+                SupervisorSummary _elem93;
+                for (int _i94 = 0; _i94 < _list92.size; ++_i94)
                 {
-                  _elem75 = new SupervisorSummary();
-                  _elem75.read(iprot);
-                  struct.supervisors.add(_elem75);
+                  _elem93 = new SupervisorSummary();
+                  _elem93.read(iprot);
+                  struct.supervisors.add(_elem93);
                 }
                 iprot.readListEnd();
               }
@@ -597,17 +680,25 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 2: // NIMBUS_UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.nimbus_uptime_secs = iprot.readI32();
+              struct.set_nimbus_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           case 3: // TOPOLOGIES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list77 = iprot.readListBegin();
-                struct.topologies = new ArrayList<TopologySummary>(_list77.size);
-                TopologySummary _elem78;
-                for (int _i79 = 0; _i79 < _list77.size; ++_i79)
+                org.apache.thrift.protocol.TList _list95 = iprot.readListBegin();
+                struct.topologies = new ArrayList<TopologySummary>(_list95.size);
+                TopologySummary _elem96;
+                for (int _i97 = 0; _i97 < _list95.size; ++_i97)
                 {
-                  _elem78 = new TopologySummary();
-                  _elem78.read(iprot);
-                  struct.topologies.add(_elem78);
+                  _elem96 = new TopologySummary();
+                  _elem96.read(iprot);
+                  struct.topologies.add(_elem96);
                 }
                 iprot.readListEnd();
               }
@@ -619,14 +710,14 @@
           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 _list98 = iprot.readListBegin();
+                struct.nimbuses = new ArrayList<NimbusSummary>(_list98.size);
+                NimbusSummary _elem99;
+                for (int _i100 = 0; _i100 < _list98.size; ++_i100)
                 {
-                  _elem81 = new NimbusSummary();
-                  _elem81.read(iprot);
-                  struct.nimbuses.add(_elem81);
+                  _elem99 = new NimbusSummary();
+                  _elem99.read(iprot);
+                  struct.nimbuses.add(_elem99);
                 }
                 iprot.readListEnd();
               }
@@ -652,21 +743,26 @@
         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 _iter101 : struct.supervisors)
           {
-            _iter83.write(oprot);
+            _iter101.write(oprot);
           }
           oprot.writeListEnd();
         }
         oprot.writeFieldEnd();
       }
+      if (struct.is_set_nimbus_uptime_secs()) {
+        oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.nimbus_uptime_secs);
+        oprot.writeFieldEnd();
+      }
       if (struct.topologies != null) {
         oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topologies.size()));
-          for (TopologySummary _iter84 : struct.topologies)
+          for (TopologySummary _iter102 : struct.topologies)
           {
-            _iter84.write(oprot);
+            _iter102.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -676,9 +772,9 @@
         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 _iter103 : struct.nimbuses)
           {
-            _iter85.write(oprot);
+            _iter103.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -703,66 +799,79 @@
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.supervisors.size());
-        for (SupervisorSummary _iter86 : struct.supervisors)
+        for (SupervisorSummary _iter104 : struct.supervisors)
         {
-          _iter86.write(oprot);
+          _iter104.write(oprot);
         }
       }
       {
         oprot.writeI32(struct.topologies.size());
-        for (TopologySummary _iter87 : struct.topologies)
+        for (TopologySummary _iter105 : struct.topologies)
         {
-          _iter87.write(oprot);
+          _iter105.write(oprot);
         }
       }
       {
         oprot.writeI32(struct.nimbuses.size());
-        for (NimbusSummary _iter88 : struct.nimbuses)
+        for (NimbusSummary _iter106 : struct.nimbuses)
         {
-          _iter88.write(oprot);
+          _iter106.write(oprot);
         }
       }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_nimbus_uptime_secs()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_nimbus_uptime_secs()) {
+        oprot.writeI32(struct.nimbus_uptime_secs);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _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 _list107 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.supervisors = new ArrayList<SupervisorSummary>(_list107.size);
+        SupervisorSummary _elem108;
+        for (int _i109 = 0; _i109 < _list107.size; ++_i109)
         {
-          _elem90 = new SupervisorSummary();
-          _elem90.read(iprot);
-          struct.supervisors.add(_elem90);
+          _elem108 = new SupervisorSummary();
+          _elem108.read(iprot);
+          struct.supervisors.add(_elem108);
         }
       }
       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 _list110 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topologies = new ArrayList<TopologySummary>(_list110.size);
+        TopologySummary _elem111;
+        for (int _i112 = 0; _i112 < _list110.size; ++_i112)
         {
-          _elem93 = new TopologySummary();
-          _elem93.read(iprot);
-          struct.topologies.add(_elem93);
+          _elem111 = new TopologySummary();
+          _elem111.read(iprot);
+          struct.topologies.add(_elem111);
         }
       }
       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 _list113 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.nimbuses = new ArrayList<NimbusSummary>(_list113.size);
+        NimbusSummary _elem114;
+        for (int _i115 = 0; _i115 < _list113.size; ++_i115)
         {
-          _elem96 = new NimbusSummary();
-          _elem96.read(iprot);
-          struct.nimbuses.add(_elem96);
+          _elem114 = new NimbusSummary();
+          _elem114.read(iprot);
+          struct.nimbuses.add(_elem114);
         }
       }
       struct.set_nimbuses_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.nimbus_uptime_secs = iprot.readI32();
+        struct.set_nimbus_uptime_secs_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
index ce2f5f4..a1b7e2e 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ClusterWorkerHeartbeat.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ClusterWorkerHeartbeat implements org.apache.thrift.TBase<ClusterWorkerHeartbeat, ClusterWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<ClusterWorkerHeartbeat> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterWorkerHeartbeat");
 
@@ -365,10 +365,10 @@
       return get_executor_stats();
 
     case TIME_SECS:
-      return Integer.valueOf(get_time_secs());
+      return get_time_secs();
 
     case UPTIME_SECS:
-      return Integer.valueOf(get_uptime_secs());
+      return get_uptime_secs();
 
     }
     throw new IllegalStateException();
@@ -635,17 +635,17 @@
           case 2: // EXECUTOR_STATS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map454 = iprot.readMapBegin();
-                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map454.size);
-                ExecutorInfo _key455;
-                ExecutorStats _val456;
-                for (int _i457 = 0; _i457 < _map454.size; ++_i457)
+                org.apache.thrift.protocol.TMap _map608 = iprot.readMapBegin();
+                struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map608.size);
+                ExecutorInfo _key609;
+                ExecutorStats _val610;
+                for (int _i611 = 0; _i611 < _map608.size; ++_i611)
                 {
-                  _key455 = new ExecutorInfo();
-                  _key455.read(iprot);
-                  _val456 = new ExecutorStats();
-                  _val456.read(iprot);
-                  struct.executor_stats.put(_key455, _val456);
+                  _key609 = new ExecutorInfo();
+                  _key609.read(iprot);
+                  _val610 = new ExecutorStats();
+                  _val610.read(iprot);
+                  struct.executor_stats.put(_key609, _val610);
                 }
                 iprot.readMapEnd();
               }
@@ -692,10 +692,10 @@
         oprot.writeFieldBegin(EXECUTOR_STATS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.executor_stats.size()));
-          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter458 : struct.executor_stats.entrySet())
+          for (Map.Entry<ExecutorInfo, ExecutorStats> _iter612 : struct.executor_stats.entrySet())
           {
-            _iter458.getKey().write(oprot);
-            _iter458.getValue().write(oprot);
+            _iter612.getKey().write(oprot);
+            _iter612.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -727,10 +727,10 @@
       oprot.writeString(struct.storm_id);
       {
         oprot.writeI32(struct.executor_stats.size());
-        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter459 : struct.executor_stats.entrySet())
+        for (Map.Entry<ExecutorInfo, ExecutorStats> _iter613 : struct.executor_stats.entrySet())
         {
-          _iter459.getKey().write(oprot);
-          _iter459.getValue().write(oprot);
+          _iter613.getKey().write(oprot);
+          _iter613.getValue().write(oprot);
         }
       }
       oprot.writeI32(struct.time_secs);
@@ -743,17 +743,17 @@
       struct.storm_id = iprot.readString();
       struct.set_storm_id_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map460 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executor_stats = new HashMap<ExecutorInfo,ExecutorStats>(2*_map460.size);
-        ExecutorInfo _key461;
-        ExecutorStats _val462;
-        for (int _i463 = 0; _i463 < _map460.size; ++_i463)
+        org.apache.thrift.protocol.TMap _map614 = 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*_map614.size);
+        ExecutorInfo _key615;
+        ExecutorStats _val616;
+        for (int _i617 = 0; _i617 < _map614.size; ++_i617)
         {
-          _key461 = new ExecutorInfo();
-          _key461.read(iprot);
-          _val462 = new ExecutorStats();
-          _val462.read(iprot);
-          struct.executor_stats.put(_key461, _val462);
+          _key615 = new ExecutorInfo();
+          _key615.read(iprot);
+          _val616 = new ExecutorStats();
+          _val616.read(iprot);
+          struct.executor_stats.put(_key615, _val616);
         }
       }
       struct.set_executor_stats_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
new file mode 100644
index 0000000..f1ff590
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/CommonAggregateStats.java
@@ -0,0 +1,902 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("emitted", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("transferred", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("acked", org.apache.thrift.protocol.TType.I64, (short)5);
+  private static final org.apache.thrift.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("failed", org.apache.thrift.protocol.TType.I64, (short)6);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CommonAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CommonAggregateStatsTupleSchemeFactory());
+  }
+
+  private int num_executors; // optional
+  private int num_tasks; // optional
+  private long emitted; // optional
+  private long transferred; // optional
+  private long acked; // optional
+  private long failed; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NUM_EXECUTORS((short)1, "num_executors"),
+    NUM_TASKS((short)2, "num_tasks"),
+    EMITTED((short)3, "emitted"),
+    TRANSFERRED((short)4, "transferred"),
+    ACKED((short)5, "acked"),
+    FAILED((short)6, "failed");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 2: // NUM_TASKS
+          return NUM_TASKS;
+        case 3: // EMITTED
+          return EMITTED;
+        case 4: // TRANSFERRED
+          return TRANSFERRED;
+        case 5: // ACKED
+          return ACKED;
+        case 6: // FAILED
+          return FAILED;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_EXECUTORS_ISSET_ID = 0;
+  private static final int __NUM_TASKS_ISSET_ID = 1;
+  private static final int __EMITTED_ISSET_ID = 2;
+  private static final int __TRANSFERRED_ISSET_ID = 3;
+  private static final int __ACKED_ISSET_ID = 4;
+  private static final int __FAILED_ISSET_ID = 5;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NUM_EXECUTORS,_Fields.NUM_TASKS,_Fields.EMITTED,_Fields.TRANSFERRED,_Fields.ACKED,_Fields.FAILED};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.EMITTED, new org.apache.thrift.meta_data.FieldMetaData("emitted", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("transferred", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.ACKED, new org.apache.thrift.meta_data.FieldMetaData("acked", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.FAILED, new org.apache.thrift.meta_data.FieldMetaData("failed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CommonAggregateStats.class, metaDataMap);
+  }
+
+  public CommonAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CommonAggregateStats(CommonAggregateStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.num_executors = other.num_executors;
+    this.num_tasks = other.num_tasks;
+    this.emitted = other.emitted;
+    this.transferred = other.transferred;
+    this.acked = other.acked;
+    this.failed = other.failed;
+  }
+
+  public CommonAggregateStats deepCopy() {
+    return new CommonAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    set_num_tasks_isSet(false);
+    this.num_tasks = 0;
+    set_emitted_isSet(false);
+    this.emitted = 0;
+    set_transferred_isSet(false);
+    this.transferred = 0;
+    set_acked_isSet(false);
+    this.acked = 0;
+    set_failed_isSet(false);
+    this.failed = 0;
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value);
+  }
+
+  public long get_emitted() {
+    return this.emitted;
+  }
+
+  public void set_emitted(long emitted) {
+    this.emitted = emitted;
+    set_emitted_isSet(true);
+  }
+
+  public void unset_emitted() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EMITTED_ISSET_ID);
+  }
+
+  /** Returns true if field emitted is set (has been assigned a value) and false otherwise */
+  public boolean is_set_emitted() {
+    return EncodingUtils.testBit(__isset_bitfield, __EMITTED_ISSET_ID);
+  }
+
+  public void set_emitted_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EMITTED_ISSET_ID, value);
+  }
+
+  public long get_transferred() {
+    return this.transferred;
+  }
+
+  public void set_transferred(long transferred) {
+    this.transferred = transferred;
+    set_transferred_isSet(true);
+  }
+
+  public void unset_transferred() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TRANSFERRED_ISSET_ID);
+  }
+
+  /** Returns true if field transferred is set (has been assigned a value) and false otherwise */
+  public boolean is_set_transferred() {
+    return EncodingUtils.testBit(__isset_bitfield, __TRANSFERRED_ISSET_ID);
+  }
+
+  public void set_transferred_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TRANSFERRED_ISSET_ID, value);
+  }
+
+  public long get_acked() {
+    return this.acked;
+  }
+
+  public void set_acked(long acked) {
+    this.acked = acked;
+    set_acked_isSet(true);
+  }
+
+  public void unset_acked() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ACKED_ISSET_ID);
+  }
+
+  /** Returns true if field acked is set (has been assigned a value) and false otherwise */
+  public boolean is_set_acked() {
+    return EncodingUtils.testBit(__isset_bitfield, __ACKED_ISSET_ID);
+  }
+
+  public void set_acked_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ACKED_ISSET_ID, value);
+  }
+
+  public long get_failed() {
+    return this.failed;
+  }
+
+  public void set_failed(long failed) {
+    this.failed = failed;
+    set_failed_isSet(true);
+  }
+
+  public void unset_failed() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FAILED_ISSET_ID);
+  }
+
+  /** Returns true if field failed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_failed() {
+    return EncodingUtils.testBit(__isset_bitfield, __FAILED_ISSET_ID);
+  }
+
+  public void set_failed_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FAILED_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case EMITTED:
+      if (value == null) {
+        unset_emitted();
+      } else {
+        set_emitted((Long)value);
+      }
+      break;
+
+    case TRANSFERRED:
+      if (value == null) {
+        unset_transferred();
+      } else {
+        set_transferred((Long)value);
+      }
+      break;
+
+    case ACKED:
+      if (value == null) {
+        unset_acked();
+      } else {
+        set_acked((Long)value);
+      }
+      break;
+
+    case FAILED:
+      if (value == null) {
+        unset_failed();
+      } else {
+        set_failed((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case NUM_TASKS:
+      return get_num_tasks();
+
+    case EMITTED:
+      return get_emitted();
+
+    case TRANSFERRED:
+      return get_transferred();
+
+    case ACKED:
+      return get_acked();
+
+    case FAILED:
+      return get_failed();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case EMITTED:
+      return is_set_emitted();
+    case TRANSFERRED:
+      return is_set_transferred();
+    case ACKED:
+      return is_set_acked();
+    case FAILED:
+      return is_set_failed();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CommonAggregateStats)
+      return this.equals((CommonAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(CommonAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_num_tasks = true && this.is_set_num_tasks();
+    boolean that_present_num_tasks = true && that.is_set_num_tasks();
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_emitted = true && this.is_set_emitted();
+    boolean that_present_emitted = true && that.is_set_emitted();
+    if (this_present_emitted || that_present_emitted) {
+      if (!(this_present_emitted && that_present_emitted))
+        return false;
+      if (this.emitted != that.emitted)
+        return false;
+    }
+
+    boolean this_present_transferred = true && this.is_set_transferred();
+    boolean that_present_transferred = true && that.is_set_transferred();
+    if (this_present_transferred || that_present_transferred) {
+      if (!(this_present_transferred && that_present_transferred))
+        return false;
+      if (this.transferred != that.transferred)
+        return false;
+    }
+
+    boolean this_present_acked = true && this.is_set_acked();
+    boolean that_present_acked = true && that.is_set_acked();
+    if (this_present_acked || that_present_acked) {
+      if (!(this_present_acked && that_present_acked))
+        return false;
+      if (this.acked != that.acked)
+        return false;
+    }
+
+    boolean this_present_failed = true && this.is_set_failed();
+    boolean that_present_failed = true && that.is_set_failed();
+    if (this_present_failed || that_present_failed) {
+      if (!(this_present_failed && that_present_failed))
+        return false;
+      if (this.failed != that.failed)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_num_tasks = true && (is_set_num_tasks());
+    list.add(present_num_tasks);
+    if (present_num_tasks)
+      list.add(num_tasks);
+
+    boolean present_emitted = true && (is_set_emitted());
+    list.add(present_emitted);
+    if (present_emitted)
+      list.add(emitted);
+
+    boolean present_transferred = true && (is_set_transferred());
+    list.add(present_transferred);
+    if (present_transferred)
+      list.add(transferred);
+
+    boolean present_acked = true && (is_set_acked());
+    list.add(present_acked);
+    if (present_acked)
+      list.add(acked);
+
+    boolean present_failed = true && (is_set_failed());
+    list.add(present_failed);
+    if (present_failed)
+      list.add(failed);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(CommonAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_emitted()).compareTo(other.is_set_emitted());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_emitted()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.emitted, other.emitted);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_transferred()).compareTo(other.is_set_transferred());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_transferred()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.transferred, other.transferred);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_acked()).compareTo(other.is_set_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acked()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.acked, other.acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_failed()).compareTo(other.is_set_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_failed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.failed, other.failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("CommonAggregateStats(");
+    boolean first = true;
+
+    if (is_set_num_executors()) {
+      sb.append("num_executors:");
+      sb.append(this.num_executors);
+      first = false;
+    }
+    if (is_set_num_tasks()) {
+      if (!first) sb.append(", ");
+      sb.append("num_tasks:");
+      sb.append(this.num_tasks);
+      first = false;
+    }
+    if (is_set_emitted()) {
+      if (!first) sb.append(", ");
+      sb.append("emitted:");
+      sb.append(this.emitted);
+      first = false;
+    }
+    if (is_set_transferred()) {
+      if (!first) sb.append(", ");
+      sb.append("transferred:");
+      sb.append(this.transferred);
+      first = false;
+    }
+    if (is_set_acked()) {
+      if (!first) sb.append(", ");
+      sb.append("acked:");
+      sb.append(this.acked);
+      first = false;
+    }
+    if (is_set_failed()) {
+      if (!first) sb.append(", ");
+      sb.append("failed:");
+      sb.append(this.failed);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CommonAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public CommonAggregateStatsStandardScheme getScheme() {
+      return new CommonAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class CommonAggregateStatsStandardScheme extends StandardScheme<CommonAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_tasks = iprot.readI32();
+              struct.set_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EMITTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.emitted = iprot.readI64();
+              struct.set_emitted_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TRANSFERRED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.transferred = iprot.readI64();
+              struct.set_transferred_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // ACKED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.acked = iprot.readI64();
+              struct.set_acked_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // FAILED
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.failed = iprot.readI64();
+              struct.set_failed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_num_executors()) {
+        oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+        oprot.writeI32(struct.num_executors);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+        oprot.writeI32(struct.num_tasks);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_emitted()) {
+        oprot.writeFieldBegin(EMITTED_FIELD_DESC);
+        oprot.writeI64(struct.emitted);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_transferred()) {
+        oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC);
+        oprot.writeI64(struct.transferred);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_acked()) {
+        oprot.writeFieldBegin(ACKED_FIELD_DESC);
+        oprot.writeI64(struct.acked);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_failed()) {
+        oprot.writeFieldBegin(FAILED_FIELD_DESC);
+        oprot.writeI64(struct.failed);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CommonAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public CommonAggregateStatsTupleScheme getScheme() {
+      return new CommonAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class CommonAggregateStatsTupleScheme extends TupleScheme<CommonAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_num_executors()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_num_tasks()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_emitted()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_transferred()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_acked()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_failed()) {
+        optionals.set(5);
+      }
+      oprot.writeBitSet(optionals, 6);
+      if (struct.is_set_num_executors()) {
+        oprot.writeI32(struct.num_executors);
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeI32(struct.num_tasks);
+      }
+      if (struct.is_set_emitted()) {
+        oprot.writeI64(struct.emitted);
+      }
+      if (struct.is_set_transferred()) {
+        oprot.writeI64(struct.transferred);
+      }
+      if (struct.is_set_acked()) {
+        oprot.writeI64(struct.acked);
+      }
+      if (struct.is_set_failed()) {
+        oprot.writeI64(struct.failed);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CommonAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(6);
+      if (incoming.get(0)) {
+        struct.num_executors = iprot.readI32();
+        struct.set_num_executors_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.num_tasks = iprot.readI32();
+        struct.set_num_tasks_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.emitted = iprot.readI64();
+        struct.set_emitted_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.transferred = iprot.readI64();
+        struct.set_transferred_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.acked = iprot.readI64();
+        struct.set_acked_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.failed = iprot.readI64();
+        struct.set_failed_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
new file mode 100644
index 0000000..eea060f
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentAggregateStats.java
@@ -0,0 +1,752 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField COMMON_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("common_stats", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField SPECIFIC_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("specific_stats", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField LAST_ERROR_FIELD_DESC = new org.apache.thrift.protocol.TField("last_error", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ComponentAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ComponentAggregateStatsTupleSchemeFactory());
+  }
+
+  private ComponentType type; // optional
+  private CommonAggregateStats common_stats; // optional
+  private SpecificAggregateStats specific_stats; // optional
+  private ErrorInfo last_error; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see ComponentType
+     */
+    TYPE((short)1, "type"),
+    COMMON_STATS((short)2, "common_stats"),
+    SPECIFIC_STATS((short)3, "specific_stats"),
+    LAST_ERROR((short)4, "last_error");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TYPE
+          return TYPE;
+        case 2: // COMMON_STATS
+          return COMMON_STATS;
+        case 3: // SPECIFIC_STATS
+          return SPECIFIC_STATS;
+        case 4: // LAST_ERROR
+          return LAST_ERROR;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.TYPE,_Fields.COMMON_STATS,_Fields.SPECIFIC_STATS,_Fields.LAST_ERROR};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ComponentType.class)));
+    tmpMap.put(_Fields.COMMON_STATS, new org.apache.thrift.meta_data.FieldMetaData("common_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CommonAggregateStats.class)));
+    tmpMap.put(_Fields.SPECIFIC_STATS, new org.apache.thrift.meta_data.FieldMetaData("specific_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpecificAggregateStats.class)));
+    tmpMap.put(_Fields.LAST_ERROR, new org.apache.thrift.meta_data.FieldMetaData("last_error", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentAggregateStats.class, metaDataMap);
+  }
+
+  public ComponentAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ComponentAggregateStats(ComponentAggregateStats other) {
+    if (other.is_set_type()) {
+      this.type = other.type;
+    }
+    if (other.is_set_common_stats()) {
+      this.common_stats = new CommonAggregateStats(other.common_stats);
+    }
+    if (other.is_set_specific_stats()) {
+      this.specific_stats = new SpecificAggregateStats(other.specific_stats);
+    }
+    if (other.is_set_last_error()) {
+      this.last_error = new ErrorInfo(other.last_error);
+    }
+  }
+
+  public ComponentAggregateStats deepCopy() {
+    return new ComponentAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.type = null;
+    this.common_stats = null;
+    this.specific_stats = null;
+    this.last_error = null;
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public ComponentType get_type() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public void set_type(ComponentType type) {
+    this.type = type;
+  }
+
+  public void unset_type() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_type() {
+    return this.type != null;
+  }
+
+  public void set_type_isSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public CommonAggregateStats get_common_stats() {
+    return this.common_stats;
+  }
+
+  public void set_common_stats(CommonAggregateStats common_stats) {
+    this.common_stats = common_stats;
+  }
+
+  public void unset_common_stats() {
+    this.common_stats = null;
+  }
+
+  /** Returns true if field common_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_common_stats() {
+    return this.common_stats != null;
+  }
+
+  public void set_common_stats_isSet(boolean value) {
+    if (!value) {
+      this.common_stats = null;
+    }
+  }
+
+  public SpecificAggregateStats get_specific_stats() {
+    return this.specific_stats;
+  }
+
+  public void set_specific_stats(SpecificAggregateStats specific_stats) {
+    this.specific_stats = specific_stats;
+  }
+
+  public void unset_specific_stats() {
+    this.specific_stats = null;
+  }
+
+  /** Returns true if field specific_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_specific_stats() {
+    return this.specific_stats != null;
+  }
+
+  public void set_specific_stats_isSet(boolean value) {
+    if (!value) {
+      this.specific_stats = null;
+    }
+  }
+
+  public ErrorInfo get_last_error() {
+    return this.last_error;
+  }
+
+  public void set_last_error(ErrorInfo last_error) {
+    this.last_error = last_error;
+  }
+
+  public void unset_last_error() {
+    this.last_error = null;
+  }
+
+  /** Returns true if field last_error is set (has been assigned a value) and false otherwise */
+  public boolean is_set_last_error() {
+    return this.last_error != null;
+  }
+
+  public void set_last_error_isSet(boolean value) {
+    if (!value) {
+      this.last_error = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TYPE:
+      if (value == null) {
+        unset_type();
+      } else {
+        set_type((ComponentType)value);
+      }
+      break;
+
+    case COMMON_STATS:
+      if (value == null) {
+        unset_common_stats();
+      } else {
+        set_common_stats((CommonAggregateStats)value);
+      }
+      break;
+
+    case SPECIFIC_STATS:
+      if (value == null) {
+        unset_specific_stats();
+      } else {
+        set_specific_stats((SpecificAggregateStats)value);
+      }
+      break;
+
+    case LAST_ERROR:
+      if (value == null) {
+        unset_last_error();
+      } else {
+        set_last_error((ErrorInfo)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TYPE:
+      return get_type();
+
+    case COMMON_STATS:
+      return get_common_stats();
+
+    case SPECIFIC_STATS:
+      return get_specific_stats();
+
+    case LAST_ERROR:
+      return get_last_error();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TYPE:
+      return is_set_type();
+    case COMMON_STATS:
+      return is_set_common_stats();
+    case SPECIFIC_STATS:
+      return is_set_specific_stats();
+    case LAST_ERROR:
+      return is_set_last_error();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ComponentAggregateStats)
+      return this.equals((ComponentAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(ComponentAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_type = true && this.is_set_type();
+    boolean that_present_type = true && that.is_set_type();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_common_stats = true && this.is_set_common_stats();
+    boolean that_present_common_stats = true && that.is_set_common_stats();
+    if (this_present_common_stats || that_present_common_stats) {
+      if (!(this_present_common_stats && that_present_common_stats))
+        return false;
+      if (!this.common_stats.equals(that.common_stats))
+        return false;
+    }
+
+    boolean this_present_specific_stats = true && this.is_set_specific_stats();
+    boolean that_present_specific_stats = true && that.is_set_specific_stats();
+    if (this_present_specific_stats || that_present_specific_stats) {
+      if (!(this_present_specific_stats && that_present_specific_stats))
+        return false;
+      if (!this.specific_stats.equals(that.specific_stats))
+        return false;
+    }
+
+    boolean this_present_last_error = true && this.is_set_last_error();
+    boolean that_present_last_error = true && that.is_set_last_error();
+    if (this_present_last_error || that_present_last_error) {
+      if (!(this_present_last_error && that_present_last_error))
+        return false;
+      if (!this.last_error.equals(that.last_error))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_type = true && (is_set_type());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
+    boolean present_common_stats = true && (is_set_common_stats());
+    list.add(present_common_stats);
+    if (present_common_stats)
+      list.add(common_stats);
+
+    boolean present_specific_stats = true && (is_set_specific_stats());
+    list.add(present_specific_stats);
+    if (present_specific_stats)
+      list.add(specific_stats);
+
+    boolean present_last_error = true && (is_set_last_error());
+    list.add(present_last_error);
+    if (present_last_error)
+      list.add(last_error);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ComponentAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common_stats()).compareTo(other.is_set_common_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common_stats, other.common_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_specific_stats()).compareTo(other.is_set_specific_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_specific_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.specific_stats, other.specific_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_last_error()).compareTo(other.is_set_last_error());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_last_error()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.last_error, other.last_error);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ComponentAggregateStats(");
+    boolean first = true;
+
+    if (is_set_type()) {
+      sb.append("type:");
+      if (this.type == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.type);
+      }
+      first = false;
+    }
+    if (is_set_common_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("common_stats:");
+      if (this.common_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.common_stats);
+      }
+      first = false;
+    }
+    if (is_set_specific_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("specific_stats:");
+      if (this.specific_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.specific_stats);
+      }
+      first = false;
+    }
+    if (is_set_last_error()) {
+      if (!first) sb.append(", ");
+      sb.append("last_error:");
+      if (this.last_error == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.last_error);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (common_stats != null) {
+      common_stats.validate();
+    }
+    if (last_error != null) {
+      last_error.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ComponentAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public ComponentAggregateStatsStandardScheme getScheme() {
+      return new ComponentAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class ComponentAggregateStatsStandardScheme extends StandardScheme<ComponentAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32());
+              struct.set_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMMON_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.common_stats = new CommonAggregateStats();
+              struct.common_stats.read(iprot);
+              struct.set_common_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // SPECIFIC_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.specific_stats = new SpecificAggregateStats();
+              struct.specific_stats.read(iprot);
+              struct.set_specific_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // LAST_ERROR
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.last_error = new ErrorInfo();
+              struct.last_error.read(iprot);
+              struct.set_last_error_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.type != null) {
+        if (struct.is_set_type()) {
+          oprot.writeFieldBegin(TYPE_FIELD_DESC);
+          oprot.writeI32(struct.type.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.common_stats != null) {
+        if (struct.is_set_common_stats()) {
+          oprot.writeFieldBegin(COMMON_STATS_FIELD_DESC);
+          struct.common_stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.specific_stats != null) {
+        if (struct.is_set_specific_stats()) {
+          oprot.writeFieldBegin(SPECIFIC_STATS_FIELD_DESC);
+          struct.specific_stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.last_error != null) {
+        if (struct.is_set_last_error()) {
+          oprot.writeFieldBegin(LAST_ERROR_FIELD_DESC);
+          struct.last_error.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ComponentAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public ComponentAggregateStatsTupleScheme getScheme() {
+      return new ComponentAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class ComponentAggregateStatsTupleScheme extends TupleScheme<ComponentAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_type()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_common_stats()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_specific_stats()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_last_error()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_type()) {
+        oprot.writeI32(struct.type.getValue());
+      }
+      if (struct.is_set_common_stats()) {
+        struct.common_stats.write(oprot);
+      }
+      if (struct.is_set_specific_stats()) {
+        struct.specific_stats.write(oprot);
+      }
+      if (struct.is_set_last_error()) {
+        struct.last_error.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ComponentAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32());
+        struct.set_type_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.common_stats = new CommonAggregateStats();
+        struct.common_stats.read(iprot);
+        struct.set_common_stats_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.specific_stats = new SpecificAggregateStats();
+        struct.specific_stats.read(iprot);
+        struct.set_specific_stats_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.last_error = new ErrorInfo();
+        struct.last_error.read(iprot);
+        struct.set_last_error_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java b/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
index 6868603..9950756 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentCommon.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ComponentCommon implements org.apache.thrift.TBase<ComponentCommon, ComponentCommon._Fields>, java.io.Serializable, Cloneable, Comparable<ComponentCommon> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentCommon");
 
@@ -386,7 +386,7 @@
       return get_streams();
 
     case PARALLELISM_HINT:
-      return Integer.valueOf(get_parallelism_hint());
+      return get_parallelism_hint();
 
     case JSON_CONF:
       return get_json_conf();
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentObject.java b/storm-core/src/jvm/backtype/storm/generated/ComponentObject.java
index 575069f..722e7db 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ComponentObject.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentObject.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
new file mode 100644
index 0000000..6152d02
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentPageInfo.java
@@ -0,0 +1,2194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField COMPONENT_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("component_type", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_name", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_stats", org.apache.thrift.protocol.TType.MAP, (short)7);
+  private static final org.apache.thrift.protocol.TField GSID_TO_INPUT_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("gsid_to_input_stats", org.apache.thrift.protocol.TType.MAP, (short)8);
+  private static final org.apache.thrift.protocol.TField SID_TO_OUTPUT_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("sid_to_output_stats", org.apache.thrift.protocol.TType.MAP, (short)9);
+  private static final org.apache.thrift.protocol.TField EXEC_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("exec_stats", org.apache.thrift.protocol.TType.LIST, (short)10);
+  private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.LIST, (short)11);
+  private static final org.apache.thrift.protocol.TField EVENTLOG_HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("eventlog_host", org.apache.thrift.protocol.TType.STRING, (short)12);
+  private static final org.apache.thrift.protocol.TField EVENTLOG_PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("eventlog_port", org.apache.thrift.protocol.TType.I32, (short)13);
+  private static final org.apache.thrift.protocol.TField DEBUG_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("debug_options", org.apache.thrift.protocol.TType.STRUCT, (short)14);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_status", org.apache.thrift.protocol.TType.STRING, (short)15);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ComponentPageInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ComponentPageInfoTupleSchemeFactory());
+  }
+
+  private String component_id; // required
+  private ComponentType component_type; // required
+  private String topology_id; // optional
+  private String topology_name; // optional
+  private int num_executors; // optional
+  private int num_tasks; // optional
+  private Map<String,ComponentAggregateStats> window_to_stats; // optional
+  private Map<GlobalStreamId,ComponentAggregateStats> gsid_to_input_stats; // optional
+  private Map<String,ComponentAggregateStats> sid_to_output_stats; // optional
+  private List<ExecutorAggregateStats> exec_stats; // optional
+  private List<ErrorInfo> errors; // optional
+  private String eventlog_host; // optional
+  private int eventlog_port; // optional
+  private DebugOptions debug_options; // optional
+  private String topology_status; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    COMPONENT_ID((short)1, "component_id"),
+    /**
+     * 
+     * @see ComponentType
+     */
+    COMPONENT_TYPE((short)2, "component_type"),
+    TOPOLOGY_ID((short)3, "topology_id"),
+    TOPOLOGY_NAME((short)4, "topology_name"),
+    NUM_EXECUTORS((short)5, "num_executors"),
+    NUM_TASKS((short)6, "num_tasks"),
+    WINDOW_TO_STATS((short)7, "window_to_stats"),
+    GSID_TO_INPUT_STATS((short)8, "gsid_to_input_stats"),
+    SID_TO_OUTPUT_STATS((short)9, "sid_to_output_stats"),
+    EXEC_STATS((short)10, "exec_stats"),
+    ERRORS((short)11, "errors"),
+    EVENTLOG_HOST((short)12, "eventlog_host"),
+    EVENTLOG_PORT((short)13, "eventlog_port"),
+    DEBUG_OPTIONS((short)14, "debug_options"),
+    TOPOLOGY_STATUS((short)15, "topology_status");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // COMPONENT_ID
+          return COMPONENT_ID;
+        case 2: // COMPONENT_TYPE
+          return COMPONENT_TYPE;
+        case 3: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 4: // TOPOLOGY_NAME
+          return TOPOLOGY_NAME;
+        case 5: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 6: // NUM_TASKS
+          return NUM_TASKS;
+        case 7: // WINDOW_TO_STATS
+          return WINDOW_TO_STATS;
+        case 8: // GSID_TO_INPUT_STATS
+          return GSID_TO_INPUT_STATS;
+        case 9: // SID_TO_OUTPUT_STATS
+          return SID_TO_OUTPUT_STATS;
+        case 10: // EXEC_STATS
+          return EXEC_STATS;
+        case 11: // ERRORS
+          return ERRORS;
+        case 12: // EVENTLOG_HOST
+          return EVENTLOG_HOST;
+        case 13: // EVENTLOG_PORT
+          return EVENTLOG_PORT;
+        case 14: // DEBUG_OPTIONS
+          return DEBUG_OPTIONS;
+        case 15: // TOPOLOGY_STATUS
+          return TOPOLOGY_STATUS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_EXECUTORS_ISSET_ID = 0;
+  private static final int __NUM_TASKS_ISSET_ID = 1;
+  private static final int __EVENTLOG_PORT_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.TOPOLOGY_ID,_Fields.TOPOLOGY_NAME,_Fields.NUM_EXECUTORS,_Fields.NUM_TASKS,_Fields.WINDOW_TO_STATS,_Fields.GSID_TO_INPUT_STATS,_Fields.SID_TO_OUTPUT_STATS,_Fields.EXEC_STATS,_Fields.ERRORS,_Fields.EVENTLOG_HOST,_Fields.EVENTLOG_PORT,_Fields.DEBUG_OPTIONS,_Fields.TOPOLOGY_STATUS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COMPONENT_TYPE, new org.apache.thrift.meta_data.FieldMetaData("component_type", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ComponentType.class)));
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topology_name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.WINDOW_TO_STATS, new org.apache.thrift.meta_data.FieldMetaData("window_to_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.GSID_TO_INPUT_STATS, new org.apache.thrift.meta_data.FieldMetaData("gsid_to_input_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.SID_TO_OUTPUT_STATS, new org.apache.thrift.meta_data.FieldMetaData("sid_to_output_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.EXEC_STATS, new org.apache.thrift.meta_data.FieldMetaData("exec_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorAggregateStats.class))));
+    tmpMap.put(_Fields.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class))));
+    tmpMap.put(_Fields.EVENTLOG_HOST, new org.apache.thrift.meta_data.FieldMetaData("eventlog_host", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EVENTLOG_PORT, new org.apache.thrift.meta_data.FieldMetaData("eventlog_port", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.DEBUG_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("debug_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class)));
+    tmpMap.put(_Fields.TOPOLOGY_STATUS, new org.apache.thrift.meta_data.FieldMetaData("topology_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentPageInfo.class, metaDataMap);
+  }
+
+  public ComponentPageInfo() {
+  }
+
+  public ComponentPageInfo(
+    String component_id,
+    ComponentType component_type)
+  {
+    this();
+    this.component_id = component_id;
+    this.component_type = component_type;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ComponentPageInfo(ComponentPageInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_component_id()) {
+      this.component_id = other.component_id;
+    }
+    if (other.is_set_component_type()) {
+      this.component_type = other.component_type;
+    }
+    if (other.is_set_topology_id()) {
+      this.topology_id = other.topology_id;
+    }
+    if (other.is_set_topology_name()) {
+      this.topology_name = other.topology_name;
+    }
+    this.num_executors = other.num_executors;
+    this.num_tasks = other.num_tasks;
+    if (other.is_set_window_to_stats()) {
+      Map<String,ComponentAggregateStats> __this__window_to_stats = new HashMap<String,ComponentAggregateStats>(other.window_to_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.window_to_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__window_to_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__window_to_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__window_to_stats.put(__this__window_to_stats_copy_key, __this__window_to_stats_copy_value);
+      }
+      this.window_to_stats = __this__window_to_stats;
+    }
+    if (other.is_set_gsid_to_input_stats()) {
+      Map<GlobalStreamId,ComponentAggregateStats> __this__gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(other.gsid_to_input_stats.size());
+      for (Map.Entry<GlobalStreamId, ComponentAggregateStats> other_element : other.gsid_to_input_stats.entrySet()) {
+
+        GlobalStreamId other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        GlobalStreamId __this__gsid_to_input_stats_copy_key = new GlobalStreamId(other_element_key);
+
+        ComponentAggregateStats __this__gsid_to_input_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__gsid_to_input_stats.put(__this__gsid_to_input_stats_copy_key, __this__gsid_to_input_stats_copy_value);
+      }
+      this.gsid_to_input_stats = __this__gsid_to_input_stats;
+    }
+    if (other.is_set_sid_to_output_stats()) {
+      Map<String,ComponentAggregateStats> __this__sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(other.sid_to_output_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.sid_to_output_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__sid_to_output_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__sid_to_output_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__sid_to_output_stats.put(__this__sid_to_output_stats_copy_key, __this__sid_to_output_stats_copy_value);
+      }
+      this.sid_to_output_stats = __this__sid_to_output_stats;
+    }
+    if (other.is_set_exec_stats()) {
+      List<ExecutorAggregateStats> __this__exec_stats = new ArrayList<ExecutorAggregateStats>(other.exec_stats.size());
+      for (ExecutorAggregateStats other_element : other.exec_stats) {
+        __this__exec_stats.add(new ExecutorAggregateStats(other_element));
+      }
+      this.exec_stats = __this__exec_stats;
+    }
+    if (other.is_set_errors()) {
+      List<ErrorInfo> __this__errors = new ArrayList<ErrorInfo>(other.errors.size());
+      for (ErrorInfo other_element : other.errors) {
+        __this__errors.add(new ErrorInfo(other_element));
+      }
+      this.errors = __this__errors;
+    }
+    if (other.is_set_eventlog_host()) {
+      this.eventlog_host = other.eventlog_host;
+    }
+    this.eventlog_port = other.eventlog_port;
+    if (other.is_set_debug_options()) {
+      this.debug_options = new DebugOptions(other.debug_options);
+    }
+    if (other.is_set_topology_status()) {
+      this.topology_status = other.topology_status;
+    }
+  }
+
+  public ComponentPageInfo deepCopy() {
+    return new ComponentPageInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.component_id = null;
+    this.component_type = null;
+    this.topology_id = null;
+    this.topology_name = null;
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    set_num_tasks_isSet(false);
+    this.num_tasks = 0;
+    this.window_to_stats = null;
+    this.gsid_to_input_stats = null;
+    this.sid_to_output_stats = null;
+    this.exec_stats = null;
+    this.errors = null;
+    this.eventlog_host = null;
+    set_eventlog_port_isSet(false);
+    this.eventlog_port = 0;
+    this.debug_options = null;
+    this.topology_status = null;
+  }
+
+  public String get_component_id() {
+    return this.component_id;
+  }
+
+  public void set_component_id(String component_id) {
+    this.component_id = component_id;
+  }
+
+  public void unset_component_id() {
+    this.component_id = null;
+  }
+
+  /** Returns true if field component_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_id() {
+    return this.component_id != null;
+  }
+
+  public void set_component_id_isSet(boolean value) {
+    if (!value) {
+      this.component_id = null;
+    }
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public ComponentType get_component_type() {
+    return this.component_type;
+  }
+
+  /**
+   * 
+   * @see ComponentType
+   */
+  public void set_component_type(ComponentType component_type) {
+    this.component_type = component_type;
+  }
+
+  public void unset_component_type() {
+    this.component_type = null;
+  }
+
+  /** Returns true if field component_type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_component_type() {
+    return this.component_type != null;
+  }
+
+  public void set_component_type_isSet(boolean value) {
+    if (!value) {
+      this.component_type = null;
+    }
+  }
+
+  public String get_topology_id() {
+    return this.topology_id;
+  }
+
+  public void set_topology_id(String topology_id) {
+    this.topology_id = topology_id;
+  }
+
+  public void unset_topology_id() {
+    this.topology_id = null;
+  }
+
+  /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_id() {
+    return this.topology_id != null;
+  }
+
+  public void set_topology_id_isSet(boolean value) {
+    if (!value) {
+      this.topology_id = null;
+    }
+  }
+
+  public String get_topology_name() {
+    return this.topology_name;
+  }
+
+  public void set_topology_name(String topology_name) {
+    this.topology_name = topology_name;
+  }
+
+  public void unset_topology_name() {
+    this.topology_name = null;
+  }
+
+  /** Returns true if field topology_name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_name() {
+    return this.topology_name != null;
+  }
+
+  public void set_topology_name_isSet(boolean value) {
+    if (!value) {
+      this.topology_name = null;
+    }
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value);
+  }
+
+  public int get_window_to_stats_size() {
+    return (this.window_to_stats == null) ? 0 : this.window_to_stats.size();
+  }
+
+  public void put_to_window_to_stats(String key, ComponentAggregateStats val) {
+    if (this.window_to_stats == null) {
+      this.window_to_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.window_to_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_window_to_stats() {
+    return this.window_to_stats;
+  }
+
+  public void set_window_to_stats(Map<String,ComponentAggregateStats> window_to_stats) {
+    this.window_to_stats = window_to_stats;
+  }
+
+  public void unset_window_to_stats() {
+    this.window_to_stats = null;
+  }
+
+  /** Returns true if field window_to_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_stats() {
+    return this.window_to_stats != null;
+  }
+
+  public void set_window_to_stats_isSet(boolean value) {
+    if (!value) {
+      this.window_to_stats = null;
+    }
+  }
+
+  public int get_gsid_to_input_stats_size() {
+    return (this.gsid_to_input_stats == null) ? 0 : this.gsid_to_input_stats.size();
+  }
+
+  public void put_to_gsid_to_input_stats(GlobalStreamId key, ComponentAggregateStats val) {
+    if (this.gsid_to_input_stats == null) {
+      this.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>();
+    }
+    this.gsid_to_input_stats.put(key, val);
+  }
+
+  public Map<GlobalStreamId,ComponentAggregateStats> get_gsid_to_input_stats() {
+    return this.gsid_to_input_stats;
+  }
+
+  public void set_gsid_to_input_stats(Map<GlobalStreamId,ComponentAggregateStats> gsid_to_input_stats) {
+    this.gsid_to_input_stats = gsid_to_input_stats;
+  }
+
+  public void unset_gsid_to_input_stats() {
+    this.gsid_to_input_stats = null;
+  }
+
+  /** Returns true if field gsid_to_input_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_gsid_to_input_stats() {
+    return this.gsid_to_input_stats != null;
+  }
+
+  public void set_gsid_to_input_stats_isSet(boolean value) {
+    if (!value) {
+      this.gsid_to_input_stats = null;
+    }
+  }
+
+  public int get_sid_to_output_stats_size() {
+    return (this.sid_to_output_stats == null) ? 0 : this.sid_to_output_stats.size();
+  }
+
+  public void put_to_sid_to_output_stats(String key, ComponentAggregateStats val) {
+    if (this.sid_to_output_stats == null) {
+      this.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.sid_to_output_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_sid_to_output_stats() {
+    return this.sid_to_output_stats;
+  }
+
+  public void set_sid_to_output_stats(Map<String,ComponentAggregateStats> sid_to_output_stats) {
+    this.sid_to_output_stats = sid_to_output_stats;
+  }
+
+  public void unset_sid_to_output_stats() {
+    this.sid_to_output_stats = null;
+  }
+
+  /** Returns true if field sid_to_output_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sid_to_output_stats() {
+    return this.sid_to_output_stats != null;
+  }
+
+  public void set_sid_to_output_stats_isSet(boolean value) {
+    if (!value) {
+      this.sid_to_output_stats = null;
+    }
+  }
+
+  public int get_exec_stats_size() {
+    return (this.exec_stats == null) ? 0 : this.exec_stats.size();
+  }
+
+  public java.util.Iterator<ExecutorAggregateStats> get_exec_stats_iterator() {
+    return (this.exec_stats == null) ? null : this.exec_stats.iterator();
+  }
+
+  public void add_to_exec_stats(ExecutorAggregateStats elem) {
+    if (this.exec_stats == null) {
+      this.exec_stats = new ArrayList<ExecutorAggregateStats>();
+    }
+    this.exec_stats.add(elem);
+  }
+
+  public List<ExecutorAggregateStats> get_exec_stats() {
+    return this.exec_stats;
+  }
+
+  public void set_exec_stats(List<ExecutorAggregateStats> exec_stats) {
+    this.exec_stats = exec_stats;
+  }
+
+  public void unset_exec_stats() {
+    this.exec_stats = null;
+  }
+
+  /** Returns true if field exec_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_exec_stats() {
+    return this.exec_stats != null;
+  }
+
+  public void set_exec_stats_isSet(boolean value) {
+    if (!value) {
+      this.exec_stats = null;
+    }
+  }
+
+  public int get_errors_size() {
+    return (this.errors == null) ? 0 : this.errors.size();
+  }
+
+  public java.util.Iterator<ErrorInfo> get_errors_iterator() {
+    return (this.errors == null) ? null : this.errors.iterator();
+  }
+
+  public void add_to_errors(ErrorInfo elem) {
+    if (this.errors == null) {
+      this.errors = new ArrayList<ErrorInfo>();
+    }
+    this.errors.add(elem);
+  }
+
+  public List<ErrorInfo> get_errors() {
+    return this.errors;
+  }
+
+  public void set_errors(List<ErrorInfo> errors) {
+    this.errors = errors;
+  }
+
+  public void unset_errors() {
+    this.errors = null;
+  }
+
+  /** Returns true if field errors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_errors() {
+    return this.errors != null;
+  }
+
+  public void set_errors_isSet(boolean value) {
+    if (!value) {
+      this.errors = null;
+    }
+  }
+
+  public String get_eventlog_host() {
+    return this.eventlog_host;
+  }
+
+  public void set_eventlog_host(String eventlog_host) {
+    this.eventlog_host = eventlog_host;
+  }
+
+  public void unset_eventlog_host() {
+    this.eventlog_host = null;
+  }
+
+  /** Returns true if field eventlog_host is set (has been assigned a value) and false otherwise */
+  public boolean is_set_eventlog_host() {
+    return this.eventlog_host != null;
+  }
+
+  public void set_eventlog_host_isSet(boolean value) {
+    if (!value) {
+      this.eventlog_host = null;
+    }
+  }
+
+  public int get_eventlog_port() {
+    return this.eventlog_port;
+  }
+
+  public void set_eventlog_port(int eventlog_port) {
+    this.eventlog_port = eventlog_port;
+    set_eventlog_port_isSet(true);
+  }
+
+  public void unset_eventlog_port() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID);
+  }
+
+  /** Returns true if field eventlog_port is set (has been assigned a value) and false otherwise */
+  public boolean is_set_eventlog_port() {
+    return EncodingUtils.testBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID);
+  }
+
+  public void set_eventlog_port_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __EVENTLOG_PORT_ISSET_ID, value);
+  }
+
+  public DebugOptions get_debug_options() {
+    return this.debug_options;
+  }
+
+  public void set_debug_options(DebugOptions debug_options) {
+    this.debug_options = debug_options;
+  }
+
+  public void unset_debug_options() {
+    this.debug_options = null;
+  }
+
+  /** Returns true if field debug_options is set (has been assigned a value) and false otherwise */
+  public boolean is_set_debug_options() {
+    return this.debug_options != null;
+  }
+
+  public void set_debug_options_isSet(boolean value) {
+    if (!value) {
+      this.debug_options = null;
+    }
+  }
+
+  public String get_topology_status() {
+    return this.topology_status;
+  }
+
+  public void set_topology_status(String topology_status) {
+    this.topology_status = topology_status;
+  }
+
+  public void unset_topology_status() {
+    this.topology_status = null;
+  }
+
+  /** Returns true if field topology_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_status() {
+    return this.topology_status != null;
+  }
+
+  public void set_topology_status_isSet(boolean value) {
+    if (!value) {
+      this.topology_status = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case COMPONENT_ID:
+      if (value == null) {
+        unset_component_id();
+      } else {
+        set_component_id((String)value);
+      }
+      break;
+
+    case COMPONENT_TYPE:
+      if (value == null) {
+        unset_component_type();
+      } else {
+        set_component_type((ComponentType)value);
+      }
+      break;
+
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topology_id();
+      } else {
+        set_topology_id((String)value);
+      }
+      break;
+
+    case TOPOLOGY_NAME:
+      if (value == null) {
+        unset_topology_name();
+      } else {
+        set_topology_name((String)value);
+      }
+      break;
+
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case WINDOW_TO_STATS:
+      if (value == null) {
+        unset_window_to_stats();
+      } else {
+        set_window_to_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case GSID_TO_INPUT_STATS:
+      if (value == null) {
+        unset_gsid_to_input_stats();
+      } else {
+        set_gsid_to_input_stats((Map<GlobalStreamId,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case SID_TO_OUTPUT_STATS:
+      if (value == null) {
+        unset_sid_to_output_stats();
+      } else {
+        set_sid_to_output_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case EXEC_STATS:
+      if (value == null) {
+        unset_exec_stats();
+      } else {
+        set_exec_stats((List<ExecutorAggregateStats>)value);
+      }
+      break;
+
+    case ERRORS:
+      if (value == null) {
+        unset_errors();
+      } else {
+        set_errors((List<ErrorInfo>)value);
+      }
+      break;
+
+    case EVENTLOG_HOST:
+      if (value == null) {
+        unset_eventlog_host();
+      } else {
+        set_eventlog_host((String)value);
+      }
+      break;
+
+    case EVENTLOG_PORT:
+      if (value == null) {
+        unset_eventlog_port();
+      } else {
+        set_eventlog_port((Integer)value);
+      }
+      break;
+
+    case DEBUG_OPTIONS:
+      if (value == null) {
+        unset_debug_options();
+      } else {
+        set_debug_options((DebugOptions)value);
+      }
+      break;
+
+    case TOPOLOGY_STATUS:
+      if (value == null) {
+        unset_topology_status();
+      } else {
+        set_topology_status((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case COMPONENT_ID:
+      return get_component_id();
+
+    case COMPONENT_TYPE:
+      return get_component_type();
+
+    case TOPOLOGY_ID:
+      return get_topology_id();
+
+    case TOPOLOGY_NAME:
+      return get_topology_name();
+
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case NUM_TASKS:
+      return get_num_tasks();
+
+    case WINDOW_TO_STATS:
+      return get_window_to_stats();
+
+    case GSID_TO_INPUT_STATS:
+      return get_gsid_to_input_stats();
+
+    case SID_TO_OUTPUT_STATS:
+      return get_sid_to_output_stats();
+
+    case EXEC_STATS:
+      return get_exec_stats();
+
+    case ERRORS:
+      return get_errors();
+
+    case EVENTLOG_HOST:
+      return get_eventlog_host();
+
+    case EVENTLOG_PORT:
+      return get_eventlog_port();
+
+    case DEBUG_OPTIONS:
+      return get_debug_options();
+
+    case TOPOLOGY_STATUS:
+      return get_topology_status();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case COMPONENT_ID:
+      return is_set_component_id();
+    case COMPONENT_TYPE:
+      return is_set_component_type();
+    case TOPOLOGY_ID:
+      return is_set_topology_id();
+    case TOPOLOGY_NAME:
+      return is_set_topology_name();
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case WINDOW_TO_STATS:
+      return is_set_window_to_stats();
+    case GSID_TO_INPUT_STATS:
+      return is_set_gsid_to_input_stats();
+    case SID_TO_OUTPUT_STATS:
+      return is_set_sid_to_output_stats();
+    case EXEC_STATS:
+      return is_set_exec_stats();
+    case ERRORS:
+      return is_set_errors();
+    case EVENTLOG_HOST:
+      return is_set_eventlog_host();
+    case EVENTLOG_PORT:
+      return is_set_eventlog_port();
+    case DEBUG_OPTIONS:
+      return is_set_debug_options();
+    case TOPOLOGY_STATUS:
+      return is_set_topology_status();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ComponentPageInfo)
+      return this.equals((ComponentPageInfo)that);
+    return false;
+  }
+
+  public boolean equals(ComponentPageInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_component_id = true && this.is_set_component_id();
+    boolean that_present_component_id = true && that.is_set_component_id();
+    if (this_present_component_id || that_present_component_id) {
+      if (!(this_present_component_id && that_present_component_id))
+        return false;
+      if (!this.component_id.equals(that.component_id))
+        return false;
+    }
+
+    boolean this_present_component_type = true && this.is_set_component_type();
+    boolean that_present_component_type = true && that.is_set_component_type();
+    if (this_present_component_type || that_present_component_type) {
+      if (!(this_present_component_type && that_present_component_type))
+        return false;
+      if (!this.component_type.equals(that.component_type))
+        return false;
+    }
+
+    boolean this_present_topology_id = true && this.is_set_topology_id();
+    boolean that_present_topology_id = true && that.is_set_topology_id();
+    if (this_present_topology_id || that_present_topology_id) {
+      if (!(this_present_topology_id && that_present_topology_id))
+        return false;
+      if (!this.topology_id.equals(that.topology_id))
+        return false;
+    }
+
+    boolean this_present_topology_name = true && this.is_set_topology_name();
+    boolean that_present_topology_name = true && that.is_set_topology_name();
+    if (this_present_topology_name || that_present_topology_name) {
+      if (!(this_present_topology_name && that_present_topology_name))
+        return false;
+      if (!this.topology_name.equals(that.topology_name))
+        return false;
+    }
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_num_tasks = true && this.is_set_num_tasks();
+    boolean that_present_num_tasks = true && that.is_set_num_tasks();
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_window_to_stats = true && this.is_set_window_to_stats();
+    boolean that_present_window_to_stats = true && that.is_set_window_to_stats();
+    if (this_present_window_to_stats || that_present_window_to_stats) {
+      if (!(this_present_window_to_stats && that_present_window_to_stats))
+        return false;
+      if (!this.window_to_stats.equals(that.window_to_stats))
+        return false;
+    }
+
+    boolean this_present_gsid_to_input_stats = true && this.is_set_gsid_to_input_stats();
+    boolean that_present_gsid_to_input_stats = true && that.is_set_gsid_to_input_stats();
+    if (this_present_gsid_to_input_stats || that_present_gsid_to_input_stats) {
+      if (!(this_present_gsid_to_input_stats && that_present_gsid_to_input_stats))
+        return false;
+      if (!this.gsid_to_input_stats.equals(that.gsid_to_input_stats))
+        return false;
+    }
+
+    boolean this_present_sid_to_output_stats = true && this.is_set_sid_to_output_stats();
+    boolean that_present_sid_to_output_stats = true && that.is_set_sid_to_output_stats();
+    if (this_present_sid_to_output_stats || that_present_sid_to_output_stats) {
+      if (!(this_present_sid_to_output_stats && that_present_sid_to_output_stats))
+        return false;
+      if (!this.sid_to_output_stats.equals(that.sid_to_output_stats))
+        return false;
+    }
+
+    boolean this_present_exec_stats = true && this.is_set_exec_stats();
+    boolean that_present_exec_stats = true && that.is_set_exec_stats();
+    if (this_present_exec_stats || that_present_exec_stats) {
+      if (!(this_present_exec_stats && that_present_exec_stats))
+        return false;
+      if (!this.exec_stats.equals(that.exec_stats))
+        return false;
+    }
+
+    boolean this_present_errors = true && this.is_set_errors();
+    boolean that_present_errors = true && that.is_set_errors();
+    if (this_present_errors || that_present_errors) {
+      if (!(this_present_errors && that_present_errors))
+        return false;
+      if (!this.errors.equals(that.errors))
+        return false;
+    }
+
+    boolean this_present_eventlog_host = true && this.is_set_eventlog_host();
+    boolean that_present_eventlog_host = true && that.is_set_eventlog_host();
+    if (this_present_eventlog_host || that_present_eventlog_host) {
+      if (!(this_present_eventlog_host && that_present_eventlog_host))
+        return false;
+      if (!this.eventlog_host.equals(that.eventlog_host))
+        return false;
+    }
+
+    boolean this_present_eventlog_port = true && this.is_set_eventlog_port();
+    boolean that_present_eventlog_port = true && that.is_set_eventlog_port();
+    if (this_present_eventlog_port || that_present_eventlog_port) {
+      if (!(this_present_eventlog_port && that_present_eventlog_port))
+        return false;
+      if (this.eventlog_port != that.eventlog_port)
+        return false;
+    }
+
+    boolean this_present_debug_options = true && this.is_set_debug_options();
+    boolean that_present_debug_options = true && that.is_set_debug_options();
+    if (this_present_debug_options || that_present_debug_options) {
+      if (!(this_present_debug_options && that_present_debug_options))
+        return false;
+      if (!this.debug_options.equals(that.debug_options))
+        return false;
+    }
+
+    boolean this_present_topology_status = true && this.is_set_topology_status();
+    boolean that_present_topology_status = true && that.is_set_topology_status();
+    if (this_present_topology_status || that_present_topology_status) {
+      if (!(this_present_topology_status && that_present_topology_status))
+        return false;
+      if (!this.topology_status.equals(that.topology_status))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_component_id = true && (is_set_component_id());
+    list.add(present_component_id);
+    if (present_component_id)
+      list.add(component_id);
+
+    boolean present_component_type = true && (is_set_component_type());
+    list.add(present_component_type);
+    if (present_component_type)
+      list.add(component_type.getValue());
+
+    boolean present_topology_id = true && (is_set_topology_id());
+    list.add(present_topology_id);
+    if (present_topology_id)
+      list.add(topology_id);
+
+    boolean present_topology_name = true && (is_set_topology_name());
+    list.add(present_topology_name);
+    if (present_topology_name)
+      list.add(topology_name);
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_num_tasks = true && (is_set_num_tasks());
+    list.add(present_num_tasks);
+    if (present_num_tasks)
+      list.add(num_tasks);
+
+    boolean present_window_to_stats = true && (is_set_window_to_stats());
+    list.add(present_window_to_stats);
+    if (present_window_to_stats)
+      list.add(window_to_stats);
+
+    boolean present_gsid_to_input_stats = true && (is_set_gsid_to_input_stats());
+    list.add(present_gsid_to_input_stats);
+    if (present_gsid_to_input_stats)
+      list.add(gsid_to_input_stats);
+
+    boolean present_sid_to_output_stats = true && (is_set_sid_to_output_stats());
+    list.add(present_sid_to_output_stats);
+    if (present_sid_to_output_stats)
+      list.add(sid_to_output_stats);
+
+    boolean present_exec_stats = true && (is_set_exec_stats());
+    list.add(present_exec_stats);
+    if (present_exec_stats)
+      list.add(exec_stats);
+
+    boolean present_errors = true && (is_set_errors());
+    list.add(present_errors);
+    if (present_errors)
+      list.add(errors);
+
+    boolean present_eventlog_host = true && (is_set_eventlog_host());
+    list.add(present_eventlog_host);
+    if (present_eventlog_host)
+      list.add(eventlog_host);
+
+    boolean present_eventlog_port = true && (is_set_eventlog_port());
+    list.add(present_eventlog_port);
+    if (present_eventlog_port)
+      list.add(eventlog_port);
+
+    boolean present_debug_options = true && (is_set_debug_options());
+    list.add(present_debug_options);
+    if (present_debug_options)
+      list.add(debug_options);
+
+    boolean present_topology_status = true && (is_set_topology_status());
+    list.add(present_topology_status);
+    if (present_topology_status)
+      list.add(topology_status);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ComponentPageInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(other.is_set_component_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, other.component_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_type()).compareTo(other.is_set_component_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_type, other.component_type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_name()).compareTo(other.is_set_topology_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_name, other.topology_name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_stats()).compareTo(other.is_set_window_to_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_stats, other.window_to_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_gsid_to_input_stats()).compareTo(other.is_set_gsid_to_input_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_gsid_to_input_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.gsid_to_input_stats, other.gsid_to_input_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_sid_to_output_stats()).compareTo(other.is_set_sid_to_output_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sid_to_output_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sid_to_output_stats, other.sid_to_output_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_exec_stats()).compareTo(other.is_set_exec_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_exec_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exec_stats, other.exec_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_errors()).compareTo(other.is_set_errors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_errors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errors, other.errors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_eventlog_host()).compareTo(other.is_set_eventlog_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_eventlog_host()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eventlog_host, other.eventlog_host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_eventlog_port()).compareTo(other.is_set_eventlog_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_eventlog_port()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.eventlog_port, other.eventlog_port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_debug_options()).compareTo(other.is_set_debug_options());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_debug_options()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug_options, other.debug_options);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_status()).compareTo(other.is_set_topology_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_status, other.topology_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ComponentPageInfo(");
+    boolean first = true;
+
+    sb.append("component_id:");
+    if (this.component_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.component_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("component_type:");
+    if (this.component_type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.component_type);
+    }
+    first = false;
+    if (is_set_topology_id()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_id:");
+      if (this.topology_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_id);
+      }
+      first = false;
+    }
+    if (is_set_topology_name()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_name:");
+      if (this.topology_name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_name);
+      }
+      first = false;
+    }
+    if (is_set_num_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("num_executors:");
+      sb.append(this.num_executors);
+      first = false;
+    }
+    if (is_set_num_tasks()) {
+      if (!first) sb.append(", ");
+      sb.append("num_tasks:");
+      sb.append(this.num_tasks);
+      first = false;
+    }
+    if (is_set_window_to_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_stats:");
+      if (this.window_to_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_stats);
+      }
+      first = false;
+    }
+    if (is_set_gsid_to_input_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("gsid_to_input_stats:");
+      if (this.gsid_to_input_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.gsid_to_input_stats);
+      }
+      first = false;
+    }
+    if (is_set_sid_to_output_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("sid_to_output_stats:");
+      if (this.sid_to_output_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sid_to_output_stats);
+      }
+      first = false;
+    }
+    if (is_set_exec_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("exec_stats:");
+      if (this.exec_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.exec_stats);
+      }
+      first = false;
+    }
+    if (is_set_errors()) {
+      if (!first) sb.append(", ");
+      sb.append("errors:");
+      if (this.errors == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.errors);
+      }
+      first = false;
+    }
+    if (is_set_eventlog_host()) {
+      if (!first) sb.append(", ");
+      sb.append("eventlog_host:");
+      if (this.eventlog_host == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.eventlog_host);
+      }
+      first = false;
+    }
+    if (is_set_eventlog_port()) {
+      if (!first) sb.append(", ");
+      sb.append("eventlog_port:");
+      sb.append(this.eventlog_port);
+      first = false;
+    }
+    if (is_set_debug_options()) {
+      if (!first) sb.append(", ");
+      sb.append("debug_options:");
+      if (this.debug_options == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.debug_options);
+      }
+      first = false;
+    }
+    if (is_set_topology_status()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_status:");
+      if (this.topology_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_status);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_component_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_component_type()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'component_type' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (debug_options != null) {
+      debug_options.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ComponentPageInfoStandardSchemeFactory implements SchemeFactory {
+    public ComponentPageInfoStandardScheme getScheme() {
+      return new ComponentPageInfoStandardScheme();
+    }
+  }
+
+  private static class ComponentPageInfoStandardScheme extends StandardScheme<ComponentPageInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // COMPONENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.component_id = iprot.readString();
+              struct.set_component_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COMPONENT_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.component_type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32());
+              struct.set_component_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_id = iprot.readString();
+              struct.set_topology_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TOPOLOGY_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_name = iprot.readString();
+              struct.set_topology_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_tasks = iprot.readI32();
+              struct.set_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // WINDOW_TO_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map422 = iprot.readMapBegin();
+                struct.window_to_stats = new HashMap<String,ComponentAggregateStats>(2*_map422.size);
+                String _key423;
+                ComponentAggregateStats _val424;
+                for (int _i425 = 0; _i425 < _map422.size; ++_i425)
+                {
+                  _key423 = iprot.readString();
+                  _val424 = new ComponentAggregateStats();
+                  _val424.read(iprot);
+                  struct.window_to_stats.put(_key423, _val424);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // GSID_TO_INPUT_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map426 = iprot.readMapBegin();
+                struct.gsid_to_input_stats = new HashMap<GlobalStreamId,ComponentAggregateStats>(2*_map426.size);
+                GlobalStreamId _key427;
+                ComponentAggregateStats _val428;
+                for (int _i429 = 0; _i429 < _map426.size; ++_i429)
+                {
+                  _key427 = new GlobalStreamId();
+                  _key427.read(iprot);
+                  _val428 = new ComponentAggregateStats();
+                  _val428.read(iprot);
+                  struct.gsid_to_input_stats.put(_key427, _val428);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_gsid_to_input_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // SID_TO_OUTPUT_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map430 = iprot.readMapBegin();
+                struct.sid_to_output_stats = new HashMap<String,ComponentAggregateStats>(2*_map430.size);
+                String _key431;
+                ComponentAggregateStats _val432;
+                for (int _i433 = 0; _i433 < _map430.size; ++_i433)
+                {
+                  _key431 = iprot.readString();
+                  _val432 = new ComponentAggregateStats();
+                  _val432.read(iprot);
+                  struct.sid_to_output_stats.put(_key431, _val432);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_sid_to_output_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // EXEC_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list434 = iprot.readListBegin();
+                struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list434.size);
+                ExecutorAggregateStats _elem435;
+                for (int _i436 = 0; _i436 < _list434.size; ++_i436)
+                {
+                  _elem435 = new ExecutorAggregateStats();
+                  _elem435.read(iprot);
+                  struct.exec_stats.add(_elem435);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_exec_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 11: // ERRORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list437 = iprot.readListBegin();
+                struct.errors = new ArrayList<ErrorInfo>(_list437.size);
+                ErrorInfo _elem438;
+                for (int _i439 = 0; _i439 < _list437.size; ++_i439)
+                {
+                  _elem438 = new ErrorInfo();
+                  _elem438.read(iprot);
+                  struct.errors.add(_elem438);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_errors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // EVENTLOG_HOST
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.eventlog_host = iprot.readString();
+              struct.set_eventlog_host_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // EVENTLOG_PORT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.eventlog_port = iprot.readI32();
+              struct.set_eventlog_port_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 14: // DEBUG_OPTIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.debug_options = new DebugOptions();
+              struct.debug_options.read(iprot);
+              struct.set_debug_options_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 15: // TOPOLOGY_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_status = iprot.readString();
+              struct.set_topology_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.component_id != null) {
+        oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+        oprot.writeString(struct.component_id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.component_type != null) {
+        oprot.writeFieldBegin(COMPONENT_TYPE_FIELD_DESC);
+        oprot.writeI32(struct.component_type.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.topology_id != null) {
+        if (struct.is_set_topology_id()) {
+          oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+          oprot.writeString(struct.topology_id);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_name != null) {
+        if (struct.is_set_topology_name()) {
+          oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC);
+          oprot.writeString(struct.topology_name);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+        oprot.writeI32(struct.num_executors);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+        oprot.writeI32(struct.num_tasks);
+        oprot.writeFieldEnd();
+      }
+      if (struct.window_to_stats != null) {
+        if (struct.is_set_window_to_stats()) {
+          oprot.writeFieldBegin(WINDOW_TO_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.window_to_stats.size()));
+            for (Map.Entry<String, ComponentAggregateStats> _iter440 : struct.window_to_stats.entrySet())
+            {
+              oprot.writeString(_iter440.getKey());
+              _iter440.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.gsid_to_input_stats != null) {
+        if (struct.is_set_gsid_to_input_stats()) {
+          oprot.writeFieldBegin(GSID_TO_INPUT_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.gsid_to_input_stats.size()));
+            for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter441 : struct.gsid_to_input_stats.entrySet())
+            {
+              _iter441.getKey().write(oprot);
+              _iter441.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.sid_to_output_stats != null) {
+        if (struct.is_set_sid_to_output_stats()) {
+          oprot.writeFieldBegin(SID_TO_OUTPUT_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.sid_to_output_stats.size()));
+            for (Map.Entry<String, ComponentAggregateStats> _iter442 : struct.sid_to_output_stats.entrySet())
+            {
+              oprot.writeString(_iter442.getKey());
+              _iter442.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.exec_stats != null) {
+        if (struct.is_set_exec_stats()) {
+          oprot.writeFieldBegin(EXEC_STATS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.exec_stats.size()));
+            for (ExecutorAggregateStats _iter443 : struct.exec_stats)
+            {
+              _iter443.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.errors != null) {
+        if (struct.is_set_errors()) {
+          oprot.writeFieldBegin(ERRORS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.errors.size()));
+            for (ErrorInfo _iter444 : struct.errors)
+            {
+              _iter444.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.eventlog_host != null) {
+        if (struct.is_set_eventlog_host()) {
+          oprot.writeFieldBegin(EVENTLOG_HOST_FIELD_DESC);
+          oprot.writeString(struct.eventlog_host);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_eventlog_port()) {
+        oprot.writeFieldBegin(EVENTLOG_PORT_FIELD_DESC);
+        oprot.writeI32(struct.eventlog_port);
+        oprot.writeFieldEnd();
+      }
+      if (struct.debug_options != null) {
+        if (struct.is_set_debug_options()) {
+          oprot.writeFieldBegin(DEBUG_OPTIONS_FIELD_DESC);
+          struct.debug_options.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_status != null) {
+        if (struct.is_set_topology_status()) {
+          oprot.writeFieldBegin(TOPOLOGY_STATUS_FIELD_DESC);
+          oprot.writeString(struct.topology_status);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ComponentPageInfoTupleSchemeFactory implements SchemeFactory {
+    public ComponentPageInfoTupleScheme getScheme() {
+      return new ComponentPageInfoTupleScheme();
+    }
+  }
+
+  private static class ComponentPageInfoTupleScheme extends TupleScheme<ComponentPageInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.component_id);
+      oprot.writeI32(struct.component_type.getValue());
+      BitSet optionals = new BitSet();
+      if (struct.is_set_topology_id()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_topology_name()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_num_executors()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_num_tasks()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_window_to_stats()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_gsid_to_input_stats()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_sid_to_output_stats()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_exec_stats()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_errors()) {
+        optionals.set(8);
+      }
+      if (struct.is_set_eventlog_host()) {
+        optionals.set(9);
+      }
+      if (struct.is_set_eventlog_port()) {
+        optionals.set(10);
+      }
+      if (struct.is_set_debug_options()) {
+        optionals.set(11);
+      }
+      if (struct.is_set_topology_status()) {
+        optionals.set(12);
+      }
+      oprot.writeBitSet(optionals, 13);
+      if (struct.is_set_topology_id()) {
+        oprot.writeString(struct.topology_id);
+      }
+      if (struct.is_set_topology_name()) {
+        oprot.writeString(struct.topology_name);
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeI32(struct.num_executors);
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeI32(struct.num_tasks);
+      }
+      if (struct.is_set_window_to_stats()) {
+        {
+          oprot.writeI32(struct.window_to_stats.size());
+          for (Map.Entry<String, ComponentAggregateStats> _iter445 : struct.window_to_stats.entrySet())
+          {
+            oprot.writeString(_iter445.getKey());
+            _iter445.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_gsid_to_input_stats()) {
+        {
+          oprot.writeI32(struct.gsid_to_input_stats.size());
+          for (Map.Entry<GlobalStreamId, ComponentAggregateStats> _iter446 : struct.gsid_to_input_stats.entrySet())
+          {
+            _iter446.getKey().write(oprot);
+            _iter446.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_sid_to_output_stats()) {
+        {
+          oprot.writeI32(struct.sid_to_output_stats.size());
+          for (Map.Entry<String, ComponentAggregateStats> _iter447 : struct.sid_to_output_stats.entrySet())
+          {
+            oprot.writeString(_iter447.getKey());
+            _iter447.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_exec_stats()) {
+        {
+          oprot.writeI32(struct.exec_stats.size());
+          for (ExecutorAggregateStats _iter448 : struct.exec_stats)
+          {
+            _iter448.write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_errors()) {
+        {
+          oprot.writeI32(struct.errors.size());
+          for (ErrorInfo _iter449 : struct.errors)
+          {
+            _iter449.write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_eventlog_host()) {
+        oprot.writeString(struct.eventlog_host);
+      }
+      if (struct.is_set_eventlog_port()) {
+        oprot.writeI32(struct.eventlog_port);
+      }
+      if (struct.is_set_debug_options()) {
+        struct.debug_options.write(oprot);
+      }
+      if (struct.is_set_topology_status()) {
+        oprot.writeString(struct.topology_status);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ComponentPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.component_id = iprot.readString();
+      struct.set_component_id_isSet(true);
+      struct.component_type = backtype.storm.generated.ComponentType.findByValue(iprot.readI32());
+      struct.set_component_type_isSet(true);
+      BitSet incoming = iprot.readBitSet(13);
+      if (incoming.get(0)) {
+        struct.topology_id = iprot.readString();
+        struct.set_topology_id_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.topology_name = iprot.readString();
+        struct.set_topology_name_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.num_executors = iprot.readI32();
+        struct.set_num_executors_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.num_tasks = iprot.readI32();
+        struct.set_num_tasks_isSet(true);
+      }
+      if (incoming.get(4)) {
+        {
+          org.apache.thrift.protocol.TMap _map450 = 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*_map450.size);
+          String _key451;
+          ComponentAggregateStats _val452;
+          for (int _i453 = 0; _i453 < _map450.size; ++_i453)
+          {
+            _key451 = iprot.readString();
+            _val452 = new ComponentAggregateStats();
+            _val452.read(iprot);
+            struct.window_to_stats.put(_key451, _val452);
+          }
+        }
+        struct.set_window_to_stats_isSet(true);
+      }
+      if (incoming.get(5)) {
+        {
+          org.apache.thrift.protocol.TMap _map454 = 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*_map454.size);
+          GlobalStreamId _key455;
+          ComponentAggregateStats _val456;
+          for (int _i457 = 0; _i457 < _map454.size; ++_i457)
+          {
+            _key455 = new GlobalStreamId();
+            _key455.read(iprot);
+            _val456 = new ComponentAggregateStats();
+            _val456.read(iprot);
+            struct.gsid_to_input_stats.put(_key455, _val456);
+          }
+        }
+        struct.set_gsid_to_input_stats_isSet(true);
+      }
+      if (incoming.get(6)) {
+        {
+          org.apache.thrift.protocol.TMap _map458 = 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*_map458.size);
+          String _key459;
+          ComponentAggregateStats _val460;
+          for (int _i461 = 0; _i461 < _map458.size; ++_i461)
+          {
+            _key459 = iprot.readString();
+            _val460 = new ComponentAggregateStats();
+            _val460.read(iprot);
+            struct.sid_to_output_stats.put(_key459, _val460);
+          }
+        }
+        struct.set_sid_to_output_stats_isSet(true);
+      }
+      if (incoming.get(7)) {
+        {
+          org.apache.thrift.protocol.TList _list462 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.exec_stats = new ArrayList<ExecutorAggregateStats>(_list462.size);
+          ExecutorAggregateStats _elem463;
+          for (int _i464 = 0; _i464 < _list462.size; ++_i464)
+          {
+            _elem463 = new ExecutorAggregateStats();
+            _elem463.read(iprot);
+            struct.exec_stats.add(_elem463);
+          }
+        }
+        struct.set_exec_stats_isSet(true);
+      }
+      if (incoming.get(8)) {
+        {
+          org.apache.thrift.protocol.TList _list465 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.errors = new ArrayList<ErrorInfo>(_list465.size);
+          ErrorInfo _elem466;
+          for (int _i467 = 0; _i467 < _list465.size; ++_i467)
+          {
+            _elem466 = new ErrorInfo();
+            _elem466.read(iprot);
+            struct.errors.add(_elem466);
+          }
+        }
+        struct.set_errors_isSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.eventlog_host = iprot.readString();
+        struct.set_eventlog_host_isSet(true);
+      }
+      if (incoming.get(10)) {
+        struct.eventlog_port = iprot.readI32();
+        struct.set_eventlog_port_isSet(true);
+      }
+      if (incoming.get(11)) {
+        struct.debug_options = new DebugOptions();
+        struct.debug_options.read(iprot);
+        struct.set_debug_options_isSet(true);
+      }
+      if (incoming.get(12)) {
+        struct.topology_status = iprot.readString();
+        struct.set_topology_status_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/ComponentType.java b/storm-core/src/jvm/backtype/storm/generated/ComponentType.java
new file mode 100644
index 0000000..086feb7
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ComponentType.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum ComponentType implements org.apache.thrift.TEnum {
+  BOLT(1),
+  SPOUT(2);
+
+  private final int value;
+
+  private ComponentType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static ComponentType findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return BOLT;
+      case 2:
+        return SPOUT;
+      default:
+        return null;
+    }
+  }
+}
diff --git a/storm-core/src/jvm/backtype/storm/generated/Credentials.java b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
index 8484b18..75cc5b6 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Credentials.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Credentials.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class Credentials implements org.apache.thrift.TBase<Credentials, Credentials._Fields>, java.io.Serializable, Cloneable, Comparable<Credentials> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials");
 
@@ -365,15 +365,15 @@
           case 1: // CREDS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map344 = iprot.readMapBegin();
-                struct.creds = new HashMap<String,String>(2*_map344.size);
-                String _key345;
-                String _val346;
-                for (int _i347 = 0; _i347 < _map344.size; ++_i347)
+                org.apache.thrift.protocol.TMap _map478 = iprot.readMapBegin();
+                struct.creds = new HashMap<String,String>(2*_map478.size);
+                String _key479;
+                String _val480;
+                for (int _i481 = 0; _i481 < _map478.size; ++_i481)
                 {
-                  _key345 = iprot.readString();
-                  _val346 = iprot.readString();
-                  struct.creds.put(_key345, _val346);
+                  _key479 = iprot.readString();
+                  _val480 = iprot.readString();
+                  struct.creds.put(_key479, _val480);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@
         oprot.writeFieldBegin(CREDS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.creds.size()));
-          for (Map.Entry<String, String> _iter348 : struct.creds.entrySet())
+          for (Map.Entry<String, String> _iter482 : struct.creds.entrySet())
           {
-            oprot.writeString(_iter348.getKey());
-            oprot.writeString(_iter348.getValue());
+            oprot.writeString(_iter482.getKey());
+            oprot.writeString(_iter482.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.creds.size());
-        for (Map.Entry<String, String> _iter349 : struct.creds.entrySet())
+        for (Map.Entry<String, String> _iter483 : struct.creds.entrySet())
         {
-          oprot.writeString(_iter349.getKey());
-          oprot.writeString(_iter349.getValue());
+          oprot.writeString(_iter483.getKey());
+          oprot.writeString(_iter483.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@
     public void read(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map350 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.creds = new HashMap<String,String>(2*_map350.size);
-        String _key351;
-        String _val352;
-        for (int _i353 = 0; _i353 < _map350.size; ++_i353)
+        org.apache.thrift.protocol.TMap _map484 = 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*_map484.size);
+        String _key485;
+        String _val486;
+        for (int _i487 = 0; _i487 < _map484.size; ++_i487)
         {
-          _key351 = iprot.readString();
-          _val352 = iprot.readString();
-          struct.creds.put(_key351, _val352);
+          _key485 = iprot.readString();
+          _val486 = iprot.readString();
+          struct.creds.put(_key485, _val486);
         }
       }
       struct.set_creds_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java b/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
index f61ffa1..7f10776 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DRPCExecutionException.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class DRPCExecutionException extends TException implements org.apache.thrift.TBase<DRPCExecutionException, DRPCExecutionException._Fields>, java.io.Serializable, Cloneable, Comparable<DRPCExecutionException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DRPCExecutionException");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java b/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
index b97fa25..569df6e 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DRPCRequest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class DRPCRequest implements org.apache.thrift.TBase<DRPCRequest, DRPCRequest._Fields>, java.io.Serializable, Cloneable, Comparable<DRPCRequest> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DRPCRequest");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
index 60c0e24..f53cf98 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DebugOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class DebugOptions implements org.apache.thrift.TBase<DebugOptions, DebugOptions._Fields>, java.io.Serializable, Cloneable, Comparable<DebugOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DebugOptions");
 
@@ -236,10 +236,10 @@
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case ENABLE:
-      return Boolean.valueOf(is_enable());
+      return is_enable();
 
     case SAMPLINGPCT:
-      return Double.valueOf(get_samplingpct());
+      return get_samplingpct();
 
     }
     throw new IllegalStateException();
diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
index c4f25ec..1a42417 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPC.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class DistributedRPC {
 
   public interface Iface {
diff --git a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
index 94b799b..48271ad 100644
--- a/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
+++ b/storm-core/src/jvm/backtype/storm/generated/DistributedRPCInvocations.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class DistributedRPCInvocations {
 
   public interface Iface {
diff --git a/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java b/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
index 32495c7..83f10e6 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ErrorInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ErrorInfo implements org.apache.thrift.TBase<ErrorInfo, ErrorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<ErrorInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ErrorInfo");
 
@@ -333,13 +333,13 @@
       return get_error();
 
     case ERROR_TIME_SECS:
-      return Integer.valueOf(get_error_time_secs());
+      return get_error_time_secs();
 
     case HOST:
       return get_host();
 
     case PORT:
-      return Integer.valueOf(get_port());
+      return get_port();
 
     }
     throw new IllegalStateException();
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
new file mode 100644
index 0000000..e9d82b4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorAggregateStats.java
@@ -0,0 +1,526 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField EXEC_SUMMARY_FIELD_DESC = new org.apache.thrift.protocol.TField("exec_summary", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("stats", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ExecutorAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ExecutorAggregateStatsTupleSchemeFactory());
+  }
+
+  private ExecutorSummary exec_summary; // optional
+  private ComponentAggregateStats stats; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXEC_SUMMARY((short)1, "exec_summary"),
+    STATS((short)2, "stats");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXEC_SUMMARY
+          return EXEC_SUMMARY;
+        case 2: // STATS
+          return STATS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.EXEC_SUMMARY,_Fields.STATS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXEC_SUMMARY, new org.apache.thrift.meta_data.FieldMetaData("exec_summary", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorSummary.class)));
+    tmpMap.put(_Fields.STATS, new org.apache.thrift.meta_data.FieldMetaData("stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorAggregateStats.class, metaDataMap);
+  }
+
+  public ExecutorAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ExecutorAggregateStats(ExecutorAggregateStats other) {
+    if (other.is_set_exec_summary()) {
+      this.exec_summary = new ExecutorSummary(other.exec_summary);
+    }
+    if (other.is_set_stats()) {
+      this.stats = new ComponentAggregateStats(other.stats);
+    }
+  }
+
+  public ExecutorAggregateStats deepCopy() {
+    return new ExecutorAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.exec_summary = null;
+    this.stats = null;
+  }
+
+  public ExecutorSummary get_exec_summary() {
+    return this.exec_summary;
+  }
+
+  public void set_exec_summary(ExecutorSummary exec_summary) {
+    this.exec_summary = exec_summary;
+  }
+
+  public void unset_exec_summary() {
+    this.exec_summary = null;
+  }
+
+  /** Returns true if field exec_summary is set (has been assigned a value) and false otherwise */
+  public boolean is_set_exec_summary() {
+    return this.exec_summary != null;
+  }
+
+  public void set_exec_summary_isSet(boolean value) {
+    if (!value) {
+      this.exec_summary = null;
+    }
+  }
+
+  public ComponentAggregateStats get_stats() {
+    return this.stats;
+  }
+
+  public void set_stats(ComponentAggregateStats stats) {
+    this.stats = stats;
+  }
+
+  public void unset_stats() {
+    this.stats = null;
+  }
+
+  /** Returns true if field stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_stats() {
+    return this.stats != null;
+  }
+
+  public void set_stats_isSet(boolean value) {
+    if (!value) {
+      this.stats = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXEC_SUMMARY:
+      if (value == null) {
+        unset_exec_summary();
+      } else {
+        set_exec_summary((ExecutorSummary)value);
+      }
+      break;
+
+    case STATS:
+      if (value == null) {
+        unset_stats();
+      } else {
+        set_stats((ComponentAggregateStats)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXEC_SUMMARY:
+      return get_exec_summary();
+
+    case STATS:
+      return get_stats();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXEC_SUMMARY:
+      return is_set_exec_summary();
+    case STATS:
+      return is_set_stats();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ExecutorAggregateStats)
+      return this.equals((ExecutorAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(ExecutorAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_exec_summary = true && this.is_set_exec_summary();
+    boolean that_present_exec_summary = true && that.is_set_exec_summary();
+    if (this_present_exec_summary || that_present_exec_summary) {
+      if (!(this_present_exec_summary && that_present_exec_summary))
+        return false;
+      if (!this.exec_summary.equals(that.exec_summary))
+        return false;
+    }
+
+    boolean this_present_stats = true && this.is_set_stats();
+    boolean that_present_stats = true && that.is_set_stats();
+    if (this_present_stats || that_present_stats) {
+      if (!(this_present_stats && that_present_stats))
+        return false;
+      if (!this.stats.equals(that.stats))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_exec_summary = true && (is_set_exec_summary());
+    list.add(present_exec_summary);
+    if (present_exec_summary)
+      list.add(exec_summary);
+
+    boolean present_stats = true && (is_set_stats());
+    list.add(present_stats);
+    if (present_stats)
+      list.add(stats);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ExecutorAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_exec_summary()).compareTo(other.is_set_exec_summary());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_exec_summary()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.exec_summary, other.exec_summary);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_stats()).compareTo(other.is_set_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stats, other.stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ExecutorAggregateStats(");
+    boolean first = true;
+
+    if (is_set_exec_summary()) {
+      sb.append("exec_summary:");
+      if (this.exec_summary == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.exec_summary);
+      }
+      first = false;
+    }
+    if (is_set_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("stats:");
+      if (this.stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.stats);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (exec_summary != null) {
+      exec_summary.validate();
+    }
+    if (stats != null) {
+      stats.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ExecutorAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public ExecutorAggregateStatsStandardScheme getScheme() {
+      return new ExecutorAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class ExecutorAggregateStatsStandardScheme extends StandardScheme<ExecutorAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXEC_SUMMARY
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.exec_summary = new ExecutorSummary();
+              struct.exec_summary.read(iprot);
+              struct.set_exec_summary_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.stats = new ComponentAggregateStats();
+              struct.stats.read(iprot);
+              struct.set_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.exec_summary != null) {
+        if (struct.is_set_exec_summary()) {
+          oprot.writeFieldBegin(EXEC_SUMMARY_FIELD_DESC);
+          struct.exec_summary.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.stats != null) {
+        if (struct.is_set_stats()) {
+          oprot.writeFieldBegin(STATS_FIELD_DESC);
+          struct.stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ExecutorAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public ExecutorAggregateStatsTupleScheme getScheme() {
+      return new ExecutorAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class ExecutorAggregateStatsTupleScheme extends TupleScheme<ExecutorAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_exec_summary()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_stats()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.is_set_exec_summary()) {
+        struct.exec_summary.write(oprot);
+      }
+      if (struct.is_set_stats()) {
+        struct.stats.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ExecutorAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.exec_summary = new ExecutorSummary();
+        struct.exec_summary.read(iprot);
+        struct.set_exec_summary_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.stats = new ComponentAggregateStats();
+        struct.stats.read(iprot);
+        struct.set_stats_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
index 44415ce..3613de7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ExecutorInfo implements org.apache.thrift.TBase<ExecutorInfo, ExecutorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorInfo");
 
@@ -246,10 +246,10 @@
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case TASK_START:
-      return Integer.valueOf(get_task_start());
+      return get_task_start();
 
     case TASK_END:
-      return Integer.valueOf(get_task_end());
+      return get_task_end();
 
     }
     throw new IllegalStateException();
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorSpecificStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorSpecificStats.java
index 06b741c..09fa80a 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorSpecificStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorSpecificStats.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
index 58a7936..8a2a796 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorStats.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ExecutorStats implements org.apache.thrift.TBase<ExecutorStats, ExecutorStats._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorStats");
 
@@ -397,7 +397,7 @@
       return get_specific();
 
     case RATE:
-      return Double.valueOf(get_rate());
+      return get_rate();
 
     }
     throw new IllegalStateException();
@@ -660,27 +660,27 @@
           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 _map276 = iprot.readMapBegin();
+                struct.emitted = new HashMap<String,Map<String,Long>>(2*_map276.size);
+                String _key277;
+                Map<String,Long> _val278;
+                for (int _i279 = 0; _i279 < _map276.size; ++_i279)
                 {
-                  _key259 = iprot.readString();
+                  _key277 = 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 _map280 = iprot.readMapBegin();
+                    _val278 = new HashMap<String,Long>(2*_map280.size);
+                    String _key281;
+                    long _val282;
+                    for (int _i283 = 0; _i283 < _map280.size; ++_i283)
                     {
-                      _key263 = iprot.readString();
-                      _val264 = iprot.readI64();
-                      _val260.put(_key263, _val264);
+                      _key281 = iprot.readString();
+                      _val282 = iprot.readI64();
+                      _val278.put(_key281, _val282);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.emitted.put(_key259, _val260);
+                  struct.emitted.put(_key277, _val278);
                 }
                 iprot.readMapEnd();
               }
@@ -692,27 +692,27 @@
           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 _map284 = iprot.readMapBegin();
+                struct.transferred = new HashMap<String,Map<String,Long>>(2*_map284.size);
+                String _key285;
+                Map<String,Long> _val286;
+                for (int _i287 = 0; _i287 < _map284.size; ++_i287)
                 {
-                  _key267 = iprot.readString();
+                  _key285 = 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 _map288 = iprot.readMapBegin();
+                    _val286 = new HashMap<String,Long>(2*_map288.size);
+                    String _key289;
+                    long _val290;
+                    for (int _i291 = 0; _i291 < _map288.size; ++_i291)
                     {
-                      _key271 = iprot.readString();
-                      _val272 = iprot.readI64();
-                      _val268.put(_key271, _val272);
+                      _key289 = iprot.readString();
+                      _val290 = iprot.readI64();
+                      _val286.put(_key289, _val290);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.transferred.put(_key267, _val268);
+                  struct.transferred.put(_key285, _val286);
                 }
                 iprot.readMapEnd();
               }
@@ -755,15 +755,15 @@
         oprot.writeFieldBegin(EMITTED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.emitted.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter274 : struct.emitted.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter292 : struct.emitted.entrySet())
           {
-            oprot.writeString(_iter274.getKey());
+            oprot.writeString(_iter292.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, _iter292.getValue().size()));
+              for (Map.Entry<String, Long> _iter293 : _iter292.getValue().entrySet())
               {
-                oprot.writeString(_iter275.getKey());
-                oprot.writeI64(_iter275.getValue());
+                oprot.writeString(_iter293.getKey());
+                oprot.writeI64(_iter293.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -776,15 +776,15 @@
         oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.transferred.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter276 : struct.transferred.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter294 : struct.transferred.entrySet())
           {
-            oprot.writeString(_iter276.getKey());
+            oprot.writeString(_iter294.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, _iter294.getValue().size()));
+              for (Map.Entry<String, Long> _iter295 : _iter294.getValue().entrySet())
               {
-                oprot.writeString(_iter277.getKey());
-                oprot.writeI64(_iter277.getValue());
+                oprot.writeString(_iter295.getKey());
+                oprot.writeI64(_iter295.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -820,30 +820,30 @@
       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>> _iter296 : struct.emitted.entrySet())
         {
-          oprot.writeString(_iter278.getKey());
+          oprot.writeString(_iter296.getKey());
           {
-            oprot.writeI32(_iter278.getValue().size());
-            for (Map.Entry<String, Long> _iter279 : _iter278.getValue().entrySet())
+            oprot.writeI32(_iter296.getValue().size());
+            for (Map.Entry<String, Long> _iter297 : _iter296.getValue().entrySet())
             {
-              oprot.writeString(_iter279.getKey());
-              oprot.writeI64(_iter279.getValue());
+              oprot.writeString(_iter297.getKey());
+              oprot.writeI64(_iter297.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.transferred.size());
-        for (Map.Entry<String, Map<String,Long>> _iter280 : struct.transferred.entrySet())
+        for (Map.Entry<String, Map<String,Long>> _iter298 : struct.transferred.entrySet())
         {
-          oprot.writeString(_iter280.getKey());
+          oprot.writeString(_iter298.getKey());
           {
-            oprot.writeI32(_iter280.getValue().size());
-            for (Map.Entry<String, Long> _iter281 : _iter280.getValue().entrySet())
+            oprot.writeI32(_iter298.getValue().size());
+            for (Map.Entry<String, Long> _iter299 : _iter298.getValue().entrySet())
             {
-              oprot.writeString(_iter281.getKey());
-              oprot.writeI64(_iter281.getValue());
+              oprot.writeString(_iter299.getKey());
+              oprot.writeI64(_iter299.getValue());
             }
           }
         }
@@ -856,50 +856,50 @@
     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 _map300 = 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*_map300.size);
+        String _key301;
+        Map<String,Long> _val302;
+        for (int _i303 = 0; _i303 < _map300.size; ++_i303)
         {
-          _key283 = iprot.readString();
+          _key301 = 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 _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)
             {
-              _key287 = iprot.readString();
-              _val288 = iprot.readI64();
-              _val284.put(_key287, _val288);
+              _key305 = iprot.readString();
+              _val306 = iprot.readI64();
+              _val302.put(_key305, _val306);
             }
           }
-          struct.emitted.put(_key283, _val284);
+          struct.emitted.put(_key301, _val302);
         }
       }
       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 _map308 = 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*_map308.size);
+        String _key309;
+        Map<String,Long> _val310;
+        for (int _i311 = 0; _i311 < _map308.size; ++_i311)
         {
-          _key291 = iprot.readString();
+          _key309 = 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 _map312 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val310 = new HashMap<String,Long>(2*_map312.size);
+            String _key313;
+            long _val314;
+            for (int _i315 = 0; _i315 < _map312.size; ++_i315)
             {
-              _key295 = iprot.readString();
-              _val296 = iprot.readI64();
-              _val292.put(_key295, _val296);
+              _key313 = iprot.readString();
+              _val314 = iprot.readI64();
+              _val310.put(_key313, _val314);
             }
           }
-          struct.transferred.put(_key291, _val292);
+          struct.transferred.put(_key309, _val310);
         }
       }
       struct.set_transferred_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java b/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
index 649342d..84de5d7 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ExecutorSummary.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ExecutorSummary implements org.apache.thrift.TBase<ExecutorSummary, ExecutorSummary._Fields>, java.io.Serializable, Cloneable, Comparable<ExecutorSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecutorSummary");
 
@@ -430,10 +430,10 @@
       return get_host();
 
     case PORT:
-      return Integer.valueOf(get_port());
+      return get_port();
 
     case UPTIME_SECS:
-      return Integer.valueOf(get_uptime_secs());
+      return get_uptime_secs();
 
     case STATS:
       return get_stats();
diff --git a/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java b/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java
index 2d5defc..dac49f1 100644
--- a/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/GetInfoOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class GetInfoOptions implements org.apache.thrift.TBase<GetInfoOptions, GetInfoOptions._Fields>, java.io.Serializable, Cloneable, Comparable<GetInfoOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfoOptions");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java b/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
index 017ad4f..71a678f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
+++ b/storm-core/src/jvm/backtype/storm/generated/GlobalStreamId.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class GlobalStreamId implements org.apache.thrift.TBase<GlobalStreamId, GlobalStreamId._Fields>, java.io.Serializable, Cloneable, Comparable<GlobalStreamId> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GlobalStreamId");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/Grouping.java b/storm-core/src/jvm/backtype/storm/generated/Grouping.java
index 3994499..5b29bc6 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Grouping.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Grouping.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java b/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java
new file mode 100644
index 0000000..db21af4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBAuthorizationException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBAuthorizationException extends TException implements org.apache.thrift.TBase<HBAuthorizationException, HBAuthorizationException._Fields>, java.io.Serializable, Cloneable, Comparable<HBAuthorizationException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBAuthorizationException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBAuthorizationExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBAuthorizationExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBAuthorizationException.class, metaDataMap);
+  }
+
+  public HBAuthorizationException() {
+  }
+
+  public HBAuthorizationException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBAuthorizationException(HBAuthorizationException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public HBAuthorizationException deepCopy() {
+    return new HBAuthorizationException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBAuthorizationException)
+      return this.equals((HBAuthorizationException)that);
+    return false;
+  }
+
+  public boolean equals(HBAuthorizationException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBAuthorizationException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBAuthorizationException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBAuthorizationExceptionStandardSchemeFactory implements SchemeFactory {
+    public HBAuthorizationExceptionStandardScheme getScheme() {
+      return new HBAuthorizationExceptionStandardScheme();
+    }
+  }
+
+  private static class HBAuthorizationExceptionStandardScheme extends StandardScheme<HBAuthorizationException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBAuthorizationExceptionTupleSchemeFactory implements SchemeFactory {
+    public HBAuthorizationExceptionTupleScheme getScheme() {
+      return new HBAuthorizationExceptionTupleScheme();
+    }
+  }
+
+  private static class HBAuthorizationExceptionTupleScheme extends TupleScheme<HBAuthorizationException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBAuthorizationException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java b/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java
new file mode 100644
index 0000000..bf391fe
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBExecutionException.java
@@ -0,0 +1,406 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBExecutionException extends TException implements org.apache.thrift.TBase<HBExecutionException, HBExecutionException._Fields>, java.io.Serializable, Cloneable, Comparable<HBExecutionException> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBExecutionException");
+
+  private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBExecutionExceptionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBExecutionExceptionTupleSchemeFactory());
+  }
+
+  private String msg; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBExecutionException.class, metaDataMap);
+  }
+
+  public HBExecutionException() {
+  }
+
+  public HBExecutionException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBExecutionException(HBExecutionException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public HBExecutionException deepCopy() {
+    return new HBExecutionException(this);
+  }
+
+  @Override
+  public void clear() {
+    this.msg = null;
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been assigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBExecutionException)
+      return this.equals((HBExecutionException)that);
+    return false;
+  }
+
+  public boolean equals(HBExecutionException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_msg = true && (is_set_msg());
+    list.add(present_msg);
+    if (present_msg)
+      list.add(msg);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBExecutionException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBExecutionException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBExecutionExceptionStandardSchemeFactory implements SchemeFactory {
+    public HBExecutionExceptionStandardScheme getScheme() {
+      return new HBExecutionExceptionStandardScheme();
+    }
+  }
+
+  private static class HBExecutionExceptionStandardScheme extends StandardScheme<HBExecutionException> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBExecutionException struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MSG
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.msg = iprot.readString();
+              struct.set_msg_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBExecutionException struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.msg != null) {
+        oprot.writeFieldBegin(MSG_FIELD_DESC);
+        oprot.writeString(struct.msg);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBExecutionExceptionTupleSchemeFactory implements SchemeFactory {
+    public HBExecutionExceptionTupleScheme getScheme() {
+      return new HBExecutionExceptionTupleScheme();
+    }
+  }
+
+  private static class HBExecutionExceptionTupleScheme extends TupleScheme<HBExecutionException> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBExecutionException struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.msg);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBExecutionException struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.msg = iprot.readString();
+      struct.set_msg_isSet(true);
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBMessage.java b/storm-core/src/jvm/backtype/storm/generated/HBMessage.java
new file mode 100644
index 0000000..b3a2147
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBMessage.java
@@ -0,0 +1,636 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBMessage implements org.apache.thrift.TBase<HBMessage, HBMessage._Fields>, java.io.Serializable, Cloneable, Comparable<HBMessage> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBMessage");
+
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("data", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField MESSAGE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("message_id", org.apache.thrift.protocol.TType.I32, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBMessageStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBMessageTupleSchemeFactory());
+  }
+
+  private HBServerMessageType type; // required
+  private HBMessageData data; // required
+  private int message_id; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see HBServerMessageType
+     */
+    TYPE((short)1, "type"),
+    DATA((short)2, "data"),
+    MESSAGE_ID((short)3, "message_id");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TYPE
+          return TYPE;
+        case 2: // DATA
+          return DATA;
+        case 3: // MESSAGE_ID
+          return MESSAGE_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __MESSAGE_ID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.MESSAGE_ID};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, HBServerMessageType.class)));
+    tmpMap.put(_Fields.DATA, new org.apache.thrift.meta_data.FieldMetaData("data", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBMessageData.class)));
+    tmpMap.put(_Fields.MESSAGE_ID, new org.apache.thrift.meta_data.FieldMetaData("message_id", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBMessage.class, metaDataMap);
+  }
+
+  public HBMessage() {
+    this.message_id = -1;
+
+  }
+
+  public HBMessage(
+    HBServerMessageType type,
+    HBMessageData data)
+  {
+    this();
+    this.type = type;
+    this.data = data;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBMessage(HBMessage other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_type()) {
+      this.type = other.type;
+    }
+    if (other.is_set_data()) {
+      this.data = new HBMessageData(other.data);
+    }
+    this.message_id = other.message_id;
+  }
+
+  public HBMessage deepCopy() {
+    return new HBMessage(this);
+  }
+
+  @Override
+  public void clear() {
+    this.type = null;
+    this.data = null;
+    this.message_id = -1;
+
+  }
+
+  /**
+   * 
+   * @see HBServerMessageType
+   */
+  public HBServerMessageType get_type() {
+    return this.type;
+  }
+
+  /**
+   * 
+   * @see HBServerMessageType
+   */
+  public void set_type(HBServerMessageType type) {
+    this.type = type;
+  }
+
+  public void unset_type() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean is_set_type() {
+    return this.type != null;
+  }
+
+  public void set_type_isSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public HBMessageData get_data() {
+    return this.data;
+  }
+
+  public void set_data(HBMessageData data) {
+    this.data = data;
+  }
+
+  public void unset_data() {
+    this.data = null;
+  }
+
+  /** Returns true if field data is set (has been assigned a value) and false otherwise */
+  public boolean is_set_data() {
+    return this.data != null;
+  }
+
+  public void set_data_isSet(boolean value) {
+    if (!value) {
+      this.data = null;
+    }
+  }
+
+  public int get_message_id() {
+    return this.message_id;
+  }
+
+  public void set_message_id(int message_id) {
+    this.message_id = message_id;
+    set_message_id_isSet(true);
+  }
+
+  public void unset_message_id() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MESSAGE_ID_ISSET_ID);
+  }
+
+  /** Returns true if field message_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_message_id() {
+    return EncodingUtils.testBit(__isset_bitfield, __MESSAGE_ID_ISSET_ID);
+  }
+
+  public void set_message_id_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MESSAGE_ID_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TYPE:
+      if (value == null) {
+        unset_type();
+      } else {
+        set_type((HBServerMessageType)value);
+      }
+      break;
+
+    case DATA:
+      if (value == null) {
+        unset_data();
+      } else {
+        set_data((HBMessageData)value);
+      }
+      break;
+
+    case MESSAGE_ID:
+      if (value == null) {
+        unset_message_id();
+      } else {
+        set_message_id((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TYPE:
+      return get_type();
+
+    case DATA:
+      return get_data();
+
+    case MESSAGE_ID:
+      return get_message_id();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TYPE:
+      return is_set_type();
+    case DATA:
+      return is_set_data();
+    case MESSAGE_ID:
+      return is_set_message_id();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBMessage)
+      return this.equals((HBMessage)that);
+    return false;
+  }
+
+  public boolean equals(HBMessage that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_type = true && this.is_set_type();
+    boolean that_present_type = true && that.is_set_type();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    boolean this_present_data = true && this.is_set_data();
+    boolean that_present_data = true && that.is_set_data();
+    if (this_present_data || that_present_data) {
+      if (!(this_present_data && that_present_data))
+        return false;
+      if (!this.data.equals(that.data))
+        return false;
+    }
+
+    boolean this_present_message_id = true && this.is_set_message_id();
+    boolean that_present_message_id = true && that.is_set_message_id();
+    if (this_present_message_id || that_present_message_id) {
+      if (!(this_present_message_id && that_present_message_id))
+        return false;
+      if (this.message_id != that.message_id)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_type = true && (is_set_type());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
+    boolean present_data = true && (is_set_data());
+    list.add(present_data);
+    if (present_data)
+      list.add(data);
+
+    boolean present_message_id = true && (is_set_message_id());
+    list.add(present_message_id);
+    if (present_message_id)
+      list.add(message_id);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBMessage other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_type()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_data()).compareTo(other.is_set_data());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_data()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.data, other.data);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_message_id()).compareTo(other.is_set_message_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_message_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.message_id, other.message_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBMessage(");
+    boolean first = true;
+
+    sb.append("type:");
+    if (this.type == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.type);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("data:");
+    if (this.data == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.data);
+    }
+    first = false;
+    if (is_set_message_id()) {
+      if (!first) sb.append(", ");
+      sb.append("message_id:");
+      sb.append(this.message_id);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBMessageStandardSchemeFactory implements SchemeFactory {
+    public HBMessageStandardScheme getScheme() {
+      return new HBMessageStandardScheme();
+    }
+  }
+
+  private static class HBMessageStandardScheme extends StandardScheme<HBMessage> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBMessage struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = backtype.storm.generated.HBServerMessageType.findByValue(iprot.readI32());
+              struct.set_type_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DATA
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.data = new HBMessageData();
+              struct.data.read(iprot);
+              struct.set_data_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // MESSAGE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.message_id = iprot.readI32();
+              struct.set_message_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBMessage struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.type != null) {
+        oprot.writeFieldBegin(TYPE_FIELD_DESC);
+        oprot.writeI32(struct.type.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.data != null) {
+        oprot.writeFieldBegin(DATA_FIELD_DESC);
+        struct.data.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_message_id()) {
+        oprot.writeFieldBegin(MESSAGE_ID_FIELD_DESC);
+        oprot.writeI32(struct.message_id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBMessageTupleSchemeFactory implements SchemeFactory {
+    public HBMessageTupleScheme getScheme() {
+      return new HBMessageTupleScheme();
+    }
+  }
+
+  private static class HBMessageTupleScheme extends TupleScheme<HBMessage> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBMessage struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_type()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_data()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_message_id()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_type()) {
+        oprot.writeI32(struct.type.getValue());
+      }
+      if (struct.is_set_data()) {
+        struct.data.write(oprot);
+      }
+      if (struct.is_set_message_id()) {
+        oprot.writeI32(struct.message_id);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBMessage struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.type = backtype.storm.generated.HBServerMessageType.findByValue(iprot.readI32());
+        struct.set_type_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.data = new HBMessageData();
+        struct.data.read(iprot);
+        struct.set_data_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.message_id = iprot.readI32();
+        struct.set_message_id_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBMessageData.java b/storm-core/src/jvm/backtype/storm/generated/HBMessageData.java
new file mode 100644
index 0000000..dba2f8b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBMessageData.java
@@ -0,0 +1,640 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class HBMessageData extends org.apache.thrift.TUnion<HBMessageData, HBMessageData._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBMessageData");
+  private static final org.apache.thrift.protocol.TField PATH_FIELD_DESC = new org.apache.thrift.protocol.TField("path", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField PULSE_FIELD_DESC = new org.apache.thrift.protocol.TField("pulse", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+  private static final org.apache.thrift.protocol.TField BOOLVAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolval", org.apache.thrift.protocol.TType.BOOL, (short)3);
+  private static final org.apache.thrift.protocol.TField RECORDS_FIELD_DESC = new org.apache.thrift.protocol.TField("records", org.apache.thrift.protocol.TType.STRUCT, (short)4);
+  private static final org.apache.thrift.protocol.TField NODES_FIELD_DESC = new org.apache.thrift.protocol.TField("nodes", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField MESSAGE_BLOB_FIELD_DESC = new org.apache.thrift.protocol.TField("message_blob", org.apache.thrift.protocol.TType.STRING, (short)7);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PATH((short)1, "path"),
+    PULSE((short)2, "pulse"),
+    BOOLVAL((short)3, "boolval"),
+    RECORDS((short)4, "records"),
+    NODES((short)5, "nodes"),
+    MESSAGE_BLOB((short)7, "message_blob");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PATH
+          return PATH;
+        case 2: // PULSE
+          return PULSE;
+        case 3: // BOOLVAL
+          return BOOLVAL;
+        case 4: // RECORDS
+          return RECORDS;
+        case 5: // NODES
+          return NODES;
+        case 7: // MESSAGE_BLOB
+          return MESSAGE_BLOB;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PATH, new org.apache.thrift.meta_data.FieldMetaData("path", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PULSE, new org.apache.thrift.meta_data.FieldMetaData("pulse", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT        , "HBPulse")));
+    tmpMap.put(_Fields.BOOLVAL, new org.apache.thrift.meta_data.FieldMetaData("boolval", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.RECORDS, new org.apache.thrift.meta_data.FieldMetaData("records", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT        , "HBRecords")));
+    tmpMap.put(_Fields.NODES, new org.apache.thrift.meta_data.FieldMetaData("nodes", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT        , "HBNodes")));
+    tmpMap.put(_Fields.MESSAGE_BLOB, new org.apache.thrift.meta_data.FieldMetaData("message_blob", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBMessageData.class, metaDataMap);
+  }
+
+  public HBMessageData() {
+    super();
+  }
+
+  public HBMessageData(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public HBMessageData(HBMessageData other) {
+    super(other);
+  }
+  public HBMessageData deepCopy() {
+    return new HBMessageData(this);
+  }
+
+  public static HBMessageData path(String value) {
+    HBMessageData x = new HBMessageData();
+    x.set_path(value);
+    return x;
+  }
+
+  public static HBMessageData pulse(HBPulse value) {
+    HBMessageData x = new HBMessageData();
+    x.set_pulse(value);
+    return x;
+  }
+
+  public static HBMessageData boolval(boolean value) {
+    HBMessageData x = new HBMessageData();
+    x.set_boolval(value);
+    return x;
+  }
+
+  public static HBMessageData records(HBRecords value) {
+    HBMessageData x = new HBMessageData();
+    x.set_records(value);
+    return x;
+  }
+
+  public static HBMessageData nodes(HBNodes value) {
+    HBMessageData x = new HBMessageData();
+    x.set_nodes(value);
+    return x;
+  }
+
+  public static HBMessageData message_blob(ByteBuffer value) {
+    HBMessageData x = new HBMessageData();
+    x.set_message_blob(value);
+    return x;
+  }
+
+  public static HBMessageData message_blob(byte[] value) {
+    HBMessageData x = new HBMessageData();
+    x.set_message_blob(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case PATH:
+        if (value instanceof String) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type String for field 'path', but got " + value.getClass().getSimpleName());
+      case PULSE:
+        if (value instanceof HBPulse) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type HBPulse for field 'pulse', but got " + value.getClass().getSimpleName());
+      case BOOLVAL:
+        if (value instanceof Boolean) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type Boolean for field 'boolval', but got " + value.getClass().getSimpleName());
+      case RECORDS:
+        if (value instanceof HBRecords) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type HBRecords for field 'records', but got " + value.getClass().getSimpleName());
+      case NODES:
+        if (value instanceof HBNodes) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type HBNodes for field 'nodes', but got " + value.getClass().getSimpleName());
+      case MESSAGE_BLOB:
+        if (value instanceof ByteBuffer) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type ByteBuffer for field 'message_blob', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case PATH:
+          if (field.type == PATH_FIELD_DESC.type) {
+            String path;
+            path = iprot.readString();
+            return path;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case PULSE:
+          if (field.type == PULSE_FIELD_DESC.type) {
+            HBPulse pulse;
+            pulse = new HBPulse();
+            pulse.read(iprot);
+            return pulse;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case BOOLVAL:
+          if (field.type == BOOLVAL_FIELD_DESC.type) {
+            Boolean boolval;
+            boolval = iprot.readBool();
+            return boolval;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case RECORDS:
+          if (field.type == RECORDS_FIELD_DESC.type) {
+            HBRecords records;
+            records = new HBRecords();
+            records.read(iprot);
+            return records;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case NODES:
+          if (field.type == NODES_FIELD_DESC.type) {
+            HBNodes nodes;
+            nodes = new HBNodes();
+            nodes.read(iprot);
+            return nodes;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case MESSAGE_BLOB:
+          if (field.type == MESSAGE_BLOB_FIELD_DESC.type) {
+            ByteBuffer message_blob;
+            message_blob = iprot.readBinary();
+            return message_blob;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case PATH:
+        String path = (String)value_;
+        oprot.writeString(path);
+        return;
+      case PULSE:
+        HBPulse pulse = (HBPulse)value_;
+        pulse.write(oprot);
+        return;
+      case BOOLVAL:
+        Boolean boolval = (Boolean)value_;
+        oprot.writeBool(boolval);
+        return;
+      case RECORDS:
+        HBRecords records = (HBRecords)value_;
+        records.write(oprot);
+        return;
+      case NODES:
+        HBNodes nodes = (HBNodes)value_;
+        nodes.write(oprot);
+        return;
+      case MESSAGE_BLOB:
+        ByteBuffer message_blob = (ByteBuffer)value_;
+        oprot.writeBinary(message_blob);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case PATH:
+          String path;
+          path = iprot.readString();
+          return path;
+        case PULSE:
+          HBPulse pulse;
+          pulse = new HBPulse();
+          pulse.read(iprot);
+          return pulse;
+        case BOOLVAL:
+          Boolean boolval;
+          boolval = iprot.readBool();
+          return boolval;
+        case RECORDS:
+          HBRecords records;
+          records = new HBRecords();
+          records.read(iprot);
+          return records;
+        case NODES:
+          HBNodes nodes;
+          nodes = new HBNodes();
+          nodes.read(iprot);
+          return nodes;
+        case MESSAGE_BLOB:
+          ByteBuffer message_blob;
+          message_blob = iprot.readBinary();
+          return message_blob;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case PATH:
+        String path = (String)value_;
+        oprot.writeString(path);
+        return;
+      case PULSE:
+        HBPulse pulse = (HBPulse)value_;
+        pulse.write(oprot);
+        return;
+      case BOOLVAL:
+        Boolean boolval = (Boolean)value_;
+        oprot.writeBool(boolval);
+        return;
+      case RECORDS:
+        HBRecords records = (HBRecords)value_;
+        records.write(oprot);
+        return;
+      case NODES:
+        HBNodes nodes = (HBNodes)value_;
+        nodes.write(oprot);
+        return;
+      case MESSAGE_BLOB:
+        ByteBuffer message_blob = (ByteBuffer)value_;
+        oprot.writeBinary(message_blob);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case PATH:
+        return PATH_FIELD_DESC;
+      case PULSE:
+        return PULSE_FIELD_DESC;
+      case BOOLVAL:
+        return BOOLVAL_FIELD_DESC;
+      case RECORDS:
+        return RECORDS_FIELD_DESC;
+      case NODES:
+        return NODES_FIELD_DESC;
+      case MESSAGE_BLOB:
+        return MESSAGE_BLOB_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public String get_path() {
+    if (getSetField() == _Fields.PATH) {
+      return (String)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'path' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_path(String value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.PATH;
+    value_ = value;
+  }
+
+  public HBPulse get_pulse() {
+    if (getSetField() == _Fields.PULSE) {
+      return (HBPulse)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'pulse' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_pulse(HBPulse value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.PULSE;
+    value_ = value;
+  }
+
+  public boolean get_boolval() {
+    if (getSetField() == _Fields.BOOLVAL) {
+      return (Boolean)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'boolval' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_boolval(boolean value) {
+    setField_ = _Fields.BOOLVAL;
+    value_ = value;
+  }
+
+  public HBRecords get_records() {
+    if (getSetField() == _Fields.RECORDS) {
+      return (HBRecords)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'records' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_records(HBRecords value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.RECORDS;
+    value_ = value;
+  }
+
+  public HBNodes get_nodes() {
+    if (getSetField() == _Fields.NODES) {
+      return (HBNodes)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'nodes' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_nodes(HBNodes value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.NODES;
+    value_ = value;
+  }
+
+  public byte[] get_message_blob() {
+    set_message_blob(org.apache.thrift.TBaseHelper.rightSize(buffer_for_message_blob()));
+    ByteBuffer b = buffer_for_message_blob();
+    return b == null ? null : b.array();
+  }
+
+  public ByteBuffer buffer_for_message_blob() {
+    if (getSetField() == _Fields.MESSAGE_BLOB) {
+      return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue());
+    } else {
+      throw new RuntimeException("Cannot get field 'message_blob' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_message_blob(byte[] value) {
+    set_message_blob(ByteBuffer.wrap(Arrays.copyOf(value, value.length)));
+  }
+
+  public void set_message_blob(ByteBuffer value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.MESSAGE_BLOB;
+    value_ = value;
+  }
+
+  public boolean is_set_path() {
+    return setField_ == _Fields.PATH;
+  }
+
+
+  public boolean is_set_pulse() {
+    return setField_ == _Fields.PULSE;
+  }
+
+
+  public boolean is_set_boolval() {
+    return setField_ == _Fields.BOOLVAL;
+  }
+
+
+  public boolean is_set_records() {
+    return setField_ == _Fields.RECORDS;
+  }
+
+
+  public boolean is_set_nodes() {
+    return setField_ == _Fields.NODES;
+  }
+
+
+  public boolean is_set_message_blob() {
+    return setField_ == _Fields.MESSAGE_BLOB;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof HBMessageData) {
+      return equals((HBMessageData)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(HBMessageData other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(HBMessageData other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBNodes.java b/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
new file mode 100644
index 0000000..b6fc526
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBNodes.java
@@ -0,0 +1,461 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBNodes implements org.apache.thrift.TBase<HBNodes, HBNodes._Fields>, java.io.Serializable, Cloneable, Comparable<HBNodes> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBNodes");
+
+  private static final org.apache.thrift.protocol.TField PULSE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("pulseIds", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBNodesStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBNodesTupleSchemeFactory());
+  }
+
+  private List<String> pulseIds; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PULSE_IDS((short)1, "pulseIds");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PULSE_IDS
+          return PULSE_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PULSE_IDS, new org.apache.thrift.meta_data.FieldMetaData("pulseIds", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBNodes.class, metaDataMap);
+  }
+
+  public HBNodes() {
+  }
+
+  public HBNodes(
+    List<String> pulseIds)
+  {
+    this();
+    this.pulseIds = pulseIds;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBNodes(HBNodes other) {
+    if (other.is_set_pulseIds()) {
+      List<String> __this__pulseIds = new ArrayList<String>(other.pulseIds);
+      this.pulseIds = __this__pulseIds;
+    }
+  }
+
+  public HBNodes deepCopy() {
+    return new HBNodes(this);
+  }
+
+  @Override
+  public void clear() {
+    this.pulseIds = null;
+  }
+
+  public int get_pulseIds_size() {
+    return (this.pulseIds == null) ? 0 : this.pulseIds.size();
+  }
+
+  public java.util.Iterator<String> get_pulseIds_iterator() {
+    return (this.pulseIds == null) ? null : this.pulseIds.iterator();
+  }
+
+  public void add_to_pulseIds(String elem) {
+    if (this.pulseIds == null) {
+      this.pulseIds = new ArrayList<String>();
+    }
+    this.pulseIds.add(elem);
+  }
+
+  public List<String> get_pulseIds() {
+    return this.pulseIds;
+  }
+
+  public void set_pulseIds(List<String> pulseIds) {
+    this.pulseIds = pulseIds;
+  }
+
+  public void unset_pulseIds() {
+    this.pulseIds = null;
+  }
+
+  /** Returns true if field pulseIds is set (has been assigned a value) and false otherwise */
+  public boolean is_set_pulseIds() {
+    return this.pulseIds != null;
+  }
+
+  public void set_pulseIds_isSet(boolean value) {
+    if (!value) {
+      this.pulseIds = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PULSE_IDS:
+      if (value == null) {
+        unset_pulseIds();
+      } else {
+        set_pulseIds((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PULSE_IDS:
+      return get_pulseIds();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case PULSE_IDS:
+      return is_set_pulseIds();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBNodes)
+      return this.equals((HBNodes)that);
+    return false;
+  }
+
+  public boolean equals(HBNodes that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_pulseIds = true && this.is_set_pulseIds();
+    boolean that_present_pulseIds = true && that.is_set_pulseIds();
+    if (this_present_pulseIds || that_present_pulseIds) {
+      if (!(this_present_pulseIds && that_present_pulseIds))
+        return false;
+      if (!this.pulseIds.equals(that.pulseIds))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_pulseIds = true && (is_set_pulseIds());
+    list.add(present_pulseIds);
+    if (present_pulseIds)
+      list.add(pulseIds);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBNodes other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_pulseIds()).compareTo(other.is_set_pulseIds());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_pulseIds()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pulseIds, other.pulseIds);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBNodes(");
+    boolean first = true;
+
+    sb.append("pulseIds:");
+    if (this.pulseIds == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pulseIds);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBNodesStandardSchemeFactory implements SchemeFactory {
+    public HBNodesStandardScheme getScheme() {
+      return new HBNodesStandardScheme();
+    }
+  }
+
+  private static class HBNodesStandardScheme extends StandardScheme<HBNodes> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBNodes struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // PULSE_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list706 = iprot.readListBegin();
+                struct.pulseIds = new ArrayList<String>(_list706.size);
+                String _elem707;
+                for (int _i708 = 0; _i708 < _list706.size; ++_i708)
+                {
+                  _elem707 = iprot.readString();
+                  struct.pulseIds.add(_elem707);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_pulseIds_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBNodes struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.pulseIds != null) {
+        oprot.writeFieldBegin(PULSE_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.pulseIds.size()));
+          for (String _iter709 : struct.pulseIds)
+          {
+            oprot.writeString(_iter709);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBNodesTupleSchemeFactory implements SchemeFactory {
+    public HBNodesTupleScheme getScheme() {
+      return new HBNodesTupleScheme();
+    }
+  }
+
+  private static class HBNodesTupleScheme extends TupleScheme<HBNodes> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBNodes struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_pulseIds()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_pulseIds()) {
+        {
+          oprot.writeI32(struct.pulseIds.size());
+          for (String _iter710 : struct.pulseIds)
+          {
+            oprot.writeString(_iter710);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBNodes struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list711 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.pulseIds = new ArrayList<String>(_list711.size);
+          String _elem712;
+          for (int _i713 = 0; _i713 < _list711.size; ++_i713)
+          {
+            _elem712 = iprot.readString();
+            struct.pulseIds.add(_elem712);
+          }
+        }
+        struct.set_pulseIds_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBPulse.java b/storm-core/src/jvm/backtype/storm/generated/HBPulse.java
new file mode 100644
index 0000000..aa244f5
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBPulse.java
@@ -0,0 +1,522 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBPulse implements org.apache.thrift.TBase<HBPulse, HBPulse._Fields>, java.io.Serializable, Cloneable, Comparable<HBPulse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBPulse");
+
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField DETAILS_FIELD_DESC = new org.apache.thrift.protocol.TField("details", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBPulseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBPulseTupleSchemeFactory());
+  }
+
+  private String id; // required
+  private ByteBuffer details; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ID((short)1, "id"),
+    DETAILS((short)2, "details");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // DETAILS
+          return DETAILS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DETAILS, new org.apache.thrift.meta_data.FieldMetaData("details", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBPulse.class, metaDataMap);
+  }
+
+  public HBPulse() {
+  }
+
+  public HBPulse(
+    String id,
+    ByteBuffer details)
+  {
+    this();
+    this.id = id;
+    this.details = org.apache.thrift.TBaseHelper.copyBinary(details);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBPulse(HBPulse other) {
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_details()) {
+      this.details = org.apache.thrift.TBaseHelper.copyBinary(other.details);
+    }
+  }
+
+  public HBPulse deepCopy() {
+    return new HBPulse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.id = null;
+    this.details = null;
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public byte[] get_details() {
+    set_details(org.apache.thrift.TBaseHelper.rightSize(details));
+    return details == null ? null : details.array();
+  }
+
+  public ByteBuffer buffer_for_details() {
+    return org.apache.thrift.TBaseHelper.copyBinary(details);
+  }
+
+  public void set_details(byte[] details) {
+    this.details = details == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(details, details.length));
+  }
+
+  public void set_details(ByteBuffer details) {
+    this.details = org.apache.thrift.TBaseHelper.copyBinary(details);
+  }
+
+  public void unset_details() {
+    this.details = null;
+  }
+
+  /** Returns true if field details is set (has been assigned a value) and false otherwise */
+  public boolean is_set_details() {
+    return this.details != null;
+  }
+
+  public void set_details_isSet(boolean value) {
+    if (!value) {
+      this.details = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case DETAILS:
+      if (value == null) {
+        unset_details();
+      } else {
+        set_details((ByteBuffer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case DETAILS:
+      return get_details();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case DETAILS:
+      return is_set_details();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBPulse)
+      return this.equals((HBPulse)that);
+    return false;
+  }
+
+  public boolean equals(HBPulse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_details = true && this.is_set_details();
+    boolean that_present_details = true && that.is_set_details();
+    if (this_present_details || that_present_details) {
+      if (!(this_present_details && that_present_details))
+        return false;
+      if (!this.details.equals(that.details))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_id = true && (is_set_id());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
+    boolean present_details = true && (is_set_details());
+    list.add(present_details);
+    if (present_details)
+      list.add(details);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBPulse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_details()).compareTo(other.is_set_details());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_details()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.details, other.details);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBPulse(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("details:");
+    if (this.details == null) {
+      sb.append("null");
+    } else {
+      org.apache.thrift.TBaseHelper.toString(this.details, sb);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBPulseStandardSchemeFactory implements SchemeFactory {
+    public HBPulseStandardScheme getScheme() {
+      return new HBPulseStandardScheme();
+    }
+  }
+
+  private static class HBPulseStandardScheme extends StandardScheme<HBPulse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBPulse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.id = iprot.readString();
+              struct.set_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DETAILS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.details = iprot.readBinary();
+              struct.set_details_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBPulse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(struct.id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.details != null) {
+        oprot.writeFieldBegin(DETAILS_FIELD_DESC);
+        oprot.writeBinary(struct.details);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBPulseTupleSchemeFactory implements SchemeFactory {
+    public HBPulseTupleScheme getScheme() {
+      return new HBPulseTupleScheme();
+    }
+  }
+
+  private static class HBPulseTupleScheme extends TupleScheme<HBPulse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBPulse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.id);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_details()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_details()) {
+        oprot.writeBinary(struct.details);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBPulse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.id = iprot.readString();
+      struct.set_id_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.details = iprot.readBinary();
+        struct.set_details_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBRecords.java b/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
new file mode 100644
index 0000000..4767068
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBRecords.java
@@ -0,0 +1,466 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class HBRecords implements org.apache.thrift.TBase<HBRecords, HBRecords._Fields>, java.io.Serializable, Cloneable, Comparable<HBRecords> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("HBRecords");
+
+  private static final org.apache.thrift.protocol.TField PULSES_FIELD_DESC = new org.apache.thrift.protocol.TField("pulses", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new HBRecordsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new HBRecordsTupleSchemeFactory());
+  }
+
+  private List<HBPulse> pulses; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    PULSES((short)1, "pulses");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // PULSES
+          return PULSES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.PULSES, new org.apache.thrift.meta_data.FieldMetaData("pulses", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, HBPulse.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(HBRecords.class, metaDataMap);
+  }
+
+  public HBRecords() {
+  }
+
+  public HBRecords(
+    List<HBPulse> pulses)
+  {
+    this();
+    this.pulses = pulses;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public HBRecords(HBRecords other) {
+    if (other.is_set_pulses()) {
+      List<HBPulse> __this__pulses = new ArrayList<HBPulse>(other.pulses.size());
+      for (HBPulse other_element : other.pulses) {
+        __this__pulses.add(new HBPulse(other_element));
+      }
+      this.pulses = __this__pulses;
+    }
+  }
+
+  public HBRecords deepCopy() {
+    return new HBRecords(this);
+  }
+
+  @Override
+  public void clear() {
+    this.pulses = null;
+  }
+
+  public int get_pulses_size() {
+    return (this.pulses == null) ? 0 : this.pulses.size();
+  }
+
+  public java.util.Iterator<HBPulse> get_pulses_iterator() {
+    return (this.pulses == null) ? null : this.pulses.iterator();
+  }
+
+  public void add_to_pulses(HBPulse elem) {
+    if (this.pulses == null) {
+      this.pulses = new ArrayList<HBPulse>();
+    }
+    this.pulses.add(elem);
+  }
+
+  public List<HBPulse> get_pulses() {
+    return this.pulses;
+  }
+
+  public void set_pulses(List<HBPulse> pulses) {
+    this.pulses = pulses;
+  }
+
+  public void unset_pulses() {
+    this.pulses = null;
+  }
+
+  /** Returns true if field pulses is set (has been assigned a value) and false otherwise */
+  public boolean is_set_pulses() {
+    return this.pulses != null;
+  }
+
+  public void set_pulses_isSet(boolean value) {
+    if (!value) {
+      this.pulses = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PULSES:
+      if (value == null) {
+        unset_pulses();
+      } else {
+        set_pulses((List<HBPulse>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PULSES:
+      return get_pulses();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case PULSES:
+      return is_set_pulses();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof HBRecords)
+      return this.equals((HBRecords)that);
+    return false;
+  }
+
+  public boolean equals(HBRecords that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_pulses = true && this.is_set_pulses();
+    boolean that_present_pulses = true && that.is_set_pulses();
+    if (this_present_pulses || that_present_pulses) {
+      if (!(this_present_pulses && that_present_pulses))
+        return false;
+      if (!this.pulses.equals(that.pulses))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_pulses = true && (is_set_pulses());
+    list.add(present_pulses);
+    if (present_pulses)
+      list.add(pulses);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(HBRecords other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_pulses()).compareTo(other.is_set_pulses());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_pulses()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pulses, other.pulses);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("HBRecords(");
+    boolean first = true;
+
+    sb.append("pulses:");
+    if (this.pulses == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.pulses);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class HBRecordsStandardSchemeFactory implements SchemeFactory {
+    public HBRecordsStandardScheme getScheme() {
+      return new HBRecordsStandardScheme();
+    }
+  }
+
+  private static class HBRecordsStandardScheme extends StandardScheme<HBRecords> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, HBRecords struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // PULSES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list698 = iprot.readListBegin();
+                struct.pulses = new ArrayList<HBPulse>(_list698.size);
+                HBPulse _elem699;
+                for (int _i700 = 0; _i700 < _list698.size; ++_i700)
+                {
+                  _elem699 = new HBPulse();
+                  _elem699.read(iprot);
+                  struct.pulses.add(_elem699);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_pulses_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, HBRecords struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.pulses != null) {
+        oprot.writeFieldBegin(PULSES_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pulses.size()));
+          for (HBPulse _iter701 : struct.pulses)
+          {
+            _iter701.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class HBRecordsTupleSchemeFactory implements SchemeFactory {
+    public HBRecordsTupleScheme getScheme() {
+      return new HBRecordsTupleScheme();
+    }
+  }
+
+  private static class HBRecordsTupleScheme extends TupleScheme<HBRecords> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, HBRecords struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_pulses()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_pulses()) {
+        {
+          oprot.writeI32(struct.pulses.size());
+          for (HBPulse _iter702 : struct.pulses)
+          {
+            _iter702.write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, HBRecords struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.pulses = new ArrayList<HBPulse>(_list703.size);
+          HBPulse _elem704;
+          for (int _i705 = 0; _i705 < _list703.size; ++_i705)
+          {
+            _elem704 = new HBPulse();
+            _elem704.read(iprot);
+            struct.pulses.add(_elem704);
+          }
+        }
+        struct.set_pulses_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/HBServerMessageType.java b/storm-core/src/jvm/backtype/storm/generated/HBServerMessageType.java
new file mode 100644
index 0000000..3cafc14
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/HBServerMessageType.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum HBServerMessageType implements org.apache.thrift.TEnum {
+  CREATE_PATH(0),
+  CREATE_PATH_RESPONSE(1),
+  EXISTS(2),
+  EXISTS_RESPONSE(3),
+  SEND_PULSE(4),
+  SEND_PULSE_RESPONSE(5),
+  GET_ALL_PULSE_FOR_PATH(6),
+  GET_ALL_PULSE_FOR_PATH_RESPONSE(7),
+  GET_ALL_NODES_FOR_PATH(8),
+  GET_ALL_NODES_FOR_PATH_RESPONSE(9),
+  GET_PULSE(10),
+  GET_PULSE_RESPONSE(11),
+  DELETE_PATH(12),
+  DELETE_PATH_RESPONSE(13),
+  DELETE_PULSE_ID(14),
+  DELETE_PULSE_ID_RESPONSE(15),
+  CONTROL_MESSAGE(16),
+  SASL_MESSAGE_TOKEN(17),
+  NOT_AUTHORIZED(18);
+
+  private final int value;
+
+  private HBServerMessageType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static HBServerMessageType findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return CREATE_PATH;
+      case 1:
+        return CREATE_PATH_RESPONSE;
+      case 2:
+        return EXISTS;
+      case 3:
+        return EXISTS_RESPONSE;
+      case 4:
+        return SEND_PULSE;
+      case 5:
+        return SEND_PULSE_RESPONSE;
+      case 6:
+        return GET_ALL_PULSE_FOR_PATH;
+      case 7:
+        return GET_ALL_PULSE_FOR_PATH_RESPONSE;
+      case 8:
+        return GET_ALL_NODES_FOR_PATH;
+      case 9:
+        return GET_ALL_NODES_FOR_PATH_RESPONSE;
+      case 10:
+        return GET_PULSE;
+      case 11:
+        return GET_PULSE_RESPONSE;
+      case 12:
+        return DELETE_PATH;
+      case 13:
+        return DELETE_PATH_RESPONSE;
+      case 14:
+        return DELETE_PULSE_ID;
+      case 15:
+        return DELETE_PULSE_ID_RESPONSE;
+      case 16:
+        return CONTROL_MESSAGE;
+      case 17:
+        return SASL_MESSAGE_TOKEN;
+      case 18:
+        return NOT_AUTHORIZED;
+      default:
+        return null;
+    }
+  }
+}
diff --git a/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java b/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
index 5a98418..934a0bf 100644
--- a/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/InvalidTopologyException.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class InvalidTopologyException extends TException implements org.apache.thrift.TBase<InvalidTopologyException, InvalidTopologyException._Fields>, java.io.Serializable, Cloneable, Comparable<InvalidTopologyException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidTopologyException");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/JavaObject.java b/storm-core/src/jvm/backtype/storm/generated/JavaObject.java
index fde0082..ddde594 100644
--- a/storm-core/src/jvm/backtype/storm/generated/JavaObject.java
+++ b/storm-core/src/jvm/backtype/storm/generated/JavaObject.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class JavaObject implements org.apache.thrift.TBase<JavaObject, JavaObject._Fields>, java.io.Serializable, Cloneable, Comparable<JavaObject> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("JavaObject");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/JavaObjectArg.java b/storm-core/src/jvm/backtype/storm/generated/JavaObjectArg.java
index e69bd34..47d0eaf 100644
--- a/storm-core/src/jvm/backtype/storm/generated/JavaObjectArg.java
+++ b/storm-core/src/jvm/backtype/storm/generated/JavaObjectArg.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/KillOptions.java b/storm-core/src/jvm/backtype/storm/generated/KillOptions.java
index f173395..f386ec1 100644
--- a/storm-core/src/jvm/backtype/storm/generated/KillOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/KillOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class KillOptions implements org.apache.thrift.TBase<KillOptions, KillOptions._Fields>, java.io.Serializable, Cloneable, Comparable<KillOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KillOptions");
 
@@ -195,7 +195,7 @@
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case WAIT_SECS:
-      return Integer.valueOf(get_wait_secs());
+      return get_wait_secs();
 
     }
     throw new IllegalStateException();
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
index fd9740b..20f0d10 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSApprovedWorkers.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class LSApprovedWorkers implements org.apache.thrift.TBase<LSApprovedWorkers, LSApprovedWorkers._Fields>, java.io.Serializable, Cloneable, Comparable<LSApprovedWorkers> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSApprovedWorkers");
 
@@ -365,15 +365,15 @@
           case 1: // APPROVED_WORKERS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map482 = iprot.readMapBegin();
-                struct.approved_workers = new HashMap<String,Integer>(2*_map482.size);
-                String _key483;
-                int _val484;
-                for (int _i485 = 0; _i485 < _map482.size; ++_i485)
+                org.apache.thrift.protocol.TMap _map636 = iprot.readMapBegin();
+                struct.approved_workers = new HashMap<String,Integer>(2*_map636.size);
+                String _key637;
+                int _val638;
+                for (int _i639 = 0; _i639 < _map636.size; ++_i639)
                 {
-                  _key483 = iprot.readString();
-                  _val484 = iprot.readI32();
-                  struct.approved_workers.put(_key483, _val484);
+                  _key637 = iprot.readString();
+                  _val638 = iprot.readI32();
+                  struct.approved_workers.put(_key637, _val638);
                 }
                 iprot.readMapEnd();
               }
@@ -399,10 +399,10 @@
         oprot.writeFieldBegin(APPROVED_WORKERS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.approved_workers.size()));
-          for (Map.Entry<String, Integer> _iter486 : struct.approved_workers.entrySet())
+          for (Map.Entry<String, Integer> _iter640 : struct.approved_workers.entrySet())
           {
-            oprot.writeString(_iter486.getKey());
-            oprot.writeI32(_iter486.getValue());
+            oprot.writeString(_iter640.getKey());
+            oprot.writeI32(_iter640.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -427,10 +427,10 @@
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.approved_workers.size());
-        for (Map.Entry<String, Integer> _iter487 : struct.approved_workers.entrySet())
+        for (Map.Entry<String, Integer> _iter641 : struct.approved_workers.entrySet())
         {
-          oprot.writeString(_iter487.getKey());
-          oprot.writeI32(_iter487.getValue());
+          oprot.writeString(_iter641.getKey());
+          oprot.writeI32(_iter641.getValue());
         }
       }
     }
@@ -439,15 +439,15 @@
     public void read(org.apache.thrift.protocol.TProtocol prot, LSApprovedWorkers struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map488 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.approved_workers = new HashMap<String,Integer>(2*_map488.size);
-        String _key489;
-        int _val490;
-        for (int _i491 = 0; _i491 < _map488.size; ++_i491)
+        org.apache.thrift.protocol.TMap _map642 = 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*_map642.size);
+        String _key643;
+        int _val644;
+        for (int _i645 = 0; _i645 < _map642.size; ++_i645)
         {
-          _key489 = iprot.readString();
-          _val490 = iprot.readI32();
-          struct.approved_workers.put(_key489, _val490);
+          _key643 = iprot.readString();
+          _val644 = iprot.readI32();
+          struct.approved_workers.put(_key643, _val644);
         }
       }
       struct.set_approved_workers_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
index 1d85cec..de4c803 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorAssignments.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class LSSupervisorAssignments implements org.apache.thrift.TBase<LSSupervisorAssignments, LSSupervisorAssignments._Fields>, java.io.Serializable, Cloneable, Comparable<LSSupervisorAssignments> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorAssignments");
 
@@ -376,16 +376,16 @@
           case 1: // ASSIGNMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map492 = iprot.readMapBegin();
-                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map492.size);
-                int _key493;
-                LocalAssignment _val494;
-                for (int _i495 = 0; _i495 < _map492.size; ++_i495)
+                org.apache.thrift.protocol.TMap _map646 = iprot.readMapBegin();
+                struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map646.size);
+                int _key647;
+                LocalAssignment _val648;
+                for (int _i649 = 0; _i649 < _map646.size; ++_i649)
                 {
-                  _key493 = iprot.readI32();
-                  _val494 = new LocalAssignment();
-                  _val494.read(iprot);
-                  struct.assignments.put(_key493, _val494);
+                  _key647 = iprot.readI32();
+                  _val648 = new LocalAssignment();
+                  _val648.read(iprot);
+                  struct.assignments.put(_key647, _val648);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@
         oprot.writeFieldBegin(ASSIGNMENTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.assignments.size()));
-          for (Map.Entry<Integer, LocalAssignment> _iter496 : struct.assignments.entrySet())
+          for (Map.Entry<Integer, LocalAssignment> _iter650 : struct.assignments.entrySet())
           {
-            oprot.writeI32(_iter496.getKey());
-            _iter496.getValue().write(oprot);
+            oprot.writeI32(_iter650.getKey());
+            _iter650.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.assignments.size());
-        for (Map.Entry<Integer, LocalAssignment> _iter497 : struct.assignments.entrySet())
+        for (Map.Entry<Integer, LocalAssignment> _iter651 : struct.assignments.entrySet())
         {
-          oprot.writeI32(_iter497.getKey());
-          _iter497.getValue().write(oprot);
+          oprot.writeI32(_iter651.getKey());
+          _iter651.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@
     public void read(org.apache.thrift.protocol.TProtocol prot, LSSupervisorAssignments struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map498 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.assignments = new HashMap<Integer,LocalAssignment>(2*_map498.size);
-        int _key499;
-        LocalAssignment _val500;
-        for (int _i501 = 0; _i501 < _map498.size; ++_i501)
+        org.apache.thrift.protocol.TMap _map652 = 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*_map652.size);
+        int _key653;
+        LocalAssignment _val654;
+        for (int _i655 = 0; _i655 < _map652.size; ++_i655)
         {
-          _key499 = iprot.readI32();
-          _val500 = new LocalAssignment();
-          _val500.read(iprot);
-          struct.assignments.put(_key499, _val500);
+          _key653 = iprot.readI32();
+          _val654 = new LocalAssignment();
+          _val654.read(iprot);
+          struct.assignments.put(_key653, _val654);
         }
       }
       struct.set_assignments_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java
index 6ee4dad..8be48fa 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSSupervisorId.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class LSSupervisorId implements org.apache.thrift.TBase<LSSupervisorId, LSSupervisorId._Fields>, java.io.Serializable, Cloneable, Comparable<LSSupervisorId> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSSupervisorId");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
new file mode 100644
index 0000000..79fea1e
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistory.java
@@ -0,0 +1,805 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSTopoHistory implements org.apache.thrift.TBase<LSTopoHistory, LSTopoHistory._Fields>, java.io.Serializable, Cloneable, Comparable<LSTopoHistory> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSTopoHistory");
+
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TIME_STAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("time_stamp", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField USERS_FIELD_DESC = new org.apache.thrift.protocol.TField("users", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField GROUPS_FIELD_DESC = new org.apache.thrift.protocol.TField("groups", org.apache.thrift.protocol.TType.LIST, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSTopoHistoryStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSTopoHistoryTupleSchemeFactory());
+  }
+
+  private String topology_id; // required
+  private long time_stamp; // required
+  private List<String> users; // required
+  private List<String> groups; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPOLOGY_ID((short)1, "topology_id"),
+    TIME_STAMP((short)2, "time_stamp"),
+    USERS((short)3, "users"),
+    GROUPS((short)4, "groups");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPOLOGY_ID
+          return TOPOLOGY_ID;
+        case 2: // TIME_STAMP
+          return TIME_STAMP;
+        case 3: // USERS
+          return USERS;
+        case 4: // GROUPS
+          return GROUPS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_STAMP_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TIME_STAMP, new org.apache.thrift.meta_data.FieldMetaData("time_stamp", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.USERS, new org.apache.thrift.meta_data.FieldMetaData("users", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.GROUPS, new org.apache.thrift.meta_data.FieldMetaData("groups", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSTopoHistory.class, metaDataMap);
+  }
+
+  public LSTopoHistory() {
+  }
+
+  public LSTopoHistory(
+    String topology_id,
+    long time_stamp,
+    List<String> users,
+    List<String> groups)
+  {
+    this();
+    this.topology_id = topology_id;
+    this.time_stamp = time_stamp;
+    set_time_stamp_isSet(true);
+    this.users = users;
+    this.groups = groups;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSTopoHistory(LSTopoHistory other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_topology_id()) {
+      this.topology_id = other.topology_id;
+    }
+    this.time_stamp = other.time_stamp;
+    if (other.is_set_users()) {
+      List<String> __this__users = new ArrayList<String>(other.users);
+      this.users = __this__users;
+    }
+    if (other.is_set_groups()) {
+      List<String> __this__groups = new ArrayList<String>(other.groups);
+      this.groups = __this__groups;
+    }
+  }
+
+  public LSTopoHistory deepCopy() {
+    return new LSTopoHistory(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topology_id = null;
+    set_time_stamp_isSet(false);
+    this.time_stamp = 0;
+    this.users = null;
+    this.groups = null;
+  }
+
+  public String get_topology_id() {
+    return this.topology_id;
+  }
+
+  public void set_topology_id(String topology_id) {
+    this.topology_id = topology_id;
+  }
+
+  public void unset_topology_id() {
+    this.topology_id = null;
+  }
+
+  /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_id() {
+    return this.topology_id != null;
+  }
+
+  public void set_topology_id_isSet(boolean value) {
+    if (!value) {
+      this.topology_id = null;
+    }
+  }
+
+  public long get_time_stamp() {
+    return this.time_stamp;
+  }
+
+  public void set_time_stamp(long time_stamp) {
+    this.time_stamp = time_stamp;
+    set_time_stamp_isSet(true);
+  }
+
+  public void unset_time_stamp() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  /** Returns true if field time_stamp is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_stamp() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  public void set_time_stamp_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_STAMP_ISSET_ID, value);
+  }
+
+  public int get_users_size() {
+    return (this.users == null) ? 0 : this.users.size();
+  }
+
+  public java.util.Iterator<String> get_users_iterator() {
+    return (this.users == null) ? null : this.users.iterator();
+  }
+
+  public void add_to_users(String elem) {
+    if (this.users == null) {
+      this.users = new ArrayList<String>();
+    }
+    this.users.add(elem);
+  }
+
+  public List<String> get_users() {
+    return this.users;
+  }
+
+  public void set_users(List<String> users) {
+    this.users = users;
+  }
+
+  public void unset_users() {
+    this.users = null;
+  }
+
+  /** Returns true if field users is set (has been assigned a value) and false otherwise */
+  public boolean is_set_users() {
+    return this.users != null;
+  }
+
+  public void set_users_isSet(boolean value) {
+    if (!value) {
+      this.users = null;
+    }
+  }
+
+  public int get_groups_size() {
+    return (this.groups == null) ? 0 : this.groups.size();
+  }
+
+  public java.util.Iterator<String> get_groups_iterator() {
+    return (this.groups == null) ? null : this.groups.iterator();
+  }
+
+  public void add_to_groups(String elem) {
+    if (this.groups == null) {
+      this.groups = new ArrayList<String>();
+    }
+    this.groups.add(elem);
+  }
+
+  public List<String> get_groups() {
+    return this.groups;
+  }
+
+  public void set_groups(List<String> groups) {
+    this.groups = groups;
+  }
+
+  public void unset_groups() {
+    this.groups = null;
+  }
+
+  /** Returns true if field groups is set (has been assigned a value) and false otherwise */
+  public boolean is_set_groups() {
+    return this.groups != null;
+  }
+
+  public void set_groups_isSet(boolean value) {
+    if (!value) {
+      this.groups = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      if (value == null) {
+        unset_topology_id();
+      } else {
+        set_topology_id((String)value);
+      }
+      break;
+
+    case TIME_STAMP:
+      if (value == null) {
+        unset_time_stamp();
+      } else {
+        set_time_stamp((Long)value);
+      }
+      break;
+
+    case USERS:
+      if (value == null) {
+        unset_users();
+      } else {
+        set_users((List<String>)value);
+      }
+      break;
+
+    case GROUPS:
+      if (value == null) {
+        unset_groups();
+      } else {
+        set_groups((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPOLOGY_ID:
+      return get_topology_id();
+
+    case TIME_STAMP:
+      return get_time_stamp();
+
+    case USERS:
+      return get_users();
+
+    case GROUPS:
+      return get_groups();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPOLOGY_ID:
+      return is_set_topology_id();
+    case TIME_STAMP:
+      return is_set_time_stamp();
+    case USERS:
+      return is_set_users();
+    case GROUPS:
+      return is_set_groups();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSTopoHistory)
+      return this.equals((LSTopoHistory)that);
+    return false;
+  }
+
+  public boolean equals(LSTopoHistory that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topology_id = true && this.is_set_topology_id();
+    boolean that_present_topology_id = true && that.is_set_topology_id();
+    if (this_present_topology_id || that_present_topology_id) {
+      if (!(this_present_topology_id && that_present_topology_id))
+        return false;
+      if (!this.topology_id.equals(that.topology_id))
+        return false;
+    }
+
+    boolean this_present_time_stamp = true;
+    boolean that_present_time_stamp = true;
+    if (this_present_time_stamp || that_present_time_stamp) {
+      if (!(this_present_time_stamp && that_present_time_stamp))
+        return false;
+      if (this.time_stamp != that.time_stamp)
+        return false;
+    }
+
+    boolean this_present_users = true && this.is_set_users();
+    boolean that_present_users = true && that.is_set_users();
+    if (this_present_users || that_present_users) {
+      if (!(this_present_users && that_present_users))
+        return false;
+      if (!this.users.equals(that.users))
+        return false;
+    }
+
+    boolean this_present_groups = true && this.is_set_groups();
+    boolean that_present_groups = true && that.is_set_groups();
+    if (this_present_groups || that_present_groups) {
+      if (!(this_present_groups && that_present_groups))
+        return false;
+      if (!this.groups.equals(that.groups))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topology_id = true && (is_set_topology_id());
+    list.add(present_topology_id);
+    if (present_topology_id)
+      list.add(topology_id);
+
+    boolean present_time_stamp = true;
+    list.add(present_time_stamp);
+    if (present_time_stamp)
+      list.add(time_stamp);
+
+    boolean present_users = true && (is_set_users());
+    list.add(present_users);
+    if (present_users)
+      list.add(users);
+
+    boolean present_groups = true && (is_set_groups());
+    list.add(present_groups);
+    if (present_groups)
+      list.add(groups);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSTopoHistory other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_stamp()).compareTo(other.is_set_time_stamp());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_stamp()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_stamp, other.time_stamp);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_users()).compareTo(other.is_set_users());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_users()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.users, other.users);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_groups()).compareTo(other.is_set_groups());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_groups()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.groups, other.groups);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSTopoHistory(");
+    boolean first = true;
+
+    sb.append("topology_id:");
+    if (this.topology_id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topology_id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("time_stamp:");
+    sb.append(this.time_stamp);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("users:");
+    if (this.users == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.users);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("groups:");
+    if (this.groups == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.groups);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_topology_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topology_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_time_stamp()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'time_stamp' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_users()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'users' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_groups()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'groups' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSTopoHistoryStandardSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryStandardScheme getScheme() {
+      return new LSTopoHistoryStandardScheme();
+    }
+  }
+
+  private static class LSTopoHistoryStandardScheme extends StandardScheme<LSTopoHistory> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPOLOGY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_id = iprot.readString();
+              struct.set_topology_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TIME_STAMP
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.time_stamp = iprot.readI64();
+              struct.set_time_stamp_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // USERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list664 = iprot.readListBegin();
+                struct.users = new ArrayList<String>(_list664.size);
+                String _elem665;
+                for (int _i666 = 0; _i666 < _list664.size; ++_i666)
+                {
+                  _elem665 = iprot.readString();
+                  struct.users.add(_elem665);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_users_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // GROUPS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list667 = iprot.readListBegin();
+                struct.groups = new ArrayList<String>(_list667.size);
+                String _elem668;
+                for (int _i669 = 0; _i669 < _list667.size; ++_i669)
+                {
+                  _elem668 = iprot.readString();
+                  struct.groups.add(_elem668);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_groups_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topology_id != null) {
+        oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+        oprot.writeString(struct.topology_id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TIME_STAMP_FIELD_DESC);
+      oprot.writeI64(struct.time_stamp);
+      oprot.writeFieldEnd();
+      if (struct.users != null) {
+        oprot.writeFieldBegin(USERS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.users.size()));
+          for (String _iter670 : struct.users)
+          {
+            oprot.writeString(_iter670);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.groups != null) {
+        oprot.writeFieldBegin(GROUPS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groups.size()));
+          for (String _iter671 : struct.groups)
+          {
+            oprot.writeString(_iter671);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSTopoHistoryTupleSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryTupleScheme getScheme() {
+      return new LSTopoHistoryTupleScheme();
+    }
+  }
+
+  private static class LSTopoHistoryTupleScheme extends TupleScheme<LSTopoHistory> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.topology_id);
+      oprot.writeI64(struct.time_stamp);
+      {
+        oprot.writeI32(struct.users.size());
+        for (String _iter672 : struct.users)
+        {
+          oprot.writeString(_iter672);
+        }
+      }
+      {
+        oprot.writeI32(struct.groups.size());
+        for (String _iter673 : struct.groups)
+        {
+          oprot.writeString(_iter673);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSTopoHistory struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.topology_id = iprot.readString();
+      struct.set_topology_id_isSet(true);
+      struct.time_stamp = iprot.readI64();
+      struct.set_time_stamp_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list674 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.users = new ArrayList<String>(_list674.size);
+        String _elem675;
+        for (int _i676 = 0; _i676 < _list674.size; ++_i676)
+        {
+          _elem675 = iprot.readString();
+          struct.users.add(_elem675);
+        }
+      }
+      struct.set_users_isSet(true);
+      {
+        org.apache.thrift.protocol.TList _list677 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.groups = new ArrayList<String>(_list677.size);
+        String _elem678;
+        for (int _i679 = 0; _i679 < _list677.size; ++_i679)
+        {
+          _elem678 = iprot.readString();
+          struct.groups.add(_elem678);
+        }
+      }
+      struct.set_groups_isSet(true);
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
new file mode 100644
index 0000000..962ece6
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/LSTopoHistoryList.java
@@ -0,0 +1,460 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class LSTopoHistoryList implements org.apache.thrift.TBase<LSTopoHistoryList, LSTopoHistoryList._Fields>, java.io.Serializable, Cloneable, Comparable<LSTopoHistoryList> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSTopoHistoryList");
+
+  private static final org.apache.thrift.protocol.TField TOPO_HISTORY_FIELD_DESC = new org.apache.thrift.protocol.TField("topo_history", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LSTopoHistoryListStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LSTopoHistoryListTupleSchemeFactory());
+  }
+
+  private List<LSTopoHistory> topo_history; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPO_HISTORY((short)1, "topo_history");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPO_HISTORY
+          return TOPO_HISTORY;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPO_HISTORY, new org.apache.thrift.meta_data.FieldMetaData("topo_history", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LSTopoHistory.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LSTopoHistoryList.class, metaDataMap);
+  }
+
+  public LSTopoHistoryList() {
+  }
+
+  public LSTopoHistoryList(
+    List<LSTopoHistory> topo_history)
+  {
+    this();
+    this.topo_history = topo_history;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LSTopoHistoryList(LSTopoHistoryList other) {
+    if (other.is_set_topo_history()) {
+      List<LSTopoHistory> __this__topo_history = new ArrayList<LSTopoHistory>(other.topo_history.size());
+      for (LSTopoHistory other_element : other.topo_history) {
+        __this__topo_history.add(new LSTopoHistory(other_element));
+      }
+      this.topo_history = __this__topo_history;
+    }
+  }
+
+  public LSTopoHistoryList deepCopy() {
+    return new LSTopoHistoryList(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topo_history = null;
+  }
+
+  public int get_topo_history_size() {
+    return (this.topo_history == null) ? 0 : this.topo_history.size();
+  }
+
+  public java.util.Iterator<LSTopoHistory> get_topo_history_iterator() {
+    return (this.topo_history == null) ? null : this.topo_history.iterator();
+  }
+
+  public void add_to_topo_history(LSTopoHistory elem) {
+    if (this.topo_history == null) {
+      this.topo_history = new ArrayList<LSTopoHistory>();
+    }
+    this.topo_history.add(elem);
+  }
+
+  public List<LSTopoHistory> get_topo_history() {
+    return this.topo_history;
+  }
+
+  public void set_topo_history(List<LSTopoHistory> topo_history) {
+    this.topo_history = topo_history;
+  }
+
+  public void unset_topo_history() {
+    this.topo_history = null;
+  }
+
+  /** Returns true if field topo_history is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topo_history() {
+    return this.topo_history != null;
+  }
+
+  public void set_topo_history_isSet(boolean value) {
+    if (!value) {
+      this.topo_history = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPO_HISTORY:
+      if (value == null) {
+        unset_topo_history();
+      } else {
+        set_topo_history((List<LSTopoHistory>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPO_HISTORY:
+      return get_topo_history();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPO_HISTORY:
+      return is_set_topo_history();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LSTopoHistoryList)
+      return this.equals((LSTopoHistoryList)that);
+    return false;
+  }
+
+  public boolean equals(LSTopoHistoryList that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topo_history = true && this.is_set_topo_history();
+    boolean that_present_topo_history = true && that.is_set_topo_history();
+    if (this_present_topo_history || that_present_topo_history) {
+      if (!(this_present_topo_history && that_present_topo_history))
+        return false;
+      if (!this.topo_history.equals(that.topo_history))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topo_history = true && (is_set_topo_history());
+    list.add(present_topo_history);
+    if (present_topo_history)
+      list.add(topo_history);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LSTopoHistoryList other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topo_history()).compareTo(other.is_set_topo_history());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topo_history()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topo_history, other.topo_history);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LSTopoHistoryList(");
+    boolean first = true;
+
+    sb.append("topo_history:");
+    if (this.topo_history == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topo_history);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_topo_history()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'topo_history' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LSTopoHistoryListStandardSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryListStandardScheme getScheme() {
+      return new LSTopoHistoryListStandardScheme();
+    }
+  }
+
+  private static class LSTopoHistoryListStandardScheme extends StandardScheme<LSTopoHistoryList> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPO_HISTORY
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list680 = iprot.readListBegin();
+                struct.topo_history = new ArrayList<LSTopoHistory>(_list680.size);
+                LSTopoHistory _elem681;
+                for (int _i682 = 0; _i682 < _list680.size; ++_i682)
+                {
+                  _elem681 = new LSTopoHistory();
+                  _elem681.read(iprot);
+                  struct.topo_history.add(_elem681);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_topo_history_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topo_history != null) {
+        oprot.writeFieldBegin(TOPO_HISTORY_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topo_history.size()));
+          for (LSTopoHistory _iter683 : struct.topo_history)
+          {
+            _iter683.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LSTopoHistoryListTupleSchemeFactory implements SchemeFactory {
+    public LSTopoHistoryListTupleScheme getScheme() {
+      return new LSTopoHistoryListTupleScheme();
+    }
+  }
+
+  private static class LSTopoHistoryListTupleScheme extends TupleScheme<LSTopoHistoryList> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      {
+        oprot.writeI32(struct.topo_history.size());
+        for (LSTopoHistory _iter684 : struct.topo_history)
+        {
+          _iter684.write(oprot);
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LSTopoHistoryList struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      {
+        org.apache.thrift.protocol.TList _list685 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.topo_history = new ArrayList<LSTopoHistory>(_list685.size);
+        LSTopoHistory _elem686;
+        for (int _i687 = 0; _i687 < _list685.size; ++_i687)
+        {
+          _elem686 = new LSTopoHistory();
+          _elem686.read(iprot);
+          struct.topo_history.add(_elem686);
+        }
+      }
+      struct.set_topo_history_isSet(true);
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
index 98726cc..d6e7c36 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LSWorkerHeartbeat.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class LSWorkerHeartbeat implements org.apache.thrift.TBase<LSWorkerHeartbeat, LSWorkerHeartbeat._Fields>, java.io.Serializable, Cloneable, Comparable<LSWorkerHeartbeat> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LSWorkerHeartbeat");
 
@@ -354,7 +354,7 @@
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case TIME_SECS:
-      return Integer.valueOf(get_time_secs());
+      return get_time_secs();
 
     case TOPOLOGY_ID:
       return get_topology_id();
@@ -363,7 +363,7 @@
       return get_executors();
 
     case PORT:
-      return Integer.valueOf(get_port());
+      return get_port();
 
     }
     throw new IllegalStateException();
@@ -638,14 +638,14 @@
           case 3: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list502 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list502.size);
-                ExecutorInfo _elem503;
-                for (int _i504 = 0; _i504 < _list502.size; ++_i504)
+                org.apache.thrift.protocol.TList _list656 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list656.size);
+                ExecutorInfo _elem657;
+                for (int _i658 = 0; _i658 < _list656.size; ++_i658)
                 {
-                  _elem503 = new ExecutorInfo();
-                  _elem503.read(iprot);
-                  struct.executors.add(_elem503);
+                  _elem657 = new ExecutorInfo();
+                  _elem657.read(iprot);
+                  struct.executors.add(_elem657);
                 }
                 iprot.readListEnd();
               }
@@ -687,9 +687,9 @@
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter505 : struct.executors)
+          for (ExecutorInfo _iter659 : struct.executors)
           {
-            _iter505.write(oprot);
+            _iter659.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter506 : struct.executors)
+        for (ExecutorInfo _iter660 : struct.executors)
         {
-          _iter506.write(oprot);
+          _iter660.write(oprot);
         }
       }
       oprot.writeI32(struct.port);
@@ -735,14 +735,14 @@
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list507 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list507.size);
-        ExecutorInfo _elem508;
-        for (int _i509 = 0; _i509 < _list507.size; ++_i509)
+        org.apache.thrift.protocol.TList _list661 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list661.size);
+        ExecutorInfo _elem662;
+        for (int _i663 = 0; _i663 < _list661.size; ++_i663)
         {
-          _elem508 = new ExecutorInfo();
-          _elem508.read(iprot);
-          struct.executors.add(_elem508);
+          _elem662 = new ExecutorInfo();
+          _elem662.read(iprot);
+          struct.executors.add(_elem662);
         }
       }
       struct.set_executors_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
index d985cba..a36e654 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalAssignment.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,12 +51,13 @@
 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.3)")
 public class LocalAssignment implements org.apache.thrift.TBase<LocalAssignment, LocalAssignment._Fields>, java.io.Serializable, Cloneable, Comparable<LocalAssignment> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalAssignment");
 
   private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("executors", org.apache.thrift.protocol.TType.LIST, (short)2);
+  private static final org.apache.thrift.protocol.TField RESOURCES_FIELD_DESC = new org.apache.thrift.protocol.TField("resources", org.apache.thrift.protocol.TType.STRUCT, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -66,11 +67,13 @@
 
   private String topology_id; // required
   private List<ExecutorInfo> executors; // required
+  private WorkerResources 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 {
     TOPOLOGY_ID((short)1, "topology_id"),
-    EXECUTORS((short)2, "executors");
+    EXECUTORS((short)2, "executors"),
+    RESOURCES((short)3, "resources");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -89,6 +92,8 @@
           return TOPOLOGY_ID;
         case 2: // EXECUTORS
           return EXECUTORS;
+        case 3: // RESOURCES
+          return RESOURCES;
         default:
           return null;
       }
@@ -129,6 +134,7 @@
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.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);
@@ -137,6 +143,8 @@
     tmpMap.put(_Fields.EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("executors", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ExecutorInfo.class))));
+    tmpMap.put(_Fields.RESOURCES, new org.apache.thrift.meta_data.FieldMetaData("resources", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerResources.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LocalAssignment.class, metaDataMap);
   }
@@ -167,6 +175,9 @@
       }
       this.executors = __this__executors;
     }
+    if (other.is_set_resources()) {
+      this.resources = new WorkerResources(other.resources);
+    }
   }
 
   public LocalAssignment deepCopy() {
@@ -177,6 +188,7 @@
   public void clear() {
     this.topology_id = null;
     this.executors = null;
+    this.resources = null;
   }
 
   public String get_topology_id() {
@@ -240,6 +252,29 @@
     }
   }
 
+  public WorkerResources get_resources() {
+    return this.resources;
+  }
+
+  public void set_resources(WorkerResources resources) {
+    this.resources = resources;
+  }
+
+  public void unset_resources() {
+    this.resources = null;
+  }
+
+  /** Returns true if field resources is set (has been assigned a value) and false otherwise */
+  public boolean is_set_resources() {
+    return this.resources != null;
+  }
+
+  public void set_resources_isSet(boolean value) {
+    if (!value) {
+      this.resources = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TOPOLOGY_ID:
@@ -258,6 +293,14 @@
       }
       break;
 
+    case RESOURCES:
+      if (value == null) {
+        unset_resources();
+      } else {
+        set_resources((WorkerResources)value);
+      }
+      break;
+
     }
   }
 
@@ -269,6 +312,9 @@
     case EXECUTORS:
       return get_executors();
 
+    case RESOURCES:
+      return get_resources();
+
     }
     throw new IllegalStateException();
   }
@@ -284,6 +330,8 @@
       return is_set_topology_id();
     case EXECUTORS:
       return is_set_executors();
+    case RESOURCES:
+      return is_set_resources();
     }
     throw new IllegalStateException();
   }
@@ -319,6 +367,15 @@
         return false;
     }
 
+    boolean this_present_resources = true && this.is_set_resources();
+    boolean that_present_resources = true && that.is_set_resources();
+    if (this_present_resources || that_present_resources) {
+      if (!(this_present_resources && that_present_resources))
+        return false;
+      if (!this.resources.equals(that.resources))
+        return false;
+    }
+
     return true;
   }
 
@@ -336,6 +393,11 @@
     if (present_executors)
       list.add(executors);
 
+    boolean present_resources = true && (is_set_resources());
+    list.add(present_resources);
+    if (present_resources)
+      list.add(resources);
+
     return list.hashCode();
   }
 
@@ -367,6 +429,16 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_resources()).compareTo(other.is_set_resources());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_resources()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.resources, other.resources);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -402,6 +474,16 @@
       sb.append(this.executors);
     }
     first = false;
+    if (is_set_resources()) {
+      if (!first) sb.append(", ");
+      sb.append("resources:");
+      if (this.resources == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.resources);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -417,6 +499,9 @@
     }
 
     // check for sub-struct validity
+    if (resources != null) {
+      resources.validate();
+    }
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -464,14 +549,14 @@
           case 2: // EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list474 = iprot.readListBegin();
-                struct.executors = new ArrayList<ExecutorInfo>(_list474.size);
-                ExecutorInfo _elem475;
-                for (int _i476 = 0; _i476 < _list474.size; ++_i476)
+                org.apache.thrift.protocol.TList _list628 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorInfo>(_list628.size);
+                ExecutorInfo _elem629;
+                for (int _i630 = 0; _i630 < _list628.size; ++_i630)
                 {
-                  _elem475 = new ExecutorInfo();
-                  _elem475.read(iprot);
-                  struct.executors.add(_elem475);
+                  _elem629 = new ExecutorInfo();
+                  _elem629.read(iprot);
+                  struct.executors.add(_elem629);
                 }
                 iprot.readListEnd();
               }
@@ -480,6 +565,15 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // RESOURCES
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.resources = new WorkerResources();
+              struct.resources.read(iprot);
+              struct.set_resources_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -502,14 +596,21 @@
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorInfo _iter477 : struct.executors)
+          for (ExecutorInfo _iter631 : struct.executors)
           {
-            _iter477.write(oprot);
+            _iter631.write(oprot);
           }
           oprot.writeListEnd();
         }
         oprot.writeFieldEnd();
       }
+      if (struct.resources != null) {
+        if (struct.is_set_resources()) {
+          oprot.writeFieldBegin(RESOURCES_FIELD_DESC);
+          struct.resources.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -530,11 +631,19 @@
       oprot.writeString(struct.topology_id);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorInfo _iter478 : struct.executors)
+        for (ExecutorInfo _iter632 : struct.executors)
         {
-          _iter478.write(oprot);
+          _iter632.write(oprot);
         }
       }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_resources()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_resources()) {
+        struct.resources.write(oprot);
+      }
     }
 
     @Override
@@ -543,17 +652,23 @@
       struct.topology_id = iprot.readString();
       struct.set_topology_id_isSet(true);
       {
-        org.apache.thrift.protocol.TList _list479 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.executors = new ArrayList<ExecutorInfo>(_list479.size);
-        ExecutorInfo _elem480;
-        for (int _i481 = 0; _i481 < _list479.size; ++_i481)
+        org.apache.thrift.protocol.TList _list633 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorInfo>(_list633.size);
+        ExecutorInfo _elem634;
+        for (int _i635 = 0; _i635 < _list633.size; ++_i635)
         {
-          _elem480 = new ExecutorInfo();
-          _elem480.read(iprot);
-          struct.executors.add(_elem480);
+          _elem634 = new ExecutorInfo();
+          _elem634.read(iprot);
+          struct.executors.add(_elem634);
         }
       }
       struct.set_executors_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.resources = new WorkerResources();
+        struct.resources.read(iprot);
+        struct.set_resources_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
index 0ce2d7a..7340926 100644
--- a/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
+++ b/storm-core/src/jvm/backtype/storm/generated/LocalStateData.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class LocalStateData implements org.apache.thrift.TBase<LocalStateData, LocalStateData._Fields>, java.io.Serializable, Cloneable, Comparable<LocalStateData> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalStateData");
 
@@ -376,16 +376,16 @@
           case 1: // SERIALIZED_PARTS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map464 = iprot.readMapBegin();
-                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map464.size);
-                String _key465;
-                ThriftSerializedObject _val466;
-                for (int _i467 = 0; _i467 < _map464.size; ++_i467)
+                org.apache.thrift.protocol.TMap _map618 = iprot.readMapBegin();
+                struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map618.size);
+                String _key619;
+                ThriftSerializedObject _val620;
+                for (int _i621 = 0; _i621 < _map618.size; ++_i621)
                 {
-                  _key465 = iprot.readString();
-                  _val466 = new ThriftSerializedObject();
-                  _val466.read(iprot);
-                  struct.serialized_parts.put(_key465, _val466);
+                  _key619 = iprot.readString();
+                  _val620 = new ThriftSerializedObject();
+                  _val620.read(iprot);
+                  struct.serialized_parts.put(_key619, _val620);
                 }
                 iprot.readMapEnd();
               }
@@ -411,10 +411,10 @@
         oprot.writeFieldBegin(SERIALIZED_PARTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.serialized_parts.size()));
-          for (Map.Entry<String, ThriftSerializedObject> _iter468 : struct.serialized_parts.entrySet())
+          for (Map.Entry<String, ThriftSerializedObject> _iter622 : struct.serialized_parts.entrySet())
           {
-            oprot.writeString(_iter468.getKey());
-            _iter468.getValue().write(oprot);
+            oprot.writeString(_iter622.getKey());
+            _iter622.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -439,10 +439,10 @@
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.serialized_parts.size());
-        for (Map.Entry<String, ThriftSerializedObject> _iter469 : struct.serialized_parts.entrySet())
+        for (Map.Entry<String, ThriftSerializedObject> _iter623 : struct.serialized_parts.entrySet())
         {
-          oprot.writeString(_iter469.getKey());
-          _iter469.getValue().write(oprot);
+          oprot.writeString(_iter623.getKey());
+          _iter623.getValue().write(oprot);
         }
       }
     }
@@ -451,16 +451,16 @@
     public void read(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map470 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.serialized_parts = new HashMap<String,ThriftSerializedObject>(2*_map470.size);
-        String _key471;
-        ThriftSerializedObject _val472;
-        for (int _i473 = 0; _i473 < _map470.size; ++_i473)
+        org.apache.thrift.protocol.TMap _map624 = 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*_map624.size);
+        String _key625;
+        ThriftSerializedObject _val626;
+        for (int _i627 = 0; _i627 < _map624.size; ++_i627)
         {
-          _key471 = iprot.readString();
-          _val472 = new ThriftSerializedObject();
-          _val472.read(iprot);
-          struct.serialized_parts.put(_key471, _val472);
+          _key625 = iprot.readString();
+          _val626 = new ThriftSerializedObject();
+          _val626.read(iprot);
+          struct.serialized_parts.put(_key625, _val626);
         }
       }
       struct.set_serialized_parts_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogConfig.java b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
new file mode 100644
index 0000000..53bc326
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/LogConfig.java
@@ -0,0 +1,475 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField NAMED_LOGGER_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("named_logger_level", org.apache.thrift.protocol.TType.MAP, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LogConfigStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LogConfigTupleSchemeFactory());
+  }
+
+  private Map<String,LogLevel> named_logger_level; // 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 {
+    NAMED_LOGGER_LEVEL((short)2, "named_logger_level");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 2: // NAMED_LOGGER_LEVEL
+          return NAMED_LOGGER_LEVEL;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.NAMED_LOGGER_LEVEL};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAMED_LOGGER_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("named_logger_level", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LogLevel.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LogConfig.class, metaDataMap);
+  }
+
+  public LogConfig() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LogConfig(LogConfig other) {
+    if (other.is_set_named_logger_level()) {
+      Map<String,LogLevel> __this__named_logger_level = new HashMap<String,LogLevel>(other.named_logger_level.size());
+      for (Map.Entry<String, LogLevel> other_element : other.named_logger_level.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        LogLevel other_element_value = other_element.getValue();
+
+        String __this__named_logger_level_copy_key = other_element_key;
+
+        LogLevel __this__named_logger_level_copy_value = new LogLevel(other_element_value);
+
+        __this__named_logger_level.put(__this__named_logger_level_copy_key, __this__named_logger_level_copy_value);
+      }
+      this.named_logger_level = __this__named_logger_level;
+    }
+  }
+
+  public LogConfig deepCopy() {
+    return new LogConfig(this);
+  }
+
+  @Override
+  public void clear() {
+    this.named_logger_level = null;
+  }
+
+  public int get_named_logger_level_size() {
+    return (this.named_logger_level == null) ? 0 : this.named_logger_level.size();
+  }
+
+  public void put_to_named_logger_level(String key, LogLevel val) {
+    if (this.named_logger_level == null) {
+      this.named_logger_level = new HashMap<String,LogLevel>();
+    }
+    this.named_logger_level.put(key, val);
+  }
+
+  public Map<String,LogLevel> get_named_logger_level() {
+    return this.named_logger_level;
+  }
+
+  public void set_named_logger_level(Map<String,LogLevel> named_logger_level) {
+    this.named_logger_level = named_logger_level;
+  }
+
+  public void unset_named_logger_level() {
+    this.named_logger_level = null;
+  }
+
+  /** Returns true if field named_logger_level is set (has been assigned a value) and false otherwise */
+  public boolean is_set_named_logger_level() {
+    return this.named_logger_level != null;
+  }
+
+  public void set_named_logger_level_isSet(boolean value) {
+    if (!value) {
+      this.named_logger_level = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAMED_LOGGER_LEVEL:
+      if (value == null) {
+        unset_named_logger_level();
+      } else {
+        set_named_logger_level((Map<String,LogLevel>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAMED_LOGGER_LEVEL:
+      return get_named_logger_level();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAMED_LOGGER_LEVEL:
+      return is_set_named_logger_level();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LogConfig)
+      return this.equals((LogConfig)that);
+    return false;
+  }
+
+  public boolean equals(LogConfig that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_named_logger_level = true && this.is_set_named_logger_level();
+    boolean that_present_named_logger_level = true && that.is_set_named_logger_level();
+    if (this_present_named_logger_level || that_present_named_logger_level) {
+      if (!(this_present_named_logger_level && that_present_named_logger_level))
+        return false;
+      if (!this.named_logger_level.equals(that.named_logger_level))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_named_logger_level = true && (is_set_named_logger_level());
+    list.add(present_named_logger_level);
+    if (present_named_logger_level)
+      list.add(named_logger_level);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LogConfig other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_named_logger_level()).compareTo(other.is_set_named_logger_level());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_named_logger_level()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.named_logger_level, other.named_logger_level);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LogConfig(");
+    boolean first = true;
+
+    if (is_set_named_logger_level()) {
+      sb.append("named_logger_level:");
+      if (this.named_logger_level == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.named_logger_level);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LogConfigStandardSchemeFactory implements SchemeFactory {
+    public LogConfigStandardScheme getScheme() {
+      return new LogConfigStandardScheme();
+    }
+  }
+
+  private static class LogConfigStandardScheme extends StandardScheme<LogConfig> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LogConfig struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 2: // NAMED_LOGGER_LEVEL
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map688 = iprot.readMapBegin();
+                struct.named_logger_level = new HashMap<String,LogLevel>(2*_map688.size);
+                String _key689;
+                LogLevel _val690;
+                for (int _i691 = 0; _i691 < _map688.size; ++_i691)
+                {
+                  _key689 = iprot.readString();
+                  _val690 = new LogLevel();
+                  _val690.read(iprot);
+                  struct.named_logger_level.put(_key689, _val690);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_named_logger_level_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LogConfig struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.named_logger_level != null) {
+        if (struct.is_set_named_logger_level()) {
+          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> _iter692 : struct.named_logger_level.entrySet())
+            {
+              oprot.writeString(_iter692.getKey());
+              _iter692.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LogConfigTupleSchemeFactory implements SchemeFactory {
+    public LogConfigTupleScheme getScheme() {
+      return new LogConfigTupleScheme();
+    }
+  }
+
+  private static class LogConfigTupleScheme extends TupleScheme<LogConfig> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_named_logger_level()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_named_logger_level()) {
+        {
+          oprot.writeI32(struct.named_logger_level.size());
+          for (Map.Entry<String, LogLevel> _iter693 : struct.named_logger_level.entrySet())
+          {
+            oprot.writeString(_iter693.getKey());
+            _iter693.getValue().write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LogConfig struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map694 = 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*_map694.size);
+          String _key695;
+          LogLevel _val696;
+          for (int _i697 = 0; _i697 < _map694.size; ++_i697)
+          {
+            _key695 = iprot.readString();
+            _val696 = new LogLevel();
+            _val696.read(iprot);
+            struct.named_logger_level.put(_key695, _val696);
+          }
+        }
+        struct.set_named_logger_level_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogLevel.java b/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
new file mode 100644
index 0000000..adc8e10
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/LogLevel.java
@@ -0,0 +1,836 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField ACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("action", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField TARGET_LOG_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("target_log_level", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField RESET_LOG_LEVEL_TIMEOUT_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("reset_log_level_timeout_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField RESET_LOG_LEVEL_TIMEOUT_EPOCH_FIELD_DESC = new org.apache.thrift.protocol.TField("reset_log_level_timeout_epoch", org.apache.thrift.protocol.TType.I64, (short)4);
+  private static final org.apache.thrift.protocol.TField RESET_LOG_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("reset_log_level", org.apache.thrift.protocol.TType.STRING, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new LogLevelStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new LogLevelTupleSchemeFactory());
+  }
+
+  private LogLevelAction action; // required
+  private String target_log_level; // optional
+  private int reset_log_level_timeout_secs; // optional
+  private long reset_log_level_timeout_epoch; // optional
+  private String reset_log_level; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see LogLevelAction
+     */
+    ACTION((short)1, "action"),
+    TARGET_LOG_LEVEL((short)2, "target_log_level"),
+    RESET_LOG_LEVEL_TIMEOUT_SECS((short)3, "reset_log_level_timeout_secs"),
+    RESET_LOG_LEVEL_TIMEOUT_EPOCH((short)4, "reset_log_level_timeout_epoch"),
+    RESET_LOG_LEVEL((short)5, "reset_log_level");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACTION
+          return ACTION;
+        case 2: // TARGET_LOG_LEVEL
+          return TARGET_LOG_LEVEL;
+        case 3: // RESET_LOG_LEVEL_TIMEOUT_SECS
+          return RESET_LOG_LEVEL_TIMEOUT_SECS;
+        case 4: // RESET_LOG_LEVEL_TIMEOUT_EPOCH
+          return RESET_LOG_LEVEL_TIMEOUT_EPOCH;
+        case 5: // RESET_LOG_LEVEL
+          return RESET_LOG_LEVEL;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID = 0;
+  private static final int __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.TARGET_LOG_LEVEL,_Fields.RESET_LOG_LEVEL_TIMEOUT_SECS,_Fields.RESET_LOG_LEVEL_TIMEOUT_EPOCH,_Fields.RESET_LOG_LEVEL};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACTION, new org.apache.thrift.meta_data.FieldMetaData("action", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, LogLevelAction.class)));
+    tmpMap.put(_Fields.TARGET_LOG_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("target_log_level", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.RESET_LOG_LEVEL_TIMEOUT_SECS, new org.apache.thrift.meta_data.FieldMetaData("reset_log_level_timeout_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.RESET_LOG_LEVEL_TIMEOUT_EPOCH, new org.apache.thrift.meta_data.FieldMetaData("reset_log_level_timeout_epoch", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.RESET_LOG_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("reset_log_level", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LogLevel.class, metaDataMap);
+  }
+
+  public LogLevel() {
+  }
+
+  public LogLevel(
+    LogLevelAction action)
+  {
+    this();
+    this.action = action;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public LogLevel(LogLevel other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_action()) {
+      this.action = other.action;
+    }
+    if (other.is_set_target_log_level()) {
+      this.target_log_level = other.target_log_level;
+    }
+    this.reset_log_level_timeout_secs = other.reset_log_level_timeout_secs;
+    this.reset_log_level_timeout_epoch = other.reset_log_level_timeout_epoch;
+    if (other.is_set_reset_log_level()) {
+      this.reset_log_level = other.reset_log_level;
+    }
+  }
+
+  public LogLevel deepCopy() {
+    return new LogLevel(this);
+  }
+
+  @Override
+  public void clear() {
+    this.action = null;
+    this.target_log_level = null;
+    set_reset_log_level_timeout_secs_isSet(false);
+    this.reset_log_level_timeout_secs = 0;
+    set_reset_log_level_timeout_epoch_isSet(false);
+    this.reset_log_level_timeout_epoch = 0;
+    this.reset_log_level = null;
+  }
+
+  /**
+   * 
+   * @see LogLevelAction
+   */
+  public LogLevelAction get_action() {
+    return this.action;
+  }
+
+  /**
+   * 
+   * @see LogLevelAction
+   */
+  public void set_action(LogLevelAction action) {
+    this.action = action;
+  }
+
+  public void unset_action() {
+    this.action = null;
+  }
+
+  /** Returns true if field action is set (has been assigned a value) and false otherwise */
+  public boolean is_set_action() {
+    return this.action != null;
+  }
+
+  public void set_action_isSet(boolean value) {
+    if (!value) {
+      this.action = null;
+    }
+  }
+
+  public String get_target_log_level() {
+    return this.target_log_level;
+  }
+
+  public void set_target_log_level(String target_log_level) {
+    this.target_log_level = target_log_level;
+  }
+
+  public void unset_target_log_level() {
+    this.target_log_level = null;
+  }
+
+  /** Returns true if field target_log_level is set (has been assigned a value) and false otherwise */
+  public boolean is_set_target_log_level() {
+    return this.target_log_level != null;
+  }
+
+  public void set_target_log_level_isSet(boolean value) {
+    if (!value) {
+      this.target_log_level = null;
+    }
+  }
+
+  public int get_reset_log_level_timeout_secs() {
+    return this.reset_log_level_timeout_secs;
+  }
+
+  public void set_reset_log_level_timeout_secs(int reset_log_level_timeout_secs) {
+    this.reset_log_level_timeout_secs = reset_log_level_timeout_secs;
+    set_reset_log_level_timeout_secs_isSet(true);
+  }
+
+  public void unset_reset_log_level_timeout_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field reset_log_level_timeout_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_reset_log_level_timeout_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID);
+  }
+
+  public void set_reset_log_level_timeout_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_SECS_ISSET_ID, value);
+  }
+
+  public long get_reset_log_level_timeout_epoch() {
+    return this.reset_log_level_timeout_epoch;
+  }
+
+  public void set_reset_log_level_timeout_epoch(long reset_log_level_timeout_epoch) {
+    this.reset_log_level_timeout_epoch = reset_log_level_timeout_epoch;
+    set_reset_log_level_timeout_epoch_isSet(true);
+  }
+
+  public void unset_reset_log_level_timeout_epoch() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID);
+  }
+
+  /** Returns true if field reset_log_level_timeout_epoch is set (has been assigned a value) and false otherwise */
+  public boolean is_set_reset_log_level_timeout_epoch() {
+    return EncodingUtils.testBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID);
+  }
+
+  public void set_reset_log_level_timeout_epoch_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RESET_LOG_LEVEL_TIMEOUT_EPOCH_ISSET_ID, value);
+  }
+
+  public String get_reset_log_level() {
+    return this.reset_log_level;
+  }
+
+  public void set_reset_log_level(String reset_log_level) {
+    this.reset_log_level = reset_log_level;
+  }
+
+  public void unset_reset_log_level() {
+    this.reset_log_level = null;
+  }
+
+  /** Returns true if field reset_log_level is set (has been assigned a value) and false otherwise */
+  public boolean is_set_reset_log_level() {
+    return this.reset_log_level != null;
+  }
+
+  public void set_reset_log_level_isSet(boolean value) {
+    if (!value) {
+      this.reset_log_level = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACTION:
+      if (value == null) {
+        unset_action();
+      } else {
+        set_action((LogLevelAction)value);
+      }
+      break;
+
+    case TARGET_LOG_LEVEL:
+      if (value == null) {
+        unset_target_log_level();
+      } else {
+        set_target_log_level((String)value);
+      }
+      break;
+
+    case RESET_LOG_LEVEL_TIMEOUT_SECS:
+      if (value == null) {
+        unset_reset_log_level_timeout_secs();
+      } else {
+        set_reset_log_level_timeout_secs((Integer)value);
+      }
+      break;
+
+    case RESET_LOG_LEVEL_TIMEOUT_EPOCH:
+      if (value == null) {
+        unset_reset_log_level_timeout_epoch();
+      } else {
+        set_reset_log_level_timeout_epoch((Long)value);
+      }
+      break;
+
+    case RESET_LOG_LEVEL:
+      if (value == null) {
+        unset_reset_log_level();
+      } else {
+        set_reset_log_level((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACTION:
+      return get_action();
+
+    case TARGET_LOG_LEVEL:
+      return get_target_log_level();
+
+    case RESET_LOG_LEVEL_TIMEOUT_SECS:
+      return get_reset_log_level_timeout_secs();
+
+    case RESET_LOG_LEVEL_TIMEOUT_EPOCH:
+      return get_reset_log_level_timeout_epoch();
+
+    case RESET_LOG_LEVEL:
+      return get_reset_log_level();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ACTION:
+      return is_set_action();
+    case TARGET_LOG_LEVEL:
+      return is_set_target_log_level();
+    case RESET_LOG_LEVEL_TIMEOUT_SECS:
+      return is_set_reset_log_level_timeout_secs();
+    case RESET_LOG_LEVEL_TIMEOUT_EPOCH:
+      return is_set_reset_log_level_timeout_epoch();
+    case RESET_LOG_LEVEL:
+      return is_set_reset_log_level();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof LogLevel)
+      return this.equals((LogLevel)that);
+    return false;
+  }
+
+  public boolean equals(LogLevel that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_action = true && this.is_set_action();
+    boolean that_present_action = true && that.is_set_action();
+    if (this_present_action || that_present_action) {
+      if (!(this_present_action && that_present_action))
+        return false;
+      if (!this.action.equals(that.action))
+        return false;
+    }
+
+    boolean this_present_target_log_level = true && this.is_set_target_log_level();
+    boolean that_present_target_log_level = true && that.is_set_target_log_level();
+    if (this_present_target_log_level || that_present_target_log_level) {
+      if (!(this_present_target_log_level && that_present_target_log_level))
+        return false;
+      if (!this.target_log_level.equals(that.target_log_level))
+        return false;
+    }
+
+    boolean this_present_reset_log_level_timeout_secs = true && this.is_set_reset_log_level_timeout_secs();
+    boolean that_present_reset_log_level_timeout_secs = true && that.is_set_reset_log_level_timeout_secs();
+    if (this_present_reset_log_level_timeout_secs || that_present_reset_log_level_timeout_secs) {
+      if (!(this_present_reset_log_level_timeout_secs && that_present_reset_log_level_timeout_secs))
+        return false;
+      if (this.reset_log_level_timeout_secs != that.reset_log_level_timeout_secs)
+        return false;
+    }
+
+    boolean this_present_reset_log_level_timeout_epoch = true && this.is_set_reset_log_level_timeout_epoch();
+    boolean that_present_reset_log_level_timeout_epoch = true && that.is_set_reset_log_level_timeout_epoch();
+    if (this_present_reset_log_level_timeout_epoch || that_present_reset_log_level_timeout_epoch) {
+      if (!(this_present_reset_log_level_timeout_epoch && that_present_reset_log_level_timeout_epoch))
+        return false;
+      if (this.reset_log_level_timeout_epoch != that.reset_log_level_timeout_epoch)
+        return false;
+    }
+
+    boolean this_present_reset_log_level = true && this.is_set_reset_log_level();
+    boolean that_present_reset_log_level = true && that.is_set_reset_log_level();
+    if (this_present_reset_log_level || that_present_reset_log_level) {
+      if (!(this_present_reset_log_level && that_present_reset_log_level))
+        return false;
+      if (!this.reset_log_level.equals(that.reset_log_level))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_action = true && (is_set_action());
+    list.add(present_action);
+    if (present_action)
+      list.add(action.getValue());
+
+    boolean present_target_log_level = true && (is_set_target_log_level());
+    list.add(present_target_log_level);
+    if (present_target_log_level)
+      list.add(target_log_level);
+
+    boolean present_reset_log_level_timeout_secs = true && (is_set_reset_log_level_timeout_secs());
+    list.add(present_reset_log_level_timeout_secs);
+    if (present_reset_log_level_timeout_secs)
+      list.add(reset_log_level_timeout_secs);
+
+    boolean present_reset_log_level_timeout_epoch = true && (is_set_reset_log_level_timeout_epoch());
+    list.add(present_reset_log_level_timeout_epoch);
+    if (present_reset_log_level_timeout_epoch)
+      list.add(reset_log_level_timeout_epoch);
+
+    boolean present_reset_log_level = true && (is_set_reset_log_level());
+    list.add(present_reset_log_level);
+    if (present_reset_log_level)
+      list.add(reset_log_level);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(LogLevel other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_action()).compareTo(other.is_set_action());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_action()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.action, other.action);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_target_log_level()).compareTo(other.is_set_target_log_level());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_target_log_level()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.target_log_level, other.target_log_level);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_reset_log_level_timeout_secs()).compareTo(other.is_set_reset_log_level_timeout_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_reset_log_level_timeout_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reset_log_level_timeout_secs, other.reset_log_level_timeout_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_reset_log_level_timeout_epoch()).compareTo(other.is_set_reset_log_level_timeout_epoch());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_reset_log_level_timeout_epoch()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reset_log_level_timeout_epoch, other.reset_log_level_timeout_epoch);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_reset_log_level()).compareTo(other.is_set_reset_log_level());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_reset_log_level()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reset_log_level, other.reset_log_level);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("LogLevel(");
+    boolean first = true;
+
+    sb.append("action:");
+    if (this.action == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.action);
+    }
+    first = false;
+    if (is_set_target_log_level()) {
+      if (!first) sb.append(", ");
+      sb.append("target_log_level:");
+      if (this.target_log_level == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.target_log_level);
+      }
+      first = false;
+    }
+    if (is_set_reset_log_level_timeout_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("reset_log_level_timeout_secs:");
+      sb.append(this.reset_log_level_timeout_secs);
+      first = false;
+    }
+    if (is_set_reset_log_level_timeout_epoch()) {
+      if (!first) sb.append(", ");
+      sb.append("reset_log_level_timeout_epoch:");
+      sb.append(this.reset_log_level_timeout_epoch);
+      first = false;
+    }
+    if (is_set_reset_log_level()) {
+      if (!first) sb.append(", ");
+      sb.append("reset_log_level:");
+      if (this.reset_log_level == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.reset_log_level);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_action()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'action' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class LogLevelStandardSchemeFactory implements SchemeFactory {
+    public LogLevelStandardScheme getScheme() {
+      return new LogLevelStandardScheme();
+    }
+  }
+
+  private static class LogLevelStandardScheme extends StandardScheme<LogLevel> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, LogLevel struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ACTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.action = backtype.storm.generated.LogLevelAction.findByValue(iprot.readI32());
+              struct.set_action_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // TARGET_LOG_LEVEL
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.target_log_level = iprot.readString();
+              struct.set_target_log_level_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // RESET_LOG_LEVEL_TIMEOUT_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.reset_log_level_timeout_secs = iprot.readI32();
+              struct.set_reset_log_level_timeout_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // RESET_LOG_LEVEL_TIMEOUT_EPOCH
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.reset_log_level_timeout_epoch = iprot.readI64();
+              struct.set_reset_log_level_timeout_epoch_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // RESET_LOG_LEVEL
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.reset_log_level = iprot.readString();
+              struct.set_reset_log_level_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, LogLevel struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.action != null) {
+        oprot.writeFieldBegin(ACTION_FIELD_DESC);
+        oprot.writeI32(struct.action.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.target_log_level != null) {
+        if (struct.is_set_target_log_level()) {
+          oprot.writeFieldBegin(TARGET_LOG_LEVEL_FIELD_DESC);
+          oprot.writeString(struct.target_log_level);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_reset_log_level_timeout_secs()) {
+        oprot.writeFieldBegin(RESET_LOG_LEVEL_TIMEOUT_SECS_FIELD_DESC);
+        oprot.writeI32(struct.reset_log_level_timeout_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_reset_log_level_timeout_epoch()) {
+        oprot.writeFieldBegin(RESET_LOG_LEVEL_TIMEOUT_EPOCH_FIELD_DESC);
+        oprot.writeI64(struct.reset_log_level_timeout_epoch);
+        oprot.writeFieldEnd();
+      }
+      if (struct.reset_log_level != null) {
+        if (struct.is_set_reset_log_level()) {
+          oprot.writeFieldBegin(RESET_LOG_LEVEL_FIELD_DESC);
+          oprot.writeString(struct.reset_log_level);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class LogLevelTupleSchemeFactory implements SchemeFactory {
+    public LogLevelTupleScheme getScheme() {
+      return new LogLevelTupleScheme();
+    }
+  }
+
+  private static class LogLevelTupleScheme extends TupleScheme<LogLevel> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, LogLevel struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI32(struct.action.getValue());
+      BitSet optionals = new BitSet();
+      if (struct.is_set_target_log_level()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_reset_log_level_timeout_secs()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_reset_log_level_timeout_epoch()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_reset_log_level()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_target_log_level()) {
+        oprot.writeString(struct.target_log_level);
+      }
+      if (struct.is_set_reset_log_level_timeout_secs()) {
+        oprot.writeI32(struct.reset_log_level_timeout_secs);
+      }
+      if (struct.is_set_reset_log_level_timeout_epoch()) {
+        oprot.writeI64(struct.reset_log_level_timeout_epoch);
+      }
+      if (struct.is_set_reset_log_level()) {
+        oprot.writeString(struct.reset_log_level);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, LogLevel struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.action = backtype.storm.generated.LogLevelAction.findByValue(iprot.readI32());
+      struct.set_action_isSet(true);
+      BitSet incoming = iprot.readBitSet(4);
+      if (incoming.get(0)) {
+        struct.target_log_level = iprot.readString();
+        struct.set_target_log_level_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.reset_log_level_timeout_secs = iprot.readI32();
+        struct.set_reset_log_level_timeout_secs_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.reset_log_level_timeout_epoch = iprot.readI64();
+        struct.set_reset_log_level_timeout_epoch_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.reset_log_level = iprot.readString();
+        struct.set_reset_log_level_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/LogLevelAction.java b/storm-core/src/jvm/backtype/storm/generated/LogLevelAction.java
new file mode 100644
index 0000000..fed2365
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/LogLevelAction.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum LogLevelAction implements org.apache.thrift.TEnum {
+  UNCHANGED(1),
+  UPDATE(2),
+  REMOVE(3);
+
+  private final int value;
+
+  private LogLevelAction(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static LogLevelAction findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return UNCHANGED;
+      case 2:
+        return UPDATE;
+      case 3:
+        return REMOVE;
+      default:
+        return null;
+    }
+  }
+}
diff --git a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
index 3a5d5df..98d2d1c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/Nimbus.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-8-24")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class Nimbus {
 
   public interface Iface {
@@ -70,6 +70,10 @@
 
     public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
+    public void setLogConfig(String name, LogConfig config) throws org.apache.thrift.TException;
+
+    public LogConfig getLogConfig(String name) throws org.apache.thrift.TException;
+
     /**
      * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
      * and if null or empty, the debug flag will apply to the entire topology.
@@ -84,6 +88,10 @@
      */
     public void debug(String name, String component, boolean enable, double samplingPercentage) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest) throws org.apache.thrift.TException;
+
+    public List<ProfileRequest> getComponentPendingProfileActions(String id, String component_id, ProfileAction action) throws org.apache.thrift.TException;
+
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException;
 
     public String beginFileUpload() throws AuthorizationException, org.apache.thrift.TException;
@@ -104,6 +112,10 @@
 
     public TopologyInfo getTopologyInfoWithOpts(String id, GetInfoOptions options) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
+    public TopologyPageInfo getTopologyPageInfo(String id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
+    public ComponentPageInfo getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
+
     public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
     /**
@@ -120,6 +132,8 @@
      */
     public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException;
 
+    public TopologyHistoryInfo getTopologyHistory(String user) throws AuthorizationException, org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -138,8 +152,16 @@
 
     public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void setLogConfig(String name, LogConfig config, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getLogConfig(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void debug(String name, String component, boolean enable, double samplingPercentage, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getComponentPendingProfileActions(String id, String component_id, ProfileAction action, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -160,12 +182,18 @@
 
     public void getTopologyInfoWithOpts(String id, GetInfoOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void getTopologyPageInfo(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void getTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void getUserTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void getTopologyHistory(String user, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -388,6 +416,50 @@
       return;
     }
 
+    public void setLogConfig(String name, LogConfig config) throws org.apache.thrift.TException
+    {
+      send_setLogConfig(name, config);
+      recv_setLogConfig();
+    }
+
+    public void send_setLogConfig(String name, LogConfig config) throws org.apache.thrift.TException
+    {
+      setLogConfig_args args = new setLogConfig_args();
+      args.set_name(name);
+      args.set_config(config);
+      sendBase("setLogConfig", args);
+    }
+
+    public void recv_setLogConfig() throws org.apache.thrift.TException
+    {
+      setLogConfig_result result = new setLogConfig_result();
+      receiveBase(result, "setLogConfig");
+      return;
+    }
+
+    public LogConfig getLogConfig(String name) throws org.apache.thrift.TException
+    {
+      send_getLogConfig(name);
+      return recv_getLogConfig();
+    }
+
+    public void send_getLogConfig(String name) throws org.apache.thrift.TException
+    {
+      getLogConfig_args args = new getLogConfig_args();
+      args.set_name(name);
+      sendBase("getLogConfig", args);
+    }
+
+    public LogConfig recv_getLogConfig() throws org.apache.thrift.TException
+    {
+      getLogConfig_result result = new getLogConfig_result();
+      receiveBase(result, "getLogConfig");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getLogConfig failed: unknown result");
+    }
+
     public void debug(String name, String component, boolean enable, double samplingPercentage) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_debug(name, component, enable, samplingPercentage);
@@ -417,6 +489,52 @@
       return;
     }
 
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest) throws org.apache.thrift.TException
+    {
+      send_setWorkerProfiler(id, profileRequest);
+      recv_setWorkerProfiler();
+    }
+
+    public void send_setWorkerProfiler(String id, ProfileRequest profileRequest) throws org.apache.thrift.TException
+    {
+      setWorkerProfiler_args args = new setWorkerProfiler_args();
+      args.set_id(id);
+      args.set_profileRequest(profileRequest);
+      sendBase("setWorkerProfiler", args);
+    }
+
+    public void recv_setWorkerProfiler() throws org.apache.thrift.TException
+    {
+      setWorkerProfiler_result result = new setWorkerProfiler_result();
+      receiveBase(result, "setWorkerProfiler");
+      return;
+    }
+
+    public List<ProfileRequest> getComponentPendingProfileActions(String id, String component_id, ProfileAction action) throws org.apache.thrift.TException
+    {
+      send_getComponentPendingProfileActions(id, component_id, action);
+      return recv_getComponentPendingProfileActions();
+    }
+
+    public void send_getComponentPendingProfileActions(String id, String component_id, ProfileAction action) throws org.apache.thrift.TException
+    {
+      getComponentPendingProfileActions_args args = new getComponentPendingProfileActions_args();
+      args.set_id(id);
+      args.set_component_id(component_id);
+      args.set_action(action);
+      sendBase("getComponentPendingProfileActions", args);
+    }
+
+    public List<ProfileRequest> recv_getComponentPendingProfileActions() throws org.apache.thrift.TException
+    {
+      getComponentPendingProfileActions_result result = new getComponentPendingProfileActions_result();
+      receiveBase(result, "getComponentPendingProfileActions");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getComponentPendingProfileActions failed: unknown result");
+    }
+
     public void uploadNewCredentials(String name, Credentials creds) throws NotAliveException, InvalidTopologyException, AuthorizationException, org.apache.thrift.TException
     {
       send_uploadNewCredentials(name, creds);
@@ -680,6 +798,69 @@
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result");
     }
 
+    public TopologyPageInfo getTopologyPageInfo(String id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopologyPageInfo(id, window, is_include_sys);
+      return recv_getTopologyPageInfo();
+    }
+
+    public void send_getTopologyPageInfo(String id, String window, boolean is_include_sys) throws org.apache.thrift.TException
+    {
+      getTopologyPageInfo_args args = new getTopologyPageInfo_args();
+      args.set_id(id);
+      args.set_window(window);
+      args.set_is_include_sys(is_include_sys);
+      sendBase("getTopologyPageInfo", args);
+    }
+
+    public TopologyPageInfo recv_getTopologyPageInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getTopologyPageInfo_result result = new getTopologyPageInfo_result();
+      receiveBase(result, "getTopologyPageInfo");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyPageInfo failed: unknown result");
+    }
+
+    public ComponentPageInfo getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      send_getComponentPageInfo(topology_id, component_id, window, is_include_sys);
+      return recv_getComponentPageInfo();
+    }
+
+    public void send_getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys) throws org.apache.thrift.TException
+    {
+      getComponentPageInfo_args args = new getComponentPageInfo_args();
+      args.set_topology_id(topology_id);
+      args.set_component_id(component_id);
+      args.set_window(window);
+      args.set_is_include_sys(is_include_sys);
+      sendBase("getComponentPageInfo", args);
+    }
+
+    public ComponentPageInfo recv_getComponentPageInfo() throws NotAliveException, AuthorizationException, org.apache.thrift.TException
+    {
+      getComponentPageInfo_result result = new getComponentPageInfo_result();
+      receiveBase(result, "getComponentPageInfo");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getComponentPageInfo failed: unknown result");
+    }
+
     public String getTopologyConf(String id) throws NotAliveException, AuthorizationException, org.apache.thrift.TException
     {
       send_getTopologyConf(id);
@@ -767,6 +948,32 @@
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result");
     }
 
+    public TopologyHistoryInfo getTopologyHistory(String user) throws AuthorizationException, org.apache.thrift.TException
+    {
+      send_getTopologyHistory(user);
+      return recv_getTopologyHistory();
+    }
+
+    public void send_getTopologyHistory(String user) throws org.apache.thrift.TException
+    {
+      getTopologyHistory_args args = new getTopologyHistory_args();
+      args.set_user(user);
+      sendBase("getTopologyHistory", args);
+    }
+
+    public TopologyHistoryInfo recv_getTopologyHistory() throws AuthorizationException, org.apache.thrift.TException
+    {
+      getTopologyHistory_result result = new getTopologyHistory_result();
+      receiveBase(result, "getTopologyHistory");
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.aze != null) {
+        throw result.aze;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyHistory failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -1036,6 +1243,73 @@
       }
     }
 
+    public void setLogConfig(String name, LogConfig config, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      setLogConfig_call method_call = new setLogConfig_call(name, config, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class setLogConfig_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      private LogConfig config;
+      public setLogConfig_call(String name, LogConfig config, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.config = config;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setLogConfig", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        setLogConfig_args args = new setLogConfig_args();
+        args.set_name(name);
+        args.set_config(config);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_setLogConfig();
+      }
+    }
+
+    public void getLogConfig(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getLogConfig_call method_call = new getLogConfig_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getLogConfig_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String name;
+      public getLogConfig_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getLogConfig", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getLogConfig_args args = new getLogConfig_args();
+        args.set_name(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public LogConfig getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getLogConfig();
+      }
+    }
+
     public void debug(String name, String component, boolean enable, double samplingPercentage, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       debug_call method_call = new debug_call(name, component, enable, samplingPercentage, resultHandler, this, ___protocolFactory, ___transport);
@@ -1077,6 +1351,79 @@
       }
     }
 
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      setWorkerProfiler_call method_call = new setWorkerProfiler_call(id, profileRequest, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class setWorkerProfiler_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      private ProfileRequest profileRequest;
+      public setWorkerProfiler_call(String id, ProfileRequest profileRequest, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.profileRequest = profileRequest;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("setWorkerProfiler", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        setWorkerProfiler_args args = new setWorkerProfiler_args();
+        args.set_id(id);
+        args.set_profileRequest(profileRequest);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_setWorkerProfiler();
+      }
+    }
+
+    public void getComponentPendingProfileActions(String id, String component_id, ProfileAction action, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getComponentPendingProfileActions_call method_call = new getComponentPendingProfileActions_call(id, component_id, action, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getComponentPendingProfileActions_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      private String component_id;
+      private ProfileAction action;
+      public getComponentPendingProfileActions_call(String id, String component_id, ProfileAction action, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.component_id = component_id;
+        this.action = action;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getComponentPendingProfileActions", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getComponentPendingProfileActions_args args = new getComponentPendingProfileActions_args();
+        args.set_id(id);
+        args.set_component_id(component_id);
+        args.set_action(action);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<ProfileRequest> getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getComponentPendingProfileActions();
+      }
+    }
+
     public void uploadNewCredentials(String name, Credentials creds, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       uploadNewCredentials_call method_call = new uploadNewCredentials_call(name, creds, resultHandler, this, ___protocolFactory, ___transport);
@@ -1397,6 +1744,85 @@
       }
     }
 
+    public void getTopologyPageInfo(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTopologyPageInfo_call method_call = new getTopologyPageInfo_call(id, window, is_include_sys, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getTopologyPageInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String id;
+      private String window;
+      private boolean is_include_sys;
+      public getTopologyPageInfo_call(String id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.window = window;
+        this.is_include_sys = is_include_sys;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyPageInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTopologyPageInfo_args args = new getTopologyPageInfo_args();
+        args.set_id(id);
+        args.set_window(window);
+        args.set_is_include_sys(is_include_sys);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TopologyPageInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTopologyPageInfo();
+      }
+    }
+
+    public void getComponentPageInfo(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getComponentPageInfo_call method_call = new getComponentPageInfo_call(topology_id, component_id, window, is_include_sys, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getComponentPageInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String topology_id;
+      private String component_id;
+      private String window;
+      private boolean is_include_sys;
+      public getComponentPageInfo_call(String topology_id, String component_id, String window, boolean is_include_sys, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.topology_id = topology_id;
+        this.component_id = component_id;
+        this.window = window;
+        this.is_include_sys = is_include_sys;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getComponentPageInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getComponentPageInfo_args args = new getComponentPageInfo_args();
+        args.set_topology_id(topology_id);
+        args.set_component_id(component_id);
+        args.set_window(window);
+        args.set_is_include_sys(is_include_sys);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public ComponentPageInfo getResult() throws NotAliveException, AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getComponentPageInfo();
+      }
+    }
+
     public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getTopologyConf_call method_call = new getTopologyConf_call(id, resultHandler, this, ___protocolFactory, ___transport);
@@ -1493,6 +1919,38 @@
       }
     }
 
+    public void getTopologyHistory(String user, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      getTopologyHistory_call method_call = new getTopologyHistory_call(user, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class getTopologyHistory_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private String user;
+      public getTopologyHistory_call(String user, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.user = user;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyHistory", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        getTopologyHistory_args args = new getTopologyHistory_args();
+        args.set_user(user);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TopologyHistoryInfo getResult() throws AuthorizationException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTopologyHistory();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -1513,7 +1971,11 @@
       processMap.put("activate", new activate());
       processMap.put("deactivate", new deactivate());
       processMap.put("rebalance", new rebalance());
+      processMap.put("setLogConfig", new setLogConfig());
+      processMap.put("getLogConfig", new getLogConfig());
       processMap.put("debug", new debug());
+      processMap.put("setWorkerProfiler", new setWorkerProfiler());
+      processMap.put("getComponentPendingProfileActions", new getComponentPendingProfileActions());
       processMap.put("uploadNewCredentials", new uploadNewCredentials());
       processMap.put("beginFileUpload", new beginFileUpload());
       processMap.put("uploadChunk", new uploadChunk());
@@ -1524,9 +1986,12 @@
       processMap.put("getClusterInfo", new getClusterInfo());
       processMap.put("getTopologyInfo", new getTopologyInfo());
       processMap.put("getTopologyInfoWithOpts", new getTopologyInfoWithOpts());
+      processMap.put("getTopologyPageInfo", new getTopologyPageInfo());
+      processMap.put("getComponentPageInfo", new getComponentPageInfo());
       processMap.put("getTopologyConf", new getTopologyConf());
       processMap.put("getTopology", new getTopology());
       processMap.put("getUserTopology", new getUserTopology());
+      processMap.put("getTopologyHistory", new getTopologyHistory());
       return processMap;
     }
 
@@ -1718,6 +2183,46 @@
       }
     }
 
+    public static class setLogConfig<I extends Iface> extends org.apache.thrift.ProcessFunction<I, setLogConfig_args> {
+      public setLogConfig() {
+        super("setLogConfig");
+      }
+
+      public setLogConfig_args getEmptyArgsInstance() {
+        return new setLogConfig_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public setLogConfig_result getResult(I iface, setLogConfig_args args) throws org.apache.thrift.TException {
+        setLogConfig_result result = new setLogConfig_result();
+        iface.setLogConfig(args.name, args.config);
+        return result;
+      }
+    }
+
+    public static class getLogConfig<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getLogConfig_args> {
+      public getLogConfig() {
+        super("getLogConfig");
+      }
+
+      public getLogConfig_args getEmptyArgsInstance() {
+        return new getLogConfig_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public getLogConfig_result getResult(I iface, getLogConfig_args args) throws org.apache.thrift.TException {
+        getLogConfig_result result = new getLogConfig_result();
+        result.success = iface.getLogConfig(args.name);
+        return result;
+      }
+    }
+
     public static class debug<I extends Iface> extends org.apache.thrift.ProcessFunction<I, debug_args> {
       public debug() {
         super("debug");
@@ -1744,6 +2249,46 @@
       }
     }
 
+    public static class setWorkerProfiler<I extends Iface> extends org.apache.thrift.ProcessFunction<I, setWorkerProfiler_args> {
+      public setWorkerProfiler() {
+        super("setWorkerProfiler");
+      }
+
+      public setWorkerProfiler_args getEmptyArgsInstance() {
+        return new setWorkerProfiler_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public setWorkerProfiler_result getResult(I iface, setWorkerProfiler_args args) throws org.apache.thrift.TException {
+        setWorkerProfiler_result result = new setWorkerProfiler_result();
+        iface.setWorkerProfiler(args.id, args.profileRequest);
+        return result;
+      }
+    }
+
+    public static class getComponentPendingProfileActions<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getComponentPendingProfileActions_args> {
+      public getComponentPendingProfileActions() {
+        super("getComponentPendingProfileActions");
+      }
+
+      public getComponentPendingProfileActions_args getEmptyArgsInstance() {
+        return new getComponentPendingProfileActions_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public getComponentPendingProfileActions_result getResult(I iface, getComponentPendingProfileActions_args args) throws org.apache.thrift.TException {
+        getComponentPendingProfileActions_result result = new getComponentPendingProfileActions_result();
+        result.success = iface.getComponentPendingProfileActions(args.id, args.component_id, args.action);
+        return result;
+      }
+    }
+
     public static class uploadNewCredentials<I extends Iface> extends org.apache.thrift.ProcessFunction<I, uploadNewCredentials_args> {
       public uploadNewCredentials() {
         super("uploadNewCredentials");
@@ -1992,6 +2537,58 @@
       }
     }
 
+    public static class getTopologyPageInfo<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTopologyPageInfo_args> {
+      public getTopologyPageInfo() {
+        super("getTopologyPageInfo");
+      }
+
+      public getTopologyPageInfo_args getEmptyArgsInstance() {
+        return new getTopologyPageInfo_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public getTopologyPageInfo_result getResult(I iface, getTopologyPageInfo_args args) throws org.apache.thrift.TException {
+        getTopologyPageInfo_result result = new getTopologyPageInfo_result();
+        try {
+          result.success = iface.getTopologyPageInfo(args.id, args.window, args.is_include_sys);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
+    public static class getComponentPageInfo<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getComponentPageInfo_args> {
+      public getComponentPageInfo() {
+        super("getComponentPageInfo");
+      }
+
+      public getComponentPageInfo_args getEmptyArgsInstance() {
+        return new getComponentPageInfo_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public getComponentPageInfo_result getResult(I iface, getComponentPageInfo_args args) throws org.apache.thrift.TException {
+        getComponentPageInfo_result result = new getComponentPageInfo_result();
+        try {
+          result.success = iface.getComponentPageInfo(args.topology_id, args.component_id, args.window, args.is_include_sys);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
     public static class getTopologyConf<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTopologyConf_args> {
       public getTopologyConf() {
         super("getTopologyConf");
@@ -2070,6 +2667,30 @@
       }
     }
 
+    public static class getTopologyHistory<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getTopologyHistory_args> {
+      public getTopologyHistory() {
+        super("getTopologyHistory");
+      }
+
+      public getTopologyHistory_args getEmptyArgsInstance() {
+        return new getTopologyHistory_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public getTopologyHistory_result getResult(I iface, getTopologyHistory_args args) throws org.apache.thrift.TException {
+        getTopologyHistory_result result = new getTopologyHistory_result();
+        try {
+          result.success = iface.getTopologyHistory(args.user);
+        } catch (AuthorizationException aze) {
+          result.aze = aze;
+        }
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends org.apache.thrift.TBaseAsyncProcessor<I> {
@@ -2090,7 +2711,11 @@
       processMap.put("activate", new activate());
       processMap.put("deactivate", new deactivate());
       processMap.put("rebalance", new rebalance());
+      processMap.put("setLogConfig", new setLogConfig());
+      processMap.put("getLogConfig", new getLogConfig());
       processMap.put("debug", new debug());
+      processMap.put("setWorkerProfiler", new setWorkerProfiler());
+      processMap.put("getComponentPendingProfileActions", new getComponentPendingProfileActions());
       processMap.put("uploadNewCredentials", new uploadNewCredentials());
       processMap.put("beginFileUpload", new beginFileUpload());
       processMap.put("uploadChunk", new uploadChunk());
@@ -2101,9 +2726,12 @@
       processMap.put("getClusterInfo", new getClusterInfo());
       processMap.put("getTopologyInfo", new getTopologyInfo());
       processMap.put("getTopologyInfoWithOpts", new getTopologyInfoWithOpts());
+      processMap.put("getTopologyPageInfo", new getTopologyPageInfo());
+      processMap.put("getComponentPageInfo", new getComponentPageInfo());
       processMap.put("getTopologyConf", new getTopologyConf());
       processMap.put("getTopology", new getTopology());
       processMap.put("getUserTopology", new getUserTopology());
+      processMap.put("getTopologyHistory", new getTopologyHistory());
       return processMap;
     }
 
@@ -2549,6 +3177,107 @@
       }
     }
 
+    public static class setLogConfig<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, setLogConfig_args, Void> {
+      public setLogConfig() {
+        super("setLogConfig");
+      }
+
+      public setLogConfig_args getEmptyArgsInstance() {
+        return new setLogConfig_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            setLogConfig_result result = new setLogConfig_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            setLogConfig_result result = new setLogConfig_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, setLogConfig_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.setLogConfig(args.name, args.config,resultHandler);
+      }
+    }
+
+    public static class getLogConfig<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getLogConfig_args, LogConfig> {
+      public getLogConfig() {
+        super("getLogConfig");
+      }
+
+      public getLogConfig_args getEmptyArgsInstance() {
+        return new getLogConfig_args();
+      }
+
+      public AsyncMethodCallback<LogConfig> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<LogConfig>() { 
+          public void onComplete(LogConfig o) {
+            getLogConfig_result result = new getLogConfig_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            getLogConfig_result result = new getLogConfig_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getLogConfig_args args, org.apache.thrift.async.AsyncMethodCallback<LogConfig> resultHandler) throws TException {
+        iface.getLogConfig(args.name,resultHandler);
+      }
+    }
+
     public static class debug<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, debug_args, Void> {
       public debug() {
         super("debug");
@@ -2610,6 +3339,107 @@
       }
     }
 
+    public static class setWorkerProfiler<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, setWorkerProfiler_args, Void> {
+      public setWorkerProfiler() {
+        super("setWorkerProfiler");
+      }
+
+      public setWorkerProfiler_args getEmptyArgsInstance() {
+        return new setWorkerProfiler_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            setWorkerProfiler_result result = new setWorkerProfiler_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            setWorkerProfiler_result result = new setWorkerProfiler_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, setWorkerProfiler_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.setWorkerProfiler(args.id, args.profileRequest,resultHandler);
+      }
+    }
+
+    public static class getComponentPendingProfileActions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getComponentPendingProfileActions_args, List<ProfileRequest>> {
+      public getComponentPendingProfileActions() {
+        super("getComponentPendingProfileActions");
+      }
+
+      public getComponentPendingProfileActions_args getEmptyArgsInstance() {
+        return new getComponentPendingProfileActions_args();
+      }
+
+      public AsyncMethodCallback<List<ProfileRequest>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<List<ProfileRequest>>() { 
+          public void onComplete(List<ProfileRequest> o) {
+            getComponentPendingProfileActions_result result = new getComponentPendingProfileActions_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            getComponentPendingProfileActions_result result = new getComponentPendingProfileActions_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getComponentPendingProfileActions_args args, org.apache.thrift.async.AsyncMethodCallback<List<ProfileRequest>> resultHandler) throws TException {
+        iface.getComponentPendingProfileActions(args.id, args.component_id, args.action,resultHandler);
+      }
+    }
+
     public static class uploadNewCredentials<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, uploadNewCredentials_args, Void> {
       public uploadNewCredentials() {
         super("uploadNewCredentials");
@@ -3197,6 +4027,130 @@
       }
     }
 
+    public static class getTopologyPageInfo<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTopologyPageInfo_args, TopologyPageInfo> {
+      public getTopologyPageInfo() {
+        super("getTopologyPageInfo");
+      }
+
+      public getTopologyPageInfo_args getEmptyArgsInstance() {
+        return new getTopologyPageInfo_args();
+      }
+
+      public AsyncMethodCallback<TopologyPageInfo> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<TopologyPageInfo>() { 
+          public void onComplete(TopologyPageInfo o) {
+            getTopologyPageInfo_result result = new getTopologyPageInfo_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            getTopologyPageInfo_result result = new getTopologyPageInfo_result();
+            if (e instanceof NotAliveException) {
+                        result.e = (NotAliveException) e;
+                        result.set_e_isSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getTopologyPageInfo_args args, org.apache.thrift.async.AsyncMethodCallback<TopologyPageInfo> resultHandler) throws TException {
+        iface.getTopologyPageInfo(args.id, args.window, args.is_include_sys,resultHandler);
+      }
+    }
+
+    public static class getComponentPageInfo<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getComponentPageInfo_args, ComponentPageInfo> {
+      public getComponentPageInfo() {
+        super("getComponentPageInfo");
+      }
+
+      public getComponentPageInfo_args getEmptyArgsInstance() {
+        return new getComponentPageInfo_args();
+      }
+
+      public AsyncMethodCallback<ComponentPageInfo> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<ComponentPageInfo>() { 
+          public void onComplete(ComponentPageInfo o) {
+            getComponentPageInfo_result result = new getComponentPageInfo_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            getComponentPageInfo_result result = new getComponentPageInfo_result();
+            if (e instanceof NotAliveException) {
+                        result.e = (NotAliveException) e;
+                        result.set_e_isSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getComponentPageInfo_args args, org.apache.thrift.async.AsyncMethodCallback<ComponentPageInfo> resultHandler) throws TException {
+        iface.getComponentPageInfo(args.topology_id, args.component_id, args.window, args.is_include_sys,resultHandler);
+      }
+    }
+
     public static class getTopologyConf<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTopologyConf_args, String> {
       public getTopologyConf() {
         super("getTopologyConf");
@@ -3383,6 +4337,63 @@
       }
     }
 
+    public static class getTopologyHistory<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getTopologyHistory_args, TopologyHistoryInfo> {
+      public getTopologyHistory() {
+        super("getTopologyHistory");
+      }
+
+      public getTopologyHistory_args getEmptyArgsInstance() {
+        return new getTopologyHistory_args();
+      }
+
+      public AsyncMethodCallback<TopologyHistoryInfo> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<TopologyHistoryInfo>() { 
+          public void onComplete(TopologyHistoryInfo o) {
+            getTopologyHistory_result result = new getTopologyHistory_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            getTopologyHistory_result result = new getTopologyHistory_result();
+            if (e instanceof AuthorizationException) {
+                        result.aze = (AuthorizationException) e;
+                        result.set_aze_isSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, getTopologyHistory_args args, org.apache.thrift.async.AsyncMethodCallback<TopologyHistoryInfo> resultHandler) throws TException {
+        iface.getTopologyHistory(args.user,resultHandler);
+      }
+    }
+
   }
 
   public static class submitTopology_args implements org.apache.thrift.TBase<submitTopology_args, submitTopology_args._Fields>, java.io.Serializable, Cloneable, Comparable<submitTopology_args>   {
@@ -10432,6 +11443,1440 @@
 
   }
 
+  public static class setLogConfig_args implements org.apache.thrift.TBase<setLogConfig_args, setLogConfig_args._Fields>, java.io.Serializable, Cloneable, Comparable<setLogConfig_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setLogConfig_args");
+
+    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField CONFIG_FIELD_DESC = new org.apache.thrift.protocol.TField("config", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new setLogConfig_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setLogConfig_argsTupleSchemeFactory());
+    }
+
+    private String name; // required
+    private LogConfig config; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      NAME((short)1, "name"),
+      CONFIG((short)2, "config");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // NAME
+            return NAME;
+          case 2: // CONFIG
+            return CONFIG;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.CONFIG, new org.apache.thrift.meta_data.FieldMetaData("config", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LogConfig.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setLogConfig_args.class, metaDataMap);
+    }
+
+    public setLogConfig_args() {
+    }
+
+    public setLogConfig_args(
+      String name,
+      LogConfig config)
+    {
+      this();
+      this.name = name;
+      this.config = config;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public setLogConfig_args(setLogConfig_args other) {
+      if (other.is_set_name()) {
+        this.name = other.name;
+      }
+      if (other.is_set_config()) {
+        this.config = new LogConfig(other.config);
+      }
+    }
+
+    public setLogConfig_args deepCopy() {
+      return new setLogConfig_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.name = null;
+      this.config = null;
+    }
+
+    public String get_name() {
+      return this.name;
+    }
+
+    public void set_name(String name) {
+      this.name = name;
+    }
+
+    public void unset_name() {
+      this.name = null;
+    }
+
+    /** Returns true if field name is set (has been assigned a value) and false otherwise */
+    public boolean is_set_name() {
+      return this.name != null;
+    }
+
+    public void set_name_isSet(boolean value) {
+      if (!value) {
+        this.name = null;
+      }
+    }
+
+    public LogConfig get_config() {
+      return this.config;
+    }
+
+    public void set_config(LogConfig config) {
+      this.config = config;
+    }
+
+    public void unset_config() {
+      this.config = null;
+    }
+
+    /** Returns true if field config is set (has been assigned a value) and false otherwise */
+    public boolean is_set_config() {
+      return this.config != null;
+    }
+
+    public void set_config_isSet(boolean value) {
+      if (!value) {
+        this.config = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case NAME:
+        if (value == null) {
+          unset_name();
+        } else {
+          set_name((String)value);
+        }
+        break;
+
+      case CONFIG:
+        if (value == null) {
+          unset_config();
+        } else {
+          set_config((LogConfig)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case NAME:
+        return get_name();
+
+      case CONFIG:
+        return get_config();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case NAME:
+        return is_set_name();
+      case CONFIG:
+        return is_set_config();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof setLogConfig_args)
+        return this.equals((setLogConfig_args)that);
+      return false;
+    }
+
+    public boolean equals(setLogConfig_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_name = true && this.is_set_name();
+      boolean that_present_name = true && that.is_set_name();
+      if (this_present_name || that_present_name) {
+        if (!(this_present_name && that_present_name))
+          return false;
+        if (!this.name.equals(that.name))
+          return false;
+      }
+
+      boolean this_present_config = true && this.is_set_config();
+      boolean that_present_config = true && that.is_set_config();
+      if (this_present_config || that_present_config) {
+        if (!(this_present_config && that_present_config))
+          return false;
+        if (!this.config.equals(that.config))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_name = true && (is_set_name());
+      list.add(present_name);
+      if (present_name)
+        list.add(name);
+
+      boolean present_config = true && (is_set_config());
+      list.add(present_config);
+      if (present_config)
+        list.add(config);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(setLogConfig_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_name()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_config()).compareTo(other.is_set_config());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_config()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.config, other.config);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("setLogConfig_args(");
+      boolean first = true;
+
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("config:");
+      if (this.config == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.config);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (config != null) {
+        config.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class setLogConfig_argsStandardSchemeFactory implements SchemeFactory {
+      public setLogConfig_argsStandardScheme getScheme() {
+        return new setLogConfig_argsStandardScheme();
+      }
+    }
+
+    private static class setLogConfig_argsStandardScheme extends StandardScheme<setLogConfig_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setLogConfig_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.name = iprot.readString();
+                struct.set_name_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CONFIG
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.config = new LogConfig();
+                struct.config.read(iprot);
+                struct.set_config_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setLogConfig_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.name != null) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+        if (struct.config != null) {
+          oprot.writeFieldBegin(CONFIG_FIELD_DESC);
+          struct.config.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class setLogConfig_argsTupleSchemeFactory implements SchemeFactory {
+      public setLogConfig_argsTupleScheme getScheme() {
+        return new setLogConfig_argsTupleScheme();
+      }
+    }
+
+    private static class setLogConfig_argsTupleScheme extends TupleScheme<setLogConfig_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, setLogConfig_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_name()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_config()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_name()) {
+          oprot.writeString(struct.name);
+        }
+        if (struct.is_set_config()) {
+          struct.config.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, setLogConfig_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.name = iprot.readString();
+          struct.set_name_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.config = new LogConfig();
+          struct.config.read(iprot);
+          struct.set_config_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class setLogConfig_result implements org.apache.thrift.TBase<setLogConfig_result, setLogConfig_result._Fields>, java.io.Serializable, Cloneable, Comparable<setLogConfig_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setLogConfig_result");
+
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new setLogConfig_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setLogConfig_resultTupleSchemeFactory());
+    }
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setLogConfig_result.class, metaDataMap);
+    }
+
+    public setLogConfig_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public setLogConfig_result(setLogConfig_result other) {
+    }
+
+    public setLogConfig_result deepCopy() {
+      return new setLogConfig_result(this);
+    }
+
+    @Override
+    public void clear() {
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof setLogConfig_result)
+        return this.equals((setLogConfig_result)that);
+      return false;
+    }
+
+    public boolean equals(setLogConfig_result that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(setLogConfig_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("setLogConfig_result(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class setLogConfig_resultStandardSchemeFactory implements SchemeFactory {
+      public setLogConfig_resultStandardScheme getScheme() {
+        return new setLogConfig_resultStandardScheme();
+      }
+    }
+
+    private static class setLogConfig_resultStandardScheme extends StandardScheme<setLogConfig_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setLogConfig_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setLogConfig_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class setLogConfig_resultTupleSchemeFactory implements SchemeFactory {
+      public setLogConfig_resultTupleScheme getScheme() {
+        return new setLogConfig_resultTupleScheme();
+      }
+    }
+
+    private static class setLogConfig_resultTupleScheme extends TupleScheme<setLogConfig_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, setLogConfig_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, setLogConfig_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+      }
+    }
+
+  }
+
+  public static class getLogConfig_args implements org.apache.thrift.TBase<getLogConfig_args, getLogConfig_args._Fields>, java.io.Serializable, Cloneable, Comparable<getLogConfig_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLogConfig_args");
+
+    private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getLogConfig_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getLogConfig_argsTupleSchemeFactory());
+    }
+
+    private String name; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      NAME((short)1, "name");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // NAME
+            return NAME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLogConfig_args.class, metaDataMap);
+    }
+
+    public getLogConfig_args() {
+    }
+
+    public getLogConfig_args(
+      String name)
+    {
+      this();
+      this.name = name;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getLogConfig_args(getLogConfig_args other) {
+      if (other.is_set_name()) {
+        this.name = other.name;
+      }
+    }
+
+    public getLogConfig_args deepCopy() {
+      return new getLogConfig_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.name = null;
+    }
+
+    public String get_name() {
+      return this.name;
+    }
+
+    public void set_name(String name) {
+      this.name = name;
+    }
+
+    public void unset_name() {
+      this.name = null;
+    }
+
+    /** Returns true if field name is set (has been assigned a value) and false otherwise */
+    public boolean is_set_name() {
+      return this.name != null;
+    }
+
+    public void set_name_isSet(boolean value) {
+      if (!value) {
+        this.name = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case NAME:
+        if (value == null) {
+          unset_name();
+        } else {
+          set_name((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case NAME:
+        return get_name();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case NAME:
+        return is_set_name();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getLogConfig_args)
+        return this.equals((getLogConfig_args)that);
+      return false;
+    }
+
+    public boolean equals(getLogConfig_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_name = true && this.is_set_name();
+      boolean that_present_name = true && that.is_set_name();
+      if (this_present_name || that_present_name) {
+        if (!(this_present_name && that_present_name))
+          return false;
+        if (!this.name.equals(that.name))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_name = true && (is_set_name());
+      list.add(present_name);
+      if (present_name)
+        list.add(name);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getLogConfig_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_name()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getLogConfig_args(");
+      boolean first = true;
+
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getLogConfig_argsStandardSchemeFactory implements SchemeFactory {
+      public getLogConfig_argsStandardScheme getScheme() {
+        return new getLogConfig_argsStandardScheme();
+      }
+    }
+
+    private static class getLogConfig_argsStandardScheme extends StandardScheme<getLogConfig_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getLogConfig_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // NAME
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.name = iprot.readString();
+                struct.set_name_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getLogConfig_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.name != null) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getLogConfig_argsTupleSchemeFactory implements SchemeFactory {
+      public getLogConfig_argsTupleScheme getScheme() {
+        return new getLogConfig_argsTupleScheme();
+      }
+    }
+
+    private static class getLogConfig_argsTupleScheme extends TupleScheme<getLogConfig_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getLogConfig_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_name()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.is_set_name()) {
+          oprot.writeString(struct.name);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getLogConfig_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.name = iprot.readString();
+          struct.set_name_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class getLogConfig_result implements org.apache.thrift.TBase<getLogConfig_result, getLogConfig_result._Fields>, java.io.Serializable, Cloneable, Comparable<getLogConfig_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getLogConfig_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getLogConfig_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getLogConfig_resultTupleSchemeFactory());
+    }
+
+    private LogConfig success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, LogConfig.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getLogConfig_result.class, metaDataMap);
+    }
+
+    public getLogConfig_result() {
+    }
+
+    public getLogConfig_result(
+      LogConfig success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getLogConfig_result(getLogConfig_result other) {
+      if (other.is_set_success()) {
+        this.success = new LogConfig(other.success);
+      }
+    }
+
+    public getLogConfig_result deepCopy() {
+      return new getLogConfig_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public LogConfig get_success() {
+      return this.success;
+    }
+
+    public void set_success(LogConfig success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((LogConfig)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getLogConfig_result)
+        return this.equals((getLogConfig_result)that);
+      return false;
+    }
+
+    public boolean equals(getLogConfig_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (is_set_success());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getLogConfig_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getLogConfig_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getLogConfig_resultStandardSchemeFactory implements SchemeFactory {
+      public getLogConfig_resultStandardScheme getScheme() {
+        return new getLogConfig_resultStandardScheme();
+      }
+    }
+
+    private static class getLogConfig_resultStandardScheme extends StandardScheme<getLogConfig_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getLogConfig_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new LogConfig();
+                struct.success.read(iprot);
+                struct.set_success_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getLogConfig_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getLogConfig_resultTupleSchemeFactory implements SchemeFactory {
+      public getLogConfig_resultTupleScheme getScheme() {
+        return new getLogConfig_resultTupleScheme();
+      }
+    }
+
+    private static class getLogConfig_resultTupleScheme extends TupleScheme<getLogConfig_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getLogConfig_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_success()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.is_set_success()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getLogConfig_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new LogConfig();
+          struct.success.read(iprot);
+          struct.set_success_isSet(true);
+        }
+      }
+    }
+
+  }
+
   public static class debug_args implements org.apache.thrift.TBase<debug_args, debug_args._Fields>, java.io.Serializable, Cloneable, Comparable<debug_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("debug_args");
 
@@ -10720,10 +13165,10 @@
         return get_component();
 
       case ENABLE:
-        return Boolean.valueOf(is_enable());
+        return is_enable();
 
       case SAMPLING_PERCENTAGE:
-        return Double.valueOf(get_samplingPercentage());
+        return get_samplingPercentage();
 
       }
       throw new IllegalStateException();
@@ -11560,6 +14005,1709 @@
 
   }
 
+  public static class setWorkerProfiler_args implements org.apache.thrift.TBase<setWorkerProfiler_args, setWorkerProfiler_args._Fields>, java.io.Serializable, Cloneable, Comparable<setWorkerProfiler_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setWorkerProfiler_args");
+
+    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField PROFILE_REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("profileRequest", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new setWorkerProfiler_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setWorkerProfiler_argsTupleSchemeFactory());
+    }
+
+    private String id; // required
+    private ProfileRequest profileRequest; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      ID((short)1, "id"),
+      PROFILE_REQUEST((short)2, "profileRequest");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          case 2: // PROFILE_REQUEST
+            return PROFILE_REQUEST;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.PROFILE_REQUEST, new org.apache.thrift.meta_data.FieldMetaData("profileRequest", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ProfileRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setWorkerProfiler_args.class, metaDataMap);
+    }
+
+    public setWorkerProfiler_args() {
+    }
+
+    public setWorkerProfiler_args(
+      String id,
+      ProfileRequest profileRequest)
+    {
+      this();
+      this.id = id;
+      this.profileRequest = profileRequest;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public setWorkerProfiler_args(setWorkerProfiler_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+      if (other.is_set_profileRequest()) {
+        this.profileRequest = new ProfileRequest(other.profileRequest);
+      }
+    }
+
+    public setWorkerProfiler_args deepCopy() {
+      return new setWorkerProfiler_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.id = null;
+      this.profileRequest = null;
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public ProfileRequest get_profileRequest() {
+      return this.profileRequest;
+    }
+
+    public void set_profileRequest(ProfileRequest profileRequest) {
+      this.profileRequest = profileRequest;
+    }
+
+    public void unset_profileRequest() {
+      this.profileRequest = null;
+    }
+
+    /** Returns true if field profileRequest is set (has been assigned a value) and false otherwise */
+    public boolean is_set_profileRequest() {
+      return this.profileRequest != null;
+    }
+
+    public void set_profileRequest_isSet(boolean value) {
+      if (!value) {
+        this.profileRequest = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      case PROFILE_REQUEST:
+        if (value == null) {
+          unset_profileRequest();
+        } else {
+          set_profileRequest((ProfileRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      case PROFILE_REQUEST:
+        return get_profileRequest();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case ID:
+        return is_set_id();
+      case PROFILE_REQUEST:
+        return is_set_profileRequest();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof setWorkerProfiler_args)
+        return this.equals((setWorkerProfiler_args)that);
+      return false;
+    }
+
+    public boolean equals(setWorkerProfiler_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      boolean this_present_profileRequest = true && this.is_set_profileRequest();
+      boolean that_present_profileRequest = true && that.is_set_profileRequest();
+      if (this_present_profileRequest || that_present_profileRequest) {
+        if (!(this_present_profileRequest && that_present_profileRequest))
+          return false;
+        if (!this.profileRequest.equals(that.profileRequest))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_id = true && (is_set_id());
+      list.add(present_id);
+      if (present_id)
+        list.add(id);
+
+      boolean present_profileRequest = true && (is_set_profileRequest());
+      list.add(present_profileRequest);
+      if (present_profileRequest)
+        list.add(profileRequest);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(setWorkerProfiler_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_profileRequest()).compareTo(other.is_set_profileRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_profileRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.profileRequest, other.profileRequest);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("setWorkerProfiler_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("profileRequest:");
+      if (this.profileRequest == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.profileRequest);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (profileRequest != null) {
+        profileRequest.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class setWorkerProfiler_argsStandardSchemeFactory implements SchemeFactory {
+      public setWorkerProfiler_argsStandardScheme getScheme() {
+        return new setWorkerProfiler_argsStandardScheme();
+      }
+    }
+
+    private static class setWorkerProfiler_argsStandardScheme extends StandardScheme<setWorkerProfiler_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setWorkerProfiler_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.id = iprot.readString();
+                struct.set_id_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // PROFILE_REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.profileRequest = new ProfileRequest();
+                struct.profileRequest.read(iprot);
+                struct.set_profileRequest_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setWorkerProfiler_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.id != null) {
+          oprot.writeFieldBegin(ID_FIELD_DESC);
+          oprot.writeString(struct.id);
+          oprot.writeFieldEnd();
+        }
+        if (struct.profileRequest != null) {
+          oprot.writeFieldBegin(PROFILE_REQUEST_FIELD_DESC);
+          struct.profileRequest.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class setWorkerProfiler_argsTupleSchemeFactory implements SchemeFactory {
+      public setWorkerProfiler_argsTupleScheme getScheme() {
+        return new setWorkerProfiler_argsTupleScheme();
+      }
+    }
+
+    private static class setWorkerProfiler_argsTupleScheme extends TupleScheme<setWorkerProfiler_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, setWorkerProfiler_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_id()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_profileRequest()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_id()) {
+          oprot.writeString(struct.id);
+        }
+        if (struct.is_set_profileRequest()) {
+          struct.profileRequest.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, setWorkerProfiler_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.id = iprot.readString();
+          struct.set_id_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.profileRequest = new ProfileRequest();
+          struct.profileRequest.read(iprot);
+          struct.set_profileRequest_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class setWorkerProfiler_result implements org.apache.thrift.TBase<setWorkerProfiler_result, setWorkerProfiler_result._Fields>, java.io.Serializable, Cloneable, Comparable<setWorkerProfiler_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("setWorkerProfiler_result");
+
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new setWorkerProfiler_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new setWorkerProfiler_resultTupleSchemeFactory());
+    }
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(setWorkerProfiler_result.class, metaDataMap);
+    }
+
+    public setWorkerProfiler_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public setWorkerProfiler_result(setWorkerProfiler_result other) {
+    }
+
+    public setWorkerProfiler_result deepCopy() {
+      return new setWorkerProfiler_result(this);
+    }
+
+    @Override
+    public void clear() {
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof setWorkerProfiler_result)
+        return this.equals((setWorkerProfiler_result)that);
+      return false;
+    }
+
+    public boolean equals(setWorkerProfiler_result that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(setWorkerProfiler_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("setWorkerProfiler_result(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class setWorkerProfiler_resultStandardSchemeFactory implements SchemeFactory {
+      public setWorkerProfiler_resultStandardScheme getScheme() {
+        return new setWorkerProfiler_resultStandardScheme();
+      }
+    }
+
+    private static class setWorkerProfiler_resultStandardScheme extends StandardScheme<setWorkerProfiler_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, setWorkerProfiler_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, setWorkerProfiler_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class setWorkerProfiler_resultTupleSchemeFactory implements SchemeFactory {
+      public setWorkerProfiler_resultTupleScheme getScheme() {
+        return new setWorkerProfiler_resultTupleScheme();
+      }
+    }
+
+    private static class setWorkerProfiler_resultTupleScheme extends TupleScheme<setWorkerProfiler_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, setWorkerProfiler_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, setWorkerProfiler_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+      }
+    }
+
+  }
+
+  public static class getComponentPendingProfileActions_args implements org.apache.thrift.TBase<getComponentPendingProfileActions_args, getComponentPendingProfileActions_args._Fields>, java.io.Serializable, Cloneable, Comparable<getComponentPendingProfileActions_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getComponentPendingProfileActions_args");
+
+    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField ACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("action", org.apache.thrift.protocol.TType.I32, (short)3);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getComponentPendingProfileActions_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getComponentPendingProfileActions_argsTupleSchemeFactory());
+    }
+
+    private String id; // required
+    private String component_id; // required
+    private ProfileAction action; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      ID((short)1, "id"),
+      COMPONENT_ID((short)2, "component_id"),
+      /**
+       * 
+       * @see ProfileAction
+       */
+      ACTION((short)3, "action");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          case 2: // COMPONENT_ID
+            return COMPONENT_ID;
+          case 3: // ACTION
+            return ACTION;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.ACTION, new org.apache.thrift.meta_data.FieldMetaData("action", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ProfileAction.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getComponentPendingProfileActions_args.class, metaDataMap);
+    }
+
+    public getComponentPendingProfileActions_args() {
+    }
+
+    public getComponentPendingProfileActions_args(
+      String id,
+      String component_id,
+      ProfileAction action)
+    {
+      this();
+      this.id = id;
+      this.component_id = component_id;
+      this.action = action;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getComponentPendingProfileActions_args(getComponentPendingProfileActions_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+      if (other.is_set_component_id()) {
+        this.component_id = other.component_id;
+      }
+      if (other.is_set_action()) {
+        this.action = other.action;
+      }
+    }
+
+    public getComponentPendingProfileActions_args deepCopy() {
+      return new getComponentPendingProfileActions_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.id = null;
+      this.component_id = null;
+      this.action = null;
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public String get_component_id() {
+      return this.component_id;
+    }
+
+    public void set_component_id(String component_id) {
+      this.component_id = component_id;
+    }
+
+    public void unset_component_id() {
+      this.component_id = null;
+    }
+
+    /** Returns true if field component_id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_component_id() {
+      return this.component_id != null;
+    }
+
+    public void set_component_id_isSet(boolean value) {
+      if (!value) {
+        this.component_id = null;
+      }
+    }
+
+    /**
+     * 
+     * @see ProfileAction
+     */
+    public ProfileAction get_action() {
+      return this.action;
+    }
+
+    /**
+     * 
+     * @see ProfileAction
+     */
+    public void set_action(ProfileAction action) {
+      this.action = action;
+    }
+
+    public void unset_action() {
+      this.action = null;
+    }
+
+    /** Returns true if field action is set (has been assigned a value) and false otherwise */
+    public boolean is_set_action() {
+      return this.action != null;
+    }
+
+    public void set_action_isSet(boolean value) {
+      if (!value) {
+        this.action = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      case COMPONENT_ID:
+        if (value == null) {
+          unset_component_id();
+        } else {
+          set_component_id((String)value);
+        }
+        break;
+
+      case ACTION:
+        if (value == null) {
+          unset_action();
+        } else {
+          set_action((ProfileAction)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      case COMPONENT_ID:
+        return get_component_id();
+
+      case ACTION:
+        return get_action();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case ID:
+        return is_set_id();
+      case COMPONENT_ID:
+        return is_set_component_id();
+      case ACTION:
+        return is_set_action();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getComponentPendingProfileActions_args)
+        return this.equals((getComponentPendingProfileActions_args)that);
+      return false;
+    }
+
+    public boolean equals(getComponentPendingProfileActions_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      boolean this_present_component_id = true && this.is_set_component_id();
+      boolean that_present_component_id = true && that.is_set_component_id();
+      if (this_present_component_id || that_present_component_id) {
+        if (!(this_present_component_id && that_present_component_id))
+          return false;
+        if (!this.component_id.equals(that.component_id))
+          return false;
+      }
+
+      boolean this_present_action = true && this.is_set_action();
+      boolean that_present_action = true && that.is_set_action();
+      if (this_present_action || that_present_action) {
+        if (!(this_present_action && that_present_action))
+          return false;
+        if (!this.action.equals(that.action))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_id = true && (is_set_id());
+      list.add(present_id);
+      if (present_id)
+        list.add(id);
+
+      boolean present_component_id = true && (is_set_component_id());
+      list.add(present_component_id);
+      if (present_component_id)
+        list.add(component_id);
+
+      boolean present_action = true && (is_set_action());
+      list.add(present_action);
+      if (present_action)
+        list.add(action.getValue());
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getComponentPendingProfileActions_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(other.is_set_component_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_component_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, other.component_id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_action()).compareTo(other.is_set_action());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_action()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.action, other.action);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getComponentPendingProfileActions_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("component_id:");
+      if (this.component_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("action:");
+      if (this.action == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.action);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getComponentPendingProfileActions_argsStandardSchemeFactory implements SchemeFactory {
+      public getComponentPendingProfileActions_argsStandardScheme getScheme() {
+        return new getComponentPendingProfileActions_argsStandardScheme();
+      }
+    }
+
+    private static class getComponentPendingProfileActions_argsStandardScheme extends StandardScheme<getComponentPendingProfileActions_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getComponentPendingProfileActions_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.id = iprot.readString();
+                struct.set_id_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // COMPONENT_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.component_id = iprot.readString();
+                struct.set_component_id_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // ACTION
+              if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+                struct.action = backtype.storm.generated.ProfileAction.findByValue(iprot.readI32());
+                struct.set_action_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getComponentPendingProfileActions_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.id != null) {
+          oprot.writeFieldBegin(ID_FIELD_DESC);
+          oprot.writeString(struct.id);
+          oprot.writeFieldEnd();
+        }
+        if (struct.component_id != null) {
+          oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+          oprot.writeString(struct.component_id);
+          oprot.writeFieldEnd();
+        }
+        if (struct.action != null) {
+          oprot.writeFieldBegin(ACTION_FIELD_DESC);
+          oprot.writeI32(struct.action.getValue());
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getComponentPendingProfileActions_argsTupleSchemeFactory implements SchemeFactory {
+      public getComponentPendingProfileActions_argsTupleScheme getScheme() {
+        return new getComponentPendingProfileActions_argsTupleScheme();
+      }
+    }
+
+    private static class getComponentPendingProfileActions_argsTupleScheme extends TupleScheme<getComponentPendingProfileActions_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPendingProfileActions_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_id()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_component_id()) {
+          optionals.set(1);
+        }
+        if (struct.is_set_action()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.is_set_id()) {
+          oprot.writeString(struct.id);
+        }
+        if (struct.is_set_component_id()) {
+          oprot.writeString(struct.component_id);
+        }
+        if (struct.is_set_action()) {
+          oprot.writeI32(struct.action.getValue());
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getComponentPendingProfileActions_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.id = iprot.readString();
+          struct.set_id_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.component_id = iprot.readString();
+          struct.set_component_id_isSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.action = backtype.storm.generated.ProfileAction.findByValue(iprot.readI32());
+          struct.set_action_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class getComponentPendingProfileActions_result implements org.apache.thrift.TBase<getComponentPendingProfileActions_result, getComponentPendingProfileActions_result._Fields>, java.io.Serializable, Cloneable, Comparable<getComponentPendingProfileActions_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getComponentPendingProfileActions_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getComponentPendingProfileActions_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getComponentPendingProfileActions_resultTupleSchemeFactory());
+    }
+
+    private List<ProfileRequest> success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ProfileRequest.class))));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getComponentPendingProfileActions_result.class, metaDataMap);
+    }
+
+    public getComponentPendingProfileActions_result() {
+    }
+
+    public getComponentPendingProfileActions_result(
+      List<ProfileRequest> success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getComponentPendingProfileActions_result(getComponentPendingProfileActions_result other) {
+      if (other.is_set_success()) {
+        List<ProfileRequest> __this__success = new ArrayList<ProfileRequest>(other.success.size());
+        for (ProfileRequest other_element : other.success) {
+          __this__success.add(new ProfileRequest(other_element));
+        }
+        this.success = __this__success;
+      }
+    }
+
+    public getComponentPendingProfileActions_result deepCopy() {
+      return new getComponentPendingProfileActions_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public int get_success_size() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<ProfileRequest> get_success_iterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void add_to_success(ProfileRequest elem) {
+      if (this.success == null) {
+        this.success = new ArrayList<ProfileRequest>();
+      }
+      this.success.add(elem);
+    }
+
+    public List<ProfileRequest> get_success() {
+      return this.success;
+    }
+
+    public void set_success(List<ProfileRequest> success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((List<ProfileRequest>)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getComponentPendingProfileActions_result)
+        return this.equals((getComponentPendingProfileActions_result)that);
+      return false;
+    }
+
+    public boolean equals(getComponentPendingProfileActions_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (is_set_success());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getComponentPendingProfileActions_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getComponentPendingProfileActions_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getComponentPendingProfileActions_resultStandardSchemeFactory implements SchemeFactory {
+      public getComponentPendingProfileActions_resultStandardScheme getScheme() {
+        return new getComponentPendingProfileActions_resultStandardScheme();
+      }
+    }
+
+    private static class getComponentPendingProfileActions_resultStandardScheme extends StandardScheme<getComponentPendingProfileActions_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getComponentPendingProfileActions_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list722 = iprot.readListBegin();
+                  struct.success = new ArrayList<ProfileRequest>(_list722.size);
+                  ProfileRequest _elem723;
+                  for (int _i724 = 0; _i724 < _list722.size; ++_i724)
+                  {
+                    _elem723 = new ProfileRequest();
+                    _elem723.read(iprot);
+                    struct.success.add(_elem723);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.set_success_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getComponentPendingProfileActions_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
+            for (ProfileRequest _iter725 : struct.success)
+            {
+              _iter725.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getComponentPendingProfileActions_resultTupleSchemeFactory implements SchemeFactory {
+      public getComponentPendingProfileActions_resultTupleScheme getScheme() {
+        return new getComponentPendingProfileActions_resultTupleScheme();
+      }
+    }
+
+    private static class getComponentPendingProfileActions_resultTupleScheme extends TupleScheme<getComponentPendingProfileActions_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPendingProfileActions_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_success()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.is_set_success()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (ProfileRequest _iter726 : struct.success)
+            {
+              _iter726.write(oprot);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getComponentPendingProfileActions_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          {
+            org.apache.thrift.protocol.TList _list727 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<ProfileRequest>(_list727.size);
+            ProfileRequest _elem728;
+            for (int _i729 = 0; _i729 < _list727.size; ++_i729)
+            {
+              _elem728 = new ProfileRequest();
+              _elem728.read(iprot);
+              struct.success.add(_elem728);
+            }
+          }
+          struct.set_success_isSet(true);
+        }
+      }
+    }
+
+  }
+
   public static class uploadNewCredentials_args implements org.apache.thrift.TBase<uploadNewCredentials_args, uploadNewCredentials_args._Fields>, java.io.Serializable, Cloneable, Comparable<uploadNewCredentials_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadNewCredentials_args");
 
@@ -19911,6 +24059,2388 @@
 
   }
 
+  public static class getTopologyPageInfo_args implements org.apache.thrift.TBase<getTopologyPageInfo_args, getTopologyPageInfo_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTopologyPageInfo_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyPageInfo_args");
+
+    private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField WINDOW_FIELD_DESC = new org.apache.thrift.protocol.TField("window", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField IS_INCLUDE_SYS_FIELD_DESC = new org.apache.thrift.protocol.TField("is_include_sys", org.apache.thrift.protocol.TType.BOOL, (short)3);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getTopologyPageInfo_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getTopologyPageInfo_argsTupleSchemeFactory());
+    }
+
+    private String id; // required
+    private String window; // required
+    private boolean is_include_sys; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      ID((short)1, "id"),
+      WINDOW((short)2, "window"),
+      IS_INCLUDE_SYS((short)3, "is_include_sys");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          case 2: // WINDOW
+            return WINDOW;
+          case 3: // IS_INCLUDE_SYS
+            return IS_INCLUDE_SYS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __IS_INCLUDE_SYS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.WINDOW, new org.apache.thrift.meta_data.FieldMetaData("window", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.IS_INCLUDE_SYS, new org.apache.thrift.meta_data.FieldMetaData("is_include_sys", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyPageInfo_args.class, metaDataMap);
+    }
+
+    public getTopologyPageInfo_args() {
+    }
+
+    public getTopologyPageInfo_args(
+      String id,
+      String window,
+      boolean is_include_sys)
+    {
+      this();
+      this.id = id;
+      this.window = window;
+      this.is_include_sys = is_include_sys;
+      set_is_include_sys_isSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyPageInfo_args(getTopologyPageInfo_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+      if (other.is_set_window()) {
+        this.window = other.window;
+      }
+      this.is_include_sys = other.is_include_sys;
+    }
+
+    public getTopologyPageInfo_args deepCopy() {
+      return new getTopologyPageInfo_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.id = null;
+      this.window = null;
+      set_is_include_sys_isSet(false);
+      this.is_include_sys = false;
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public String get_window() {
+      return this.window;
+    }
+
+    public void set_window(String window) {
+      this.window = window;
+    }
+
+    public void unset_window() {
+      this.window = null;
+    }
+
+    /** Returns true if field window is set (has been assigned a value) and false otherwise */
+    public boolean is_set_window() {
+      return this.window != null;
+    }
+
+    public void set_window_isSet(boolean value) {
+      if (!value) {
+        this.window = null;
+      }
+    }
+
+    public boolean is_is_include_sys() {
+      return this.is_include_sys;
+    }
+
+    public void set_is_include_sys(boolean is_include_sys) {
+      this.is_include_sys = is_include_sys;
+      set_is_include_sys_isSet(true);
+    }
+
+    public void unset_is_include_sys() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID);
+    }
+
+    /** Returns true if field is_include_sys is set (has been assigned a value) and false otherwise */
+    public boolean is_set_is_include_sys() {
+      return EncodingUtils.testBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID);
+    }
+
+    public void set_is_include_sys_isSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      case WINDOW:
+        if (value == null) {
+          unset_window();
+        } else {
+          set_window((String)value);
+        }
+        break;
+
+      case IS_INCLUDE_SYS:
+        if (value == null) {
+          unset_is_include_sys();
+        } else {
+          set_is_include_sys((Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      case WINDOW:
+        return get_window();
+
+      case IS_INCLUDE_SYS:
+        return is_is_include_sys();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case ID:
+        return is_set_id();
+      case WINDOW:
+        return is_set_window();
+      case IS_INCLUDE_SYS:
+        return is_set_is_include_sys();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyPageInfo_args)
+        return this.equals((getTopologyPageInfo_args)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyPageInfo_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      boolean this_present_window = true && this.is_set_window();
+      boolean that_present_window = true && that.is_set_window();
+      if (this_present_window || that_present_window) {
+        if (!(this_present_window && that_present_window))
+          return false;
+        if (!this.window.equals(that.window))
+          return false;
+      }
+
+      boolean this_present_is_include_sys = true;
+      boolean that_present_is_include_sys = true;
+      if (this_present_is_include_sys || that_present_is_include_sys) {
+        if (!(this_present_is_include_sys && that_present_is_include_sys))
+          return false;
+        if (this.is_include_sys != that.is_include_sys)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_id = true && (is_set_id());
+      list.add(present_id);
+      if (present_id)
+        list.add(id);
+
+      boolean present_window = true && (is_set_window());
+      list.add(present_window);
+      if (present_window)
+        list.add(window);
+
+      boolean present_is_include_sys = true;
+      list.add(present_is_include_sys);
+      if (present_is_include_sys)
+        list.add(is_include_sys);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getTopologyPageInfo_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_window()).compareTo(other.is_set_window());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_window()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window, other.window);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_is_include_sys()).compareTo(other.is_set_is_include_sys());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_is_include_sys()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.is_include_sys, other.is_include_sys);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyPageInfo_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("window:");
+      if (this.window == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("is_include_sys:");
+      sb.append(this.is_include_sys);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getTopologyPageInfo_argsStandardSchemeFactory implements SchemeFactory {
+      public getTopologyPageInfo_argsStandardScheme getScheme() {
+        return new getTopologyPageInfo_argsStandardScheme();
+      }
+    }
+
+    private static class getTopologyPageInfo_argsStandardScheme extends StandardScheme<getTopologyPageInfo_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.id = iprot.readString();
+                struct.set_id_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // WINDOW
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.window = iprot.readString();
+                struct.set_window_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // IS_INCLUDE_SYS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.is_include_sys = iprot.readBool();
+                struct.set_is_include_sys_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.id != null) {
+          oprot.writeFieldBegin(ID_FIELD_DESC);
+          oprot.writeString(struct.id);
+          oprot.writeFieldEnd();
+        }
+        if (struct.window != null) {
+          oprot.writeFieldBegin(WINDOW_FIELD_DESC);
+          oprot.writeString(struct.window);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(IS_INCLUDE_SYS_FIELD_DESC);
+        oprot.writeBool(struct.is_include_sys);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTopologyPageInfo_argsTupleSchemeFactory implements SchemeFactory {
+      public getTopologyPageInfo_argsTupleScheme getScheme() {
+        return new getTopologyPageInfo_argsTupleScheme();
+      }
+    }
+
+    private static class getTopologyPageInfo_argsTupleScheme extends TupleScheme<getTopologyPageInfo_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_id()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_window()) {
+          optionals.set(1);
+        }
+        if (struct.is_set_is_include_sys()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.is_set_id()) {
+          oprot.writeString(struct.id);
+        }
+        if (struct.is_set_window()) {
+          oprot.writeString(struct.window);
+        }
+        if (struct.is_set_is_include_sys()) {
+          oprot.writeBool(struct.is_include_sys);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.id = iprot.readString();
+          struct.set_id_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.window = iprot.readString();
+          struct.set_window_isSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.is_include_sys = iprot.readBool();
+          struct.set_is_include_sys_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class getTopologyPageInfo_result implements org.apache.thrift.TBase<getTopologyPageInfo_result, getTopologyPageInfo_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTopologyPageInfo_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyPageInfo_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getTopologyPageInfo_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getTopologyPageInfo_resultTupleSchemeFactory());
+    }
+
+    private TopologyPageInfo success; // required
+    private NotAliveException e; // required
+    private AuthorizationException aze; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      E((short)1, "e"),
+      AZE((short)2, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // E
+            return E;
+          case 2: // AZE
+            return AZE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyPageInfo.class)));
+      tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyPageInfo_result.class, metaDataMap);
+    }
+
+    public getTopologyPageInfo_result() {
+    }
+
+    public getTopologyPageInfo_result(
+      TopologyPageInfo success,
+      NotAliveException e,
+      AuthorizationException aze)
+    {
+      this();
+      this.success = success;
+      this.e = e;
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyPageInfo_result(getTopologyPageInfo_result other) {
+      if (other.is_set_success()) {
+        this.success = new TopologyPageInfo(other.success);
+      }
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public getTopologyPageInfo_result deepCopy() {
+      return new getTopologyPageInfo_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.e = null;
+      this.aze = null;
+    }
+
+    public TopologyPageInfo get_success() {
+      return this.success;
+    }
+
+    public void set_success(TopologyPageInfo success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public NotAliveException get_e() {
+      return this.e;
+    }
+
+    public void set_e(NotAliveException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been assigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public AuthorizationException get_aze() {
+      return this.aze;
+    }
+
+    public void set_aze(AuthorizationException aze) {
+      this.aze = aze;
+    }
+
+    public void unset_aze() {
+      this.aze = null;
+    }
+
+    /** Returns true if field aze is set (has been assigned a value) and false otherwise */
+    public boolean is_set_aze() {
+      return this.aze != null;
+    }
+
+    public void set_aze_isSet(boolean value) {
+      if (!value) {
+        this.aze = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((TopologyPageInfo)value);
+        }
+        break;
+
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((NotAliveException)value);
+        }
+        break;
+
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case E:
+        return get_e();
+
+      case AZE:
+        return get_aze();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case E:
+        return is_set_e();
+      case AZE:
+        return is_set_aze();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyPageInfo_result)
+        return this.equals((getTopologyPageInfo_result)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyPageInfo_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      boolean this_present_aze = true && this.is_set_aze();
+      boolean that_present_aze = true && that.is_set_aze();
+      if (this_present_aze || that_present_aze) {
+        if (!(this_present_aze && that_present_aze))
+          return false;
+        if (!this.aze.equals(that.aze))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (is_set_success());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_e = true && (is_set_e());
+      list.add(present_e);
+      if (present_e)
+        list.add(e);
+
+      boolean present_aze = true && (is_set_aze());
+      list.add(present_aze);
+      if (present_aze)
+        list.add(aze);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getTopologyPageInfo_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyPageInfo_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("aze:");
+      if (this.aze == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.aze);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getTopologyPageInfo_resultStandardSchemeFactory implements SchemeFactory {
+      public getTopologyPageInfo_resultStandardScheme getScheme() {
+        return new getTopologyPageInfo_resultStandardScheme();
+      }
+    }
+
+    private static class getTopologyPageInfo_resultStandardScheme extends StandardScheme<getTopologyPageInfo_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new TopologyPageInfo();
+                struct.success.read(iprot);
+                struct.set_success_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // E
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.e = new NotAliveException();
+                struct.e.read(iprot);
+                struct.set_e_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // AZE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.aze = new AuthorizationException();
+                struct.aze.read(iprot);
+                struct.set_aze_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.e != null) {
+          oprot.writeFieldBegin(E_FIELD_DESC);
+          struct.e.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.aze != null) {
+          oprot.writeFieldBegin(AZE_FIELD_DESC);
+          struct.aze.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTopologyPageInfo_resultTupleSchemeFactory implements SchemeFactory {
+      public getTopologyPageInfo_resultTupleScheme getScheme() {
+        return new getTopologyPageInfo_resultTupleScheme();
+      }
+    }
+
+    private static class getTopologyPageInfo_resultTupleScheme extends TupleScheme<getTopologyPageInfo_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_success()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_e()) {
+          optionals.set(1);
+        }
+        if (struct.is_set_aze()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.is_set_success()) {
+          struct.success.write(oprot);
+        }
+        if (struct.is_set_e()) {
+          struct.e.write(oprot);
+        }
+        if (struct.is_set_aze()) {
+          struct.aze.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyPageInfo_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.success = new TopologyPageInfo();
+          struct.success.read(iprot);
+          struct.set_success_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.e = new NotAliveException();
+          struct.e.read(iprot);
+          struct.set_e_isSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.aze = new AuthorizationException();
+          struct.aze.read(iprot);
+          struct.set_aze_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class getComponentPageInfo_args implements org.apache.thrift.TBase<getComponentPageInfo_args, getComponentPageInfo_args._Fields>, java.io.Serializable, Cloneable, Comparable<getComponentPageInfo_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getComponentPageInfo_args");
+
+    private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("component_id", org.apache.thrift.protocol.TType.STRING, (short)2);
+    private static final org.apache.thrift.protocol.TField WINDOW_FIELD_DESC = new org.apache.thrift.protocol.TField("window", org.apache.thrift.protocol.TType.STRING, (short)3);
+    private static final org.apache.thrift.protocol.TField IS_INCLUDE_SYS_FIELD_DESC = new org.apache.thrift.protocol.TField("is_include_sys", org.apache.thrift.protocol.TType.BOOL, (short)4);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getComponentPageInfo_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getComponentPageInfo_argsTupleSchemeFactory());
+    }
+
+    private String topology_id; // required
+    private String component_id; // required
+    private String window; // required
+    private boolean is_include_sys; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TOPOLOGY_ID((short)1, "topology_id"),
+      COMPONENT_ID((short)2, "component_id"),
+      WINDOW((short)3, "window"),
+      IS_INCLUDE_SYS((short)4, "is_include_sys");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TOPOLOGY_ID
+            return TOPOLOGY_ID;
+          case 2: // COMPONENT_ID
+            return COMPONENT_ID;
+          case 3: // WINDOW
+            return WINDOW;
+          case 4: // IS_INCLUDE_SYS
+            return IS_INCLUDE_SYS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    private static final int __IS_INCLUDE_SYS_ISSET_ID = 0;
+    private byte __isset_bitfield = 0;
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("component_id", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.WINDOW, new org.apache.thrift.meta_data.FieldMetaData("window", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      tmpMap.put(_Fields.IS_INCLUDE_SYS, new org.apache.thrift.meta_data.FieldMetaData("is_include_sys", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getComponentPageInfo_args.class, metaDataMap);
+    }
+
+    public getComponentPageInfo_args() {
+    }
+
+    public getComponentPageInfo_args(
+      String topology_id,
+      String component_id,
+      String window,
+      boolean is_include_sys)
+    {
+      this();
+      this.topology_id = topology_id;
+      this.component_id = component_id;
+      this.window = window;
+      this.is_include_sys = is_include_sys;
+      set_is_include_sys_isSet(true);
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getComponentPageInfo_args(getComponentPageInfo_args other) {
+      __isset_bitfield = other.__isset_bitfield;
+      if (other.is_set_topology_id()) {
+        this.topology_id = other.topology_id;
+      }
+      if (other.is_set_component_id()) {
+        this.component_id = other.component_id;
+      }
+      if (other.is_set_window()) {
+        this.window = other.window;
+      }
+      this.is_include_sys = other.is_include_sys;
+    }
+
+    public getComponentPageInfo_args deepCopy() {
+      return new getComponentPageInfo_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.topology_id = null;
+      this.component_id = null;
+      this.window = null;
+      set_is_include_sys_isSet(false);
+      this.is_include_sys = false;
+    }
+
+    public String get_topology_id() {
+      return this.topology_id;
+    }
+
+    public void set_topology_id(String topology_id) {
+      this.topology_id = topology_id;
+    }
+
+    public void unset_topology_id() {
+      this.topology_id = null;
+    }
+
+    /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_topology_id() {
+      return this.topology_id != null;
+    }
+
+    public void set_topology_id_isSet(boolean value) {
+      if (!value) {
+        this.topology_id = null;
+      }
+    }
+
+    public String get_component_id() {
+      return this.component_id;
+    }
+
+    public void set_component_id(String component_id) {
+      this.component_id = component_id;
+    }
+
+    public void unset_component_id() {
+      this.component_id = null;
+    }
+
+    /** Returns true if field component_id is set (has been assigned a value) and false otherwise */
+    public boolean is_set_component_id() {
+      return this.component_id != null;
+    }
+
+    public void set_component_id_isSet(boolean value) {
+      if (!value) {
+        this.component_id = null;
+      }
+    }
+
+    public String get_window() {
+      return this.window;
+    }
+
+    public void set_window(String window) {
+      this.window = window;
+    }
+
+    public void unset_window() {
+      this.window = null;
+    }
+
+    /** Returns true if field window is set (has been assigned a value) and false otherwise */
+    public boolean is_set_window() {
+      return this.window != null;
+    }
+
+    public void set_window_isSet(boolean value) {
+      if (!value) {
+        this.window = null;
+      }
+    }
+
+    public boolean is_is_include_sys() {
+      return this.is_include_sys;
+    }
+
+    public void set_is_include_sys(boolean is_include_sys) {
+      this.is_include_sys = is_include_sys;
+      set_is_include_sys_isSet(true);
+    }
+
+    public void unset_is_include_sys() {
+      __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID);
+    }
+
+    /** Returns true if field is_include_sys is set (has been assigned a value) and false otherwise */
+    public boolean is_set_is_include_sys() {
+      return EncodingUtils.testBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID);
+    }
+
+    public void set_is_include_sys_isSet(boolean value) {
+      __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __IS_INCLUDE_SYS_ISSET_ID, value);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TOPOLOGY_ID:
+        if (value == null) {
+          unset_topology_id();
+        } else {
+          set_topology_id((String)value);
+        }
+        break;
+
+      case COMPONENT_ID:
+        if (value == null) {
+          unset_component_id();
+        } else {
+          set_component_id((String)value);
+        }
+        break;
+
+      case WINDOW:
+        if (value == null) {
+          unset_window();
+        } else {
+          set_window((String)value);
+        }
+        break;
+
+      case IS_INCLUDE_SYS:
+        if (value == null) {
+          unset_is_include_sys();
+        } else {
+          set_is_include_sys((Boolean)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TOPOLOGY_ID:
+        return get_topology_id();
+
+      case COMPONENT_ID:
+        return get_component_id();
+
+      case WINDOW:
+        return get_window();
+
+      case IS_INCLUDE_SYS:
+        return is_is_include_sys();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TOPOLOGY_ID:
+        return is_set_topology_id();
+      case COMPONENT_ID:
+        return is_set_component_id();
+      case WINDOW:
+        return is_set_window();
+      case IS_INCLUDE_SYS:
+        return is_set_is_include_sys();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getComponentPageInfo_args)
+        return this.equals((getComponentPageInfo_args)that);
+      return false;
+    }
+
+    public boolean equals(getComponentPageInfo_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_topology_id = true && this.is_set_topology_id();
+      boolean that_present_topology_id = true && that.is_set_topology_id();
+      if (this_present_topology_id || that_present_topology_id) {
+        if (!(this_present_topology_id && that_present_topology_id))
+          return false;
+        if (!this.topology_id.equals(that.topology_id))
+          return false;
+      }
+
+      boolean this_present_component_id = true && this.is_set_component_id();
+      boolean that_present_component_id = true && that.is_set_component_id();
+      if (this_present_component_id || that_present_component_id) {
+        if (!(this_present_component_id && that_present_component_id))
+          return false;
+        if (!this.component_id.equals(that.component_id))
+          return false;
+      }
+
+      boolean this_present_window = true && this.is_set_window();
+      boolean that_present_window = true && that.is_set_window();
+      if (this_present_window || that_present_window) {
+        if (!(this_present_window && that_present_window))
+          return false;
+        if (!this.window.equals(that.window))
+          return false;
+      }
+
+      boolean this_present_is_include_sys = true;
+      boolean that_present_is_include_sys = true;
+      if (this_present_is_include_sys || that_present_is_include_sys) {
+        if (!(this_present_is_include_sys && that_present_is_include_sys))
+          return false;
+        if (this.is_include_sys != that.is_include_sys)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_topology_id = true && (is_set_topology_id());
+      list.add(present_topology_id);
+      if (present_topology_id)
+        list.add(topology_id);
+
+      boolean present_component_id = true && (is_set_component_id());
+      list.add(present_component_id);
+      if (present_component_id)
+        list.add(component_id);
+
+      boolean present_window = true && (is_set_window());
+      list.add(present_window);
+      if (present_window)
+        list.add(window);
+
+      boolean present_is_include_sys = true;
+      list.add(present_is_include_sys);
+      if (present_is_include_sys)
+        list.add(is_include_sys);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getComponentPageInfo_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_topology_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(other.is_set_component_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_component_id()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component_id, other.component_id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_window()).compareTo(other.is_set_window());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_window()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window, other.window);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_is_include_sys()).compareTo(other.is_set_is_include_sys());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_is_include_sys()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.is_include_sys, other.is_include_sys);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getComponentPageInfo_args(");
+      boolean first = true;
+
+      sb.append("topology_id:");
+      if (this.topology_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("component_id:");
+      if (this.component_id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.component_id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("window:");
+      if (this.window == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("is_include_sys:");
+      sb.append(this.is_include_sys);
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+        __isset_bitfield = 0;
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getComponentPageInfo_argsStandardSchemeFactory implements SchemeFactory {
+      public getComponentPageInfo_argsStandardScheme getScheme() {
+        return new getComponentPageInfo_argsStandardScheme();
+      }
+    }
+
+    private static class getComponentPageInfo_argsStandardScheme extends StandardScheme<getComponentPageInfo_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getComponentPageInfo_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TOPOLOGY_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.topology_id = iprot.readString();
+                struct.set_topology_id_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // COMPONENT_ID
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.component_id = iprot.readString();
+                struct.set_component_id_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // WINDOW
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.window = iprot.readString();
+                struct.set_window_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 4: // IS_INCLUDE_SYS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.is_include_sys = iprot.readBool();
+                struct.set_is_include_sys_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getComponentPageInfo_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.topology_id != null) {
+          oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC);
+          oprot.writeString(struct.topology_id);
+          oprot.writeFieldEnd();
+        }
+        if (struct.component_id != null) {
+          oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+          oprot.writeString(struct.component_id);
+          oprot.writeFieldEnd();
+        }
+        if (struct.window != null) {
+          oprot.writeFieldBegin(WINDOW_FIELD_DESC);
+          oprot.writeString(struct.window);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldBegin(IS_INCLUDE_SYS_FIELD_DESC);
+        oprot.writeBool(struct.is_include_sys);
+        oprot.writeFieldEnd();
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getComponentPageInfo_argsTupleSchemeFactory implements SchemeFactory {
+      public getComponentPageInfo_argsTupleScheme getScheme() {
+        return new getComponentPageInfo_argsTupleScheme();
+      }
+    }
+
+    private static class getComponentPageInfo_argsTupleScheme extends TupleScheme<getComponentPageInfo_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_topology_id()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_component_id()) {
+          optionals.set(1);
+        }
+        if (struct.is_set_window()) {
+          optionals.set(2);
+        }
+        if (struct.is_set_is_include_sys()) {
+          optionals.set(3);
+        }
+        oprot.writeBitSet(optionals, 4);
+        if (struct.is_set_topology_id()) {
+          oprot.writeString(struct.topology_id);
+        }
+        if (struct.is_set_component_id()) {
+          oprot.writeString(struct.component_id);
+        }
+        if (struct.is_set_window()) {
+          oprot.writeString(struct.window);
+        }
+        if (struct.is_set_is_include_sys()) {
+          oprot.writeBool(struct.is_include_sys);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(4);
+        if (incoming.get(0)) {
+          struct.topology_id = iprot.readString();
+          struct.set_topology_id_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.component_id = iprot.readString();
+          struct.set_component_id_isSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.window = iprot.readString();
+          struct.set_window_isSet(true);
+        }
+        if (incoming.get(3)) {
+          struct.is_include_sys = iprot.readBool();
+          struct.set_is_include_sys_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class getComponentPageInfo_result implements org.apache.thrift.TBase<getComponentPageInfo_result, getComponentPageInfo_result._Fields>, java.io.Serializable, Cloneable, Comparable<getComponentPageInfo_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getComponentPageInfo_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getComponentPageInfo_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getComponentPageInfo_resultTupleSchemeFactory());
+    }
+
+    private ComponentPageInfo success; // required
+    private NotAliveException e; // required
+    private AuthorizationException aze; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      E((short)1, "e"),
+      AZE((short)2, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // E
+            return E;
+          case 2: // AZE
+            return AZE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentPageInfo.class)));
+      tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getComponentPageInfo_result.class, metaDataMap);
+    }
+
+    public getComponentPageInfo_result() {
+    }
+
+    public getComponentPageInfo_result(
+      ComponentPageInfo success,
+      NotAliveException e,
+      AuthorizationException aze)
+    {
+      this();
+      this.success = success;
+      this.e = e;
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getComponentPageInfo_result(getComponentPageInfo_result other) {
+      if (other.is_set_success()) {
+        this.success = new ComponentPageInfo(other.success);
+      }
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public getComponentPageInfo_result deepCopy() {
+      return new getComponentPageInfo_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.e = null;
+      this.aze = null;
+    }
+
+    public ComponentPageInfo get_success() {
+      return this.success;
+    }
+
+    public void set_success(ComponentPageInfo success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public NotAliveException get_e() {
+      return this.e;
+    }
+
+    public void set_e(NotAliveException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been assigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public AuthorizationException get_aze() {
+      return this.aze;
+    }
+
+    public void set_aze(AuthorizationException aze) {
+      this.aze = aze;
+    }
+
+    public void unset_aze() {
+      this.aze = null;
+    }
+
+    /** Returns true if field aze is set (has been assigned a value) and false otherwise */
+    public boolean is_set_aze() {
+      return this.aze != null;
+    }
+
+    public void set_aze_isSet(boolean value) {
+      if (!value) {
+        this.aze = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((ComponentPageInfo)value);
+        }
+        break;
+
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((NotAliveException)value);
+        }
+        break;
+
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case E:
+        return get_e();
+
+      case AZE:
+        return get_aze();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case E:
+        return is_set_e();
+      case AZE:
+        return is_set_aze();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getComponentPageInfo_result)
+        return this.equals((getComponentPageInfo_result)that);
+      return false;
+    }
+
+    public boolean equals(getComponentPageInfo_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      boolean this_present_aze = true && this.is_set_aze();
+      boolean that_present_aze = true && that.is_set_aze();
+      if (this_present_aze || that_present_aze) {
+        if (!(this_present_aze && that_present_aze))
+          return false;
+        if (!this.aze.equals(that.aze))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (is_set_success());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_e = true && (is_set_e());
+      list.add(present_e);
+      if (present_e)
+        list.add(e);
+
+      boolean present_aze = true && (is_set_aze());
+      list.add(present_aze);
+      if (present_aze)
+        list.add(aze);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getComponentPageInfo_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getComponentPageInfo_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("aze:");
+      if (this.aze == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.aze);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getComponentPageInfo_resultStandardSchemeFactory implements SchemeFactory {
+      public getComponentPageInfo_resultStandardScheme getScheme() {
+        return new getComponentPageInfo_resultStandardScheme();
+      }
+    }
+
+    private static class getComponentPageInfo_resultStandardScheme extends StandardScheme<getComponentPageInfo_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getComponentPageInfo_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new ComponentPageInfo();
+                struct.success.read(iprot);
+                struct.set_success_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // E
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.e = new NotAliveException();
+                struct.e.read(iprot);
+                struct.set_e_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // AZE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.aze = new AuthorizationException();
+                struct.aze.read(iprot);
+                struct.set_aze_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getComponentPageInfo_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.e != null) {
+          oprot.writeFieldBegin(E_FIELD_DESC);
+          struct.e.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.aze != null) {
+          oprot.writeFieldBegin(AZE_FIELD_DESC);
+          struct.aze.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getComponentPageInfo_resultTupleSchemeFactory implements SchemeFactory {
+      public getComponentPageInfo_resultTupleScheme getScheme() {
+        return new getComponentPageInfo_resultTupleScheme();
+      }
+    }
+
+    private static class getComponentPageInfo_resultTupleScheme extends TupleScheme<getComponentPageInfo_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_success()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_e()) {
+          optionals.set(1);
+        }
+        if (struct.is_set_aze()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.is_set_success()) {
+          struct.success.write(oprot);
+        }
+        if (struct.is_set_e()) {
+          struct.e.write(oprot);
+        }
+        if (struct.is_set_aze()) {
+          struct.aze.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getComponentPageInfo_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.success = new ComponentPageInfo();
+          struct.success.read(iprot);
+          struct.set_success_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.e = new NotAliveException();
+          struct.e.read(iprot);
+          struct.set_e_isSet(true);
+        }
+        if (incoming.get(2)) {
+          struct.aze = new AuthorizationException();
+          struct.aze.read(iprot);
+          struct.set_aze_isSet(true);
+        }
+      }
+    }
+
+  }
+
   public static class getTopologyConf_args implements org.apache.thrift.TBase<getTopologyConf_args, getTopologyConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTopologyConf_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyConf_args");
 
@@ -22705,4 +29235,831 @@
 
   }
 
+  public static class getTopologyHistory_args implements org.apache.thrift.TBase<getTopologyHistory_args, getTopologyHistory_args._Fields>, java.io.Serializable, Cloneable, Comparable<getTopologyHistory_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyHistory_args");
+
+    private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getTopologyHistory_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getTopologyHistory_argsTupleSchemeFactory());
+    }
+
+    private String user; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      USER((short)1, "user");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // USER
+            return USER;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyHistory_args.class, metaDataMap);
+    }
+
+    public getTopologyHistory_args() {
+    }
+
+    public getTopologyHistory_args(
+      String user)
+    {
+      this();
+      this.user = user;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyHistory_args(getTopologyHistory_args other) {
+      if (other.is_set_user()) {
+        this.user = other.user;
+      }
+    }
+
+    public getTopologyHistory_args deepCopy() {
+      return new getTopologyHistory_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.user = null;
+    }
+
+    public String get_user() {
+      return this.user;
+    }
+
+    public void set_user(String user) {
+      this.user = user;
+    }
+
+    public void unset_user() {
+      this.user = null;
+    }
+
+    /** Returns true if field user is set (has been assigned a value) and false otherwise */
+    public boolean is_set_user() {
+      return this.user != null;
+    }
+
+    public void set_user_isSet(boolean value) {
+      if (!value) {
+        this.user = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case USER:
+        if (value == null) {
+          unset_user();
+        } else {
+          set_user((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case USER:
+        return get_user();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case USER:
+        return is_set_user();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyHistory_args)
+        return this.equals((getTopologyHistory_args)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyHistory_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_user = true && this.is_set_user();
+      boolean that_present_user = true && that.is_set_user();
+      if (this_present_user || that_present_user) {
+        if (!(this_present_user && that_present_user))
+          return false;
+        if (!this.user.equals(that.user))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_user = true && (is_set_user());
+      list.add(present_user);
+      if (present_user)
+        list.add(user);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getTopologyHistory_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_user()).compareTo(other.is_set_user());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_user()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, other.user);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyHistory_args(");
+      boolean first = true;
+
+      sb.append("user:");
+      if (this.user == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.user);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getTopologyHistory_argsStandardSchemeFactory implements SchemeFactory {
+      public getTopologyHistory_argsStandardScheme getScheme() {
+        return new getTopologyHistory_argsStandardScheme();
+      }
+    }
+
+    private static class getTopologyHistory_argsStandardScheme extends StandardScheme<getTopologyHistory_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // USER
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.user = iprot.readString();
+                struct.set_user_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.user != null) {
+          oprot.writeFieldBegin(USER_FIELD_DESC);
+          oprot.writeString(struct.user);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTopologyHistory_argsTupleSchemeFactory implements SchemeFactory {
+      public getTopologyHistory_argsTupleScheme getScheme() {
+        return new getTopologyHistory_argsTupleScheme();
+      }
+    }
+
+    private static class getTopologyHistory_argsTupleScheme extends TupleScheme<getTopologyHistory_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_user()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.is_set_user()) {
+          oprot.writeString(struct.user);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.user = iprot.readString();
+          struct.set_user_isSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class getTopologyHistory_result implements org.apache.thrift.TBase<getTopologyHistory_result, getTopologyHistory_result._Fields>, java.io.Serializable, Cloneable, Comparable<getTopologyHistory_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyHistory_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    private static final org.apache.thrift.protocol.TField AZE_FIELD_DESC = new org.apache.thrift.protocol.TField("aze", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getTopologyHistory_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getTopologyHistory_resultTupleSchemeFactory());
+    }
+
+    private TopologyHistoryInfo success; // required
+    private AuthorizationException aze; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      AZE((short)1, "aze");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // AZE
+            return AZE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyHistoryInfo.class)));
+      tmpMap.put(_Fields.AZE, new org.apache.thrift.meta_data.FieldMetaData("aze", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyHistory_result.class, metaDataMap);
+    }
+
+    public getTopologyHistory_result() {
+    }
+
+    public getTopologyHistory_result(
+      TopologyHistoryInfo success,
+      AuthorizationException aze)
+    {
+      this();
+      this.success = success;
+      this.aze = aze;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyHistory_result(getTopologyHistory_result other) {
+      if (other.is_set_success()) {
+        this.success = new TopologyHistoryInfo(other.success);
+      }
+      if (other.is_set_aze()) {
+        this.aze = new AuthorizationException(other.aze);
+      }
+    }
+
+    public getTopologyHistory_result deepCopy() {
+      return new getTopologyHistory_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+      this.aze = null;
+    }
+
+    public TopologyHistoryInfo get_success() {
+      return this.success;
+    }
+
+    public void set_success(TopologyHistoryInfo success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public AuthorizationException get_aze() {
+      return this.aze;
+    }
+
+    public void set_aze(AuthorizationException aze) {
+      this.aze = aze;
+    }
+
+    public void unset_aze() {
+      this.aze = null;
+    }
+
+    /** Returns true if field aze is set (has been assigned a value) and false otherwise */
+    public boolean is_set_aze() {
+      return this.aze != null;
+    }
+
+    public void set_aze_isSet(boolean value) {
+      if (!value) {
+        this.aze = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((TopologyHistoryInfo)value);
+        }
+        break;
+
+      case AZE:
+        if (value == null) {
+          unset_aze();
+        } else {
+          set_aze((AuthorizationException)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case AZE:
+        return get_aze();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case AZE:
+        return is_set_aze();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyHistory_result)
+        return this.equals((getTopologyHistory_result)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyHistory_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_aze = true && this.is_set_aze();
+      boolean that_present_aze = true && that.is_set_aze();
+      if (this_present_aze || that_present_aze) {
+        if (!(this_present_aze && that_present_aze))
+          return false;
+        if (!this.aze.equals(that.aze))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (is_set_success());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      boolean present_aze = true && (is_set_aze());
+      list.add(present_aze);
+      if (present_aze)
+        list.add(aze);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getTopologyHistory_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_aze()).compareTo(other.is_set_aze());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_aze()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.aze, other.aze);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+      }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyHistory_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("aze:");
+      if (this.aze == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.aze);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class getTopologyHistory_resultStandardSchemeFactory implements SchemeFactory {
+      public getTopologyHistory_resultStandardScheme getScheme() {
+        return new getTopologyHistory_resultStandardScheme();
+      }
+    }
+
+    private static class getTopologyHistory_resultStandardScheme extends StandardScheme<getTopologyHistory_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyHistory_result struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new TopologyHistoryInfo();
+                struct.success.read(iprot);
+                struct.set_success_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 1: // AZE
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.aze = new AuthorizationException();
+                struct.aze.read(iprot);
+                struct.set_aze_isSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyHistory_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.aze != null) {
+          oprot.writeFieldBegin(AZE_FIELD_DESC);
+          struct.aze.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getTopologyHistory_resultTupleSchemeFactory implements SchemeFactory {
+      public getTopologyHistory_resultTupleScheme getScheme() {
+        return new getTopologyHistory_resultTupleScheme();
+      }
+    }
+
+    private static class getTopologyHistory_resultTupleScheme extends TupleScheme<getTopologyHistory_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.is_set_success()) {
+          optionals.set(0);
+        }
+        if (struct.is_set_aze()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.is_set_success()) {
+          struct.success.write(oprot);
+        }
+        if (struct.is_set_aze()) {
+          struct.aze.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyHistory_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(2);
+        if (incoming.get(0)) {
+          struct.success = new TopologyHistoryInfo();
+          struct.success.read(iprot);
+          struct.set_success_isSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.aze = new AuthorizationException();
+          struct.aze.read(iprot);
+          struct.set_aze_isSet(true);
+        }
+      }
+    }
+
+  }
+
 }
diff --git a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
index 1d0f3fb..6278be8 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NimbusSummary.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 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.3)")
 public class NimbusSummary implements org.apache.thrift.TBase<NimbusSummary, NimbusSummary._Fields>, java.io.Serializable, Cloneable, Comparable<NimbusSummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusSummary");
 
@@ -381,13 +381,13 @@
       return get_host();
 
     case PORT:
-      return Integer.valueOf(get_port());
+      return get_port();
 
     case UPTIME_SECS:
-      return Integer.valueOf(get_uptime_secs());
+      return get_uptime_secs();
 
     case IS_LEADER:
-      return Boolean.valueOf(is_isLeader());
+      return is_isLeader();
 
     case VERSION:
       return get_version();
diff --git a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
index f272cd8..5ef4b5b 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NodeInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class NodeInfo implements org.apache.thrift.TBase<NodeInfo, NodeInfo._Fields>, java.io.Serializable, Cloneable, Comparable<NodeInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NodeInfo");
 
@@ -461,13 +461,13 @@
           case 2: // PORT
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set380 = iprot.readSetBegin();
-                struct.port = new HashSet<Long>(2*_set380.size);
-                long _elem381;
-                for (int _i382 = 0; _i382 < _set380.size; ++_i382)
+                org.apache.thrift.protocol.TSet _set524 = iprot.readSetBegin();
+                struct.port = new HashSet<Long>(2*_set524.size);
+                long _elem525;
+                for (int _i526 = 0; _i526 < _set524.size; ++_i526)
                 {
-                  _elem381 = iprot.readI64();
-                  struct.port.add(_elem381);
+                  _elem525 = iprot.readI64();
+                  struct.port.add(_elem525);
                 }
                 iprot.readSetEnd();
               }
@@ -498,9 +498,9 @@
         oprot.writeFieldBegin(PORT_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.port.size()));
-          for (long _iter383 : struct.port)
+          for (long _iter527 : struct.port)
           {
-            oprot.writeI64(_iter383);
+            oprot.writeI64(_iter527);
           }
           oprot.writeSetEnd();
         }
@@ -526,9 +526,9 @@
       oprot.writeString(struct.node);
       {
         oprot.writeI32(struct.port.size());
-        for (long _iter384 : struct.port)
+        for (long _iter528 : struct.port)
         {
-          oprot.writeI64(_iter384);
+          oprot.writeI64(_iter528);
         }
       }
     }
@@ -539,13 +539,13 @@
       struct.node = iprot.readString();
       struct.set_node_isSet(true);
       {
-        org.apache.thrift.protocol.TSet _set385 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.port = new HashSet<Long>(2*_set385.size);
-        long _elem386;
-        for (int _i387 = 0; _i387 < _set385.size; ++_i387)
+        org.apache.thrift.protocol.TSet _set529 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.port = new HashSet<Long>(2*_set529.size);
+        long _elem530;
+        for (int _i531 = 0; _i531 < _set529.size; ++_i531)
         {
-          _elem386 = iprot.readI64();
-          struct.port.add(_elem386);
+          _elem530 = iprot.readI64();
+          struct.port.add(_elem530);
         }
       }
       struct.set_port_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java b/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java
index 100da84..275e8ec 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NotAliveException.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class NotAliveException extends TException implements org.apache.thrift.TBase<NotAliveException, NotAliveException._Fields>, java.io.Serializable, Cloneable, Comparable<NotAliveException> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotAliveException");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/NullStruct.java b/storm-core/src/jvm/backtype/storm/generated/NullStruct.java
index f44bccd..9949c2f 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NullStruct.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NullStruct.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class NullStruct implements org.apache.thrift.TBase<NullStruct, NullStruct._Fields>, java.io.Serializable, Cloneable, Comparable<NullStruct> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NullStruct");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/NumErrorsChoice.java b/storm-core/src/jvm/backtype/storm/generated/NumErrorsChoice.java
index 8da6671..17970bc 100644
--- a/storm-core/src/jvm/backtype/storm/generated/NumErrorsChoice.java
+++ b/storm-core/src/jvm/backtype/storm/generated/NumErrorsChoice.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/ProfileAction.java b/storm-core/src/jvm/backtype/storm/generated/ProfileAction.java
new file mode 100644
index 0000000..635a25d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ProfileAction.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum ProfileAction implements org.apache.thrift.TEnum {
+  JPROFILE_STOP(0),
+  JPROFILE_START(1),
+  JPROFILE_DUMP(2),
+  JMAP_DUMP(3),
+  JSTACK_DUMP(4),
+  JVM_RESTART(5);
+
+  private final int value;
+
+  private ProfileAction(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static ProfileAction findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return JPROFILE_STOP;
+      case 1:
+        return JPROFILE_START;
+      case 2:
+        return JPROFILE_DUMP;
+      case 3:
+        return JMAP_DUMP;
+      case 4:
+        return JSTACK_DUMP;
+      case 5:
+        return JVM_RESTART;
+      default:
+        return null;
+    }
+  }
+}
diff --git a/storm-core/src/jvm/backtype/storm/generated/ProfileRequest.java b/storm-core/src/jvm/backtype/storm/generated/ProfileRequest.java
new file mode 100644
index 0000000..42560b5
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/ProfileRequest.java
@@ -0,0 +1,631 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ProfileRequest implements org.apache.thrift.TBase<ProfileRequest, ProfileRequest._Fields>, java.io.Serializable, Cloneable, Comparable<ProfileRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ProfileRequest");
+
+  private static final org.apache.thrift.protocol.TField NODE_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("nodeInfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField ACTION_FIELD_DESC = new org.apache.thrift.protocol.TField("action", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField TIME_STAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("time_stamp", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ProfileRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ProfileRequestTupleSchemeFactory());
+  }
+
+  private NodeInfo nodeInfo; // required
+  private ProfileAction action; // required
+  private long time_stamp; // 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 {
+    NODE_INFO((short)1, "nodeInfo"),
+    /**
+     * 
+     * @see ProfileAction
+     */
+    ACTION((short)2, "action"),
+    TIME_STAMP((short)3, "time_stamp");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NODE_INFO
+          return NODE_INFO;
+        case 2: // ACTION
+          return ACTION;
+        case 3: // TIME_STAMP
+          return TIME_STAMP;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TIME_STAMP_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.TIME_STAMP};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NODE_INFO, new org.apache.thrift.meta_data.FieldMetaData("nodeInfo", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NodeInfo.class)));
+    tmpMap.put(_Fields.ACTION, new org.apache.thrift.meta_data.FieldMetaData("action", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, ProfileAction.class)));
+    tmpMap.put(_Fields.TIME_STAMP, new org.apache.thrift.meta_data.FieldMetaData("time_stamp", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ProfileRequest.class, metaDataMap);
+  }
+
+  public ProfileRequest() {
+  }
+
+  public ProfileRequest(
+    NodeInfo nodeInfo,
+    ProfileAction action)
+  {
+    this();
+    this.nodeInfo = nodeInfo;
+    this.action = action;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ProfileRequest(ProfileRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_nodeInfo()) {
+      this.nodeInfo = new NodeInfo(other.nodeInfo);
+    }
+    if (other.is_set_action()) {
+      this.action = other.action;
+    }
+    this.time_stamp = other.time_stamp;
+  }
+
+  public ProfileRequest deepCopy() {
+    return new ProfileRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.nodeInfo = null;
+    this.action = null;
+    set_time_stamp_isSet(false);
+    this.time_stamp = 0;
+  }
+
+  public NodeInfo get_nodeInfo() {
+    return this.nodeInfo;
+  }
+
+  public void set_nodeInfo(NodeInfo nodeInfo) {
+    this.nodeInfo = nodeInfo;
+  }
+
+  public void unset_nodeInfo() {
+    this.nodeInfo = null;
+  }
+
+  /** Returns true if field nodeInfo is set (has been assigned a value) and false otherwise */
+  public boolean is_set_nodeInfo() {
+    return this.nodeInfo != null;
+  }
+
+  public void set_nodeInfo_isSet(boolean value) {
+    if (!value) {
+      this.nodeInfo = null;
+    }
+  }
+
+  /**
+   * 
+   * @see ProfileAction
+   */
+  public ProfileAction get_action() {
+    return this.action;
+  }
+
+  /**
+   * 
+   * @see ProfileAction
+   */
+  public void set_action(ProfileAction action) {
+    this.action = action;
+  }
+
+  public void unset_action() {
+    this.action = null;
+  }
+
+  /** Returns true if field action is set (has been assigned a value) and false otherwise */
+  public boolean is_set_action() {
+    return this.action != null;
+  }
+
+  public void set_action_isSet(boolean value) {
+    if (!value) {
+      this.action = null;
+    }
+  }
+
+  public long get_time_stamp() {
+    return this.time_stamp;
+  }
+
+  public void set_time_stamp(long time_stamp) {
+    this.time_stamp = time_stamp;
+    set_time_stamp_isSet(true);
+  }
+
+  public void unset_time_stamp() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  /** Returns true if field time_stamp is set (has been assigned a value) and false otherwise */
+  public boolean is_set_time_stamp() {
+    return EncodingUtils.testBit(__isset_bitfield, __TIME_STAMP_ISSET_ID);
+  }
+
+  public void set_time_stamp_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIME_STAMP_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NODE_INFO:
+      if (value == null) {
+        unset_nodeInfo();
+      } else {
+        set_nodeInfo((NodeInfo)value);
+      }
+      break;
+
+    case ACTION:
+      if (value == null) {
+        unset_action();
+      } else {
+        set_action((ProfileAction)value);
+      }
+      break;
+
+    case TIME_STAMP:
+      if (value == null) {
+        unset_time_stamp();
+      } else {
+        set_time_stamp((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NODE_INFO:
+      return get_nodeInfo();
+
+    case ACTION:
+      return get_action();
+
+    case TIME_STAMP:
+      return get_time_stamp();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NODE_INFO:
+      return is_set_nodeInfo();
+    case ACTION:
+      return is_set_action();
+    case TIME_STAMP:
+      return is_set_time_stamp();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ProfileRequest)
+      return this.equals((ProfileRequest)that);
+    return false;
+  }
+
+  public boolean equals(ProfileRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_nodeInfo = true && this.is_set_nodeInfo();
+    boolean that_present_nodeInfo = true && that.is_set_nodeInfo();
+    if (this_present_nodeInfo || that_present_nodeInfo) {
+      if (!(this_present_nodeInfo && that_present_nodeInfo))
+        return false;
+      if (!this.nodeInfo.equals(that.nodeInfo))
+        return false;
+    }
+
+    boolean this_present_action = true && this.is_set_action();
+    boolean that_present_action = true && that.is_set_action();
+    if (this_present_action || that_present_action) {
+      if (!(this_present_action && that_present_action))
+        return false;
+      if (!this.action.equals(that.action))
+        return false;
+    }
+
+    boolean this_present_time_stamp = true && this.is_set_time_stamp();
+    boolean that_present_time_stamp = true && that.is_set_time_stamp();
+    if (this_present_time_stamp || that_present_time_stamp) {
+      if (!(this_present_time_stamp && that_present_time_stamp))
+        return false;
+      if (this.time_stamp != that.time_stamp)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_nodeInfo = true && (is_set_nodeInfo());
+    list.add(present_nodeInfo);
+    if (present_nodeInfo)
+      list.add(nodeInfo);
+
+    boolean present_action = true && (is_set_action());
+    list.add(present_action);
+    if (present_action)
+      list.add(action.getValue());
+
+    boolean present_time_stamp = true && (is_set_time_stamp());
+    list.add(present_time_stamp);
+    if (present_time_stamp)
+      list.add(time_stamp);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ProfileRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_nodeInfo()).compareTo(other.is_set_nodeInfo());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_nodeInfo()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nodeInfo, other.nodeInfo);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_action()).compareTo(other.is_set_action());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_action()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.action, other.action);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_time_stamp()).compareTo(other.is_set_time_stamp());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_time_stamp()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.time_stamp, other.time_stamp);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ProfileRequest(");
+    boolean first = true;
+
+    sb.append("nodeInfo:");
+    if (this.nodeInfo == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.nodeInfo);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("action:");
+    if (this.action == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.action);
+    }
+    first = false;
+    if (is_set_time_stamp()) {
+      if (!first) sb.append(", ");
+      sb.append("time_stamp:");
+      sb.append(this.time_stamp);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_nodeInfo()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'nodeInfo' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_action()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'action' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (nodeInfo != null) {
+      nodeInfo.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ProfileRequestStandardSchemeFactory implements SchemeFactory {
+    public ProfileRequestStandardScheme getScheme() {
+      return new ProfileRequestStandardScheme();
+    }
+  }
+
+  private static class ProfileRequestStandardScheme extends StandardScheme<ProfileRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ProfileRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NODE_INFO
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.nodeInfo = new NodeInfo();
+              struct.nodeInfo.read(iprot);
+              struct.set_nodeInfo_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // ACTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.action = backtype.storm.generated.ProfileAction.findByValue(iprot.readI32());
+              struct.set_action_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TIME_STAMP
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.time_stamp = iprot.readI64();
+              struct.set_time_stamp_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ProfileRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.nodeInfo != null) {
+        oprot.writeFieldBegin(NODE_INFO_FIELD_DESC);
+        struct.nodeInfo.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.action != null) {
+        oprot.writeFieldBegin(ACTION_FIELD_DESC);
+        oprot.writeI32(struct.action.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_time_stamp()) {
+        oprot.writeFieldBegin(TIME_STAMP_FIELD_DESC);
+        oprot.writeI64(struct.time_stamp);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ProfileRequestTupleSchemeFactory implements SchemeFactory {
+    public ProfileRequestTupleScheme getScheme() {
+      return new ProfileRequestTupleScheme();
+    }
+  }
+
+  private static class ProfileRequestTupleScheme extends TupleScheme<ProfileRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ProfileRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      struct.nodeInfo.write(oprot);
+      oprot.writeI32(struct.action.getValue());
+      BitSet optionals = new BitSet();
+      if (struct.is_set_time_stamp()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_time_stamp()) {
+        oprot.writeI64(struct.time_stamp);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ProfileRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.nodeInfo = new NodeInfo();
+      struct.nodeInfo.read(iprot);
+      struct.set_nodeInfo_isSet(true);
+      struct.action = backtype.storm.generated.ProfileAction.findByValue(iprot.readI32());
+      struct.set_action_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.time_stamp = iprot.readI64();
+        struct.set_time_stamp_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
index 2cc7762..49a5631 100644
--- a/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/RebalanceOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class RebalanceOptions implements org.apache.thrift.TBase<RebalanceOptions, RebalanceOptions._Fields>, java.io.Serializable, Cloneable, Comparable<RebalanceOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RebalanceOptions");
 
@@ -292,10 +292,10 @@
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case WAIT_SECS:
-      return Integer.valueOf(get_wait_secs());
+      return get_wait_secs();
 
     case NUM_WORKERS:
-      return Integer.valueOf(get_num_workers());
+      return get_num_workers();
 
     case NUM_EXECUTORS:
       return get_num_executors();
@@ -529,15 +529,15 @@
           case 3: // NUM_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map334 = iprot.readMapBegin();
-                struct.num_executors = new HashMap<String,Integer>(2*_map334.size);
-                String _key335;
-                int _val336;
-                for (int _i337 = 0; _i337 < _map334.size; ++_i337)
+                org.apache.thrift.protocol.TMap _map468 = iprot.readMapBegin();
+                struct.num_executors = new HashMap<String,Integer>(2*_map468.size);
+                String _key469;
+                int _val470;
+                for (int _i471 = 0; _i471 < _map468.size; ++_i471)
                 {
-                  _key335 = iprot.readString();
-                  _val336 = iprot.readI32();
-                  struct.num_executors.put(_key335, _val336);
+                  _key469 = iprot.readString();
+                  _val470 = iprot.readI32();
+                  struct.num_executors.put(_key469, _val470);
                 }
                 iprot.readMapEnd();
               }
@@ -574,10 +574,10 @@
           oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.num_executors.size()));
-            for (Map.Entry<String, Integer> _iter338 : struct.num_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter472 : struct.num_executors.entrySet())
             {
-              oprot.writeString(_iter338.getKey());
-              oprot.writeI32(_iter338.getValue());
+              oprot.writeString(_iter472.getKey());
+              oprot.writeI32(_iter472.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -621,10 +621,10 @@
       if (struct.is_set_num_executors()) {
         {
           oprot.writeI32(struct.num_executors.size());
-          for (Map.Entry<String, Integer> _iter339 : struct.num_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter473 : struct.num_executors.entrySet())
           {
-            oprot.writeString(_iter339.getKey());
-            oprot.writeI32(_iter339.getValue());
+            oprot.writeString(_iter473.getKey());
+            oprot.writeI32(_iter473.getValue());
           }
         }
       }
@@ -644,15 +644,15 @@
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map340 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-          struct.num_executors = new HashMap<String,Integer>(2*_map340.size);
-          String _key341;
-          int _val342;
-          for (int _i343 = 0; _i343 < _map340.size; ++_i343)
+          org.apache.thrift.protocol.TMap _map474 = 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*_map474.size);
+          String _key475;
+          int _val476;
+          for (int _i477 = 0; _i477 < _map474.size; ++_i477)
           {
-            _key341 = iprot.readString();
-            _val342 = iprot.readI32();
-            struct.num_executors.put(_key341, _val342);
+            _key475 = iprot.readString();
+            _val476 = iprot.readI32();
+            struct.num_executors.put(_key475, _val476);
           }
         }
         struct.set_num_executors_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java b/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java
index 27303f1..72c09ab 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ShellComponent.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ShellComponent implements org.apache.thrift.TBase<ShellComponent, ShellComponent._Fields>, java.io.Serializable, Cloneable, Comparable<ShellComponent> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShellComponent");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpecificAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/SpecificAggregateStats.java
new file mode 100644
index 0000000..ccdb919
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/SpecificAggregateStats.java
@@ -0,0 +1,387 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+public class SpecificAggregateStats extends org.apache.thrift.TUnion<SpecificAggregateStats, SpecificAggregateStats._Fields> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpecificAggregateStats");
+  private static final org.apache.thrift.protocol.TField BOLT_FIELD_DESC = new org.apache.thrift.protocol.TField("bolt", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField SPOUT_FIELD_DESC = new org.apache.thrift.protocol.TField("spout", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    BOLT((short)1, "bolt"),
+    SPOUT((short)2, "spout");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // BOLT
+          return BOLT;
+        case 2: // SPOUT
+          return SPOUT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.BOLT, new org.apache.thrift.meta_data.FieldMetaData("bolt", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, BoltAggregateStats.class)));
+    tmpMap.put(_Fields.SPOUT, new org.apache.thrift.meta_data.FieldMetaData("spout", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpoutAggregateStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpecificAggregateStats.class, metaDataMap);
+  }
+
+  public SpecificAggregateStats() {
+    super();
+  }
+
+  public SpecificAggregateStats(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public SpecificAggregateStats(SpecificAggregateStats other) {
+    super(other);
+  }
+  public SpecificAggregateStats deepCopy() {
+    return new SpecificAggregateStats(this);
+  }
+
+  public static SpecificAggregateStats bolt(BoltAggregateStats value) {
+    SpecificAggregateStats x = new SpecificAggregateStats();
+    x.set_bolt(value);
+    return x;
+  }
+
+  public static SpecificAggregateStats spout(SpoutAggregateStats value) {
+    SpecificAggregateStats x = new SpecificAggregateStats();
+    x.set_spout(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case BOLT:
+        if (value instanceof BoltAggregateStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type BoltAggregateStats for field 'bolt', but got " + value.getClass().getSimpleName());
+      case SPOUT:
+        if (value instanceof SpoutAggregateStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type SpoutAggregateStats for field 'spout', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case BOLT:
+          if (field.type == BOLT_FIELD_DESC.type) {
+            BoltAggregateStats bolt;
+            bolt = new BoltAggregateStats();
+            bolt.read(iprot);
+            return bolt;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SPOUT:
+          if (field.type == SPOUT_FIELD_DESC.type) {
+            SpoutAggregateStats spout;
+            spout = new SpoutAggregateStats();
+            spout.read(iprot);
+            return spout;
+          } else {
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case BOLT:
+        BoltAggregateStats bolt = (BoltAggregateStats)value_;
+        bolt.write(oprot);
+        return;
+      case SPOUT:
+        SpoutAggregateStats spout = (SpoutAggregateStats)value_;
+        spout.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException {
+    _Fields setField = _Fields.findByThriftId(fieldID);
+    if (setField != null) {
+      switch (setField) {
+        case BOLT:
+          BoltAggregateStats bolt;
+          bolt = new BoltAggregateStats();
+          bolt.read(iprot);
+          return bolt;
+        case SPOUT:
+          SpoutAggregateStats spout;
+          spout = new SpoutAggregateStats();
+          spout.read(iprot);
+          return spout;
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      throw new TProtocolException("Couldn't find a field with field id " + fieldID);
+    }
+  }
+
+  @Override
+  protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    switch (setField_) {
+      case BOLT:
+        BoltAggregateStats bolt = (BoltAggregateStats)value_;
+        bolt.write(oprot);
+        return;
+      case SPOUT:
+        SpoutAggregateStats spout = (SpoutAggregateStats)value_;
+        spout.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case BOLT:
+        return BOLT_FIELD_DESC;
+      case SPOUT:
+        return SPOUT_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected org.apache.thrift.protocol.TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+
+  public BoltAggregateStats get_bolt() {
+    if (getSetField() == _Fields.BOLT) {
+      return (BoltAggregateStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'bolt' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_bolt(BoltAggregateStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.BOLT;
+    value_ = value;
+  }
+
+  public SpoutAggregateStats get_spout() {
+    if (getSetField() == _Fields.SPOUT) {
+      return (SpoutAggregateStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'spout' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_spout(SpoutAggregateStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SPOUT;
+    value_ = value;
+  }
+
+  public boolean is_set_bolt() {
+    return setField_ == _Fields.BOLT;
+  }
+
+
+  public boolean is_set_spout() {
+    return setField_ == _Fields.SPOUT;
+  }
+
+
+  public boolean equals(Object other) {
+    if (other instanceof SpecificAggregateStats) {
+      return equals((SpecificAggregateStats)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(SpecificAggregateStats other) {
+    return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue());
+  }
+
+  @Override
+  public int compareTo(SpecificAggregateStats other) {
+    int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue());
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+    list.add(this.getClass().getName());
+    org.apache.thrift.TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      list.add(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof org.apache.thrift.TEnum) {
+        list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue());
+      } else {
+        list.add(value);
+      }
+    }
+    return list.hashCode();
+  }
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+
+}
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
new file mode 100644
index 0000000..053db0a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutAggregateStats.java
@@ -0,0 +1,407 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField COMPLETE_LATENCY_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("complete_latency_ms", org.apache.thrift.protocol.TType.DOUBLE, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SpoutAggregateStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SpoutAggregateStatsTupleSchemeFactory());
+  }
+
+  private double complete_latency_ms; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    COMPLETE_LATENCY_MS((short)1, "complete_latency_ms");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // COMPLETE_LATENCY_MS
+          return COMPLETE_LATENCY_MS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __COMPLETE_LATENCY_MS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.COMPLETE_LATENCY_MS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.COMPLETE_LATENCY_MS, new org.apache.thrift.meta_data.FieldMetaData("complete_latency_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpoutAggregateStats.class, metaDataMap);
+  }
+
+  public SpoutAggregateStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SpoutAggregateStats(SpoutAggregateStats other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.complete_latency_ms = other.complete_latency_ms;
+  }
+
+  public SpoutAggregateStats deepCopy() {
+    return new SpoutAggregateStats(this);
+  }
+
+  @Override
+  public void clear() {
+    set_complete_latency_ms_isSet(false);
+    this.complete_latency_ms = 0.0;
+  }
+
+  public double get_complete_latency_ms() {
+    return this.complete_latency_ms;
+  }
+
+  public void set_complete_latency_ms(double complete_latency_ms) {
+    this.complete_latency_ms = complete_latency_ms;
+    set_complete_latency_ms_isSet(true);
+  }
+
+  public void unset_complete_latency_ms() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID);
+  }
+
+  /** Returns true if field complete_latency_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_complete_latency_ms() {
+    return EncodingUtils.testBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID);
+  }
+
+  public void set_complete_latency_ms_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __COMPLETE_LATENCY_MS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case COMPLETE_LATENCY_MS:
+      if (value == null) {
+        unset_complete_latency_ms();
+      } else {
+        set_complete_latency_ms((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case COMPLETE_LATENCY_MS:
+      return get_complete_latency_ms();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case COMPLETE_LATENCY_MS:
+      return is_set_complete_latency_ms();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SpoutAggregateStats)
+      return this.equals((SpoutAggregateStats)that);
+    return false;
+  }
+
+  public boolean equals(SpoutAggregateStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_complete_latency_ms = true && this.is_set_complete_latency_ms();
+    boolean that_present_complete_latency_ms = true && that.is_set_complete_latency_ms();
+    if (this_present_complete_latency_ms || that_present_complete_latency_ms) {
+      if (!(this_present_complete_latency_ms && that_present_complete_latency_ms))
+        return false;
+      if (this.complete_latency_ms != that.complete_latency_ms)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_complete_latency_ms = true && (is_set_complete_latency_ms());
+    list.add(present_complete_latency_ms);
+    if (present_complete_latency_ms)
+      list.add(complete_latency_ms);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SpoutAggregateStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_complete_latency_ms()).compareTo(other.is_set_complete_latency_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_complete_latency_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.complete_latency_ms, other.complete_latency_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SpoutAggregateStats(");
+    boolean first = true;
+
+    if (is_set_complete_latency_ms()) {
+      sb.append("complete_latency_ms:");
+      sb.append(this.complete_latency_ms);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SpoutAggregateStatsStandardSchemeFactory implements SchemeFactory {
+    public SpoutAggregateStatsStandardScheme getScheme() {
+      return new SpoutAggregateStatsStandardScheme();
+    }
+  }
+
+  private static class SpoutAggregateStatsStandardScheme extends StandardScheme<SpoutAggregateStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // COMPLETE_LATENCY_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.complete_latency_ms = iprot.readDouble();
+              struct.set_complete_latency_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_complete_latency_ms()) {
+        oprot.writeFieldBegin(COMPLETE_LATENCY_MS_FIELD_DESC);
+        oprot.writeDouble(struct.complete_latency_ms);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SpoutAggregateStatsTupleSchemeFactory implements SchemeFactory {
+    public SpoutAggregateStatsTupleScheme getScheme() {
+      return new SpoutAggregateStatsTupleScheme();
+    }
+  }
+
+  private static class SpoutAggregateStatsTupleScheme extends TupleScheme<SpoutAggregateStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_complete_latency_ms()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_complete_latency_ms()) {
+        oprot.writeDouble(struct.complete_latency_ms);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SpoutAggregateStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.complete_latency_ms = iprot.readDouble();
+        struct.set_complete_latency_ms_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java b/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java
index 42fd37b..492cc36 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutSpec.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class SpoutSpec implements org.apache.thrift.TBase<SpoutSpec, SpoutSpec._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutSpec> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutSpec");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
index c18a7d0..03fb7fe 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SpoutStats.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class SpoutStats implements org.apache.thrift.TBase<SpoutStats, SpoutStats._Fields>, java.io.Serializable, Cloneable, Comparable<SpoutStats> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutStats");
 
@@ -602,27 +602,27 @@
           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 _map216 = iprot.readMapBegin();
+                struct.acked = new HashMap<String,Map<String,Long>>(2*_map216.size);
+                String _key217;
+                Map<String,Long> _val218;
+                for (int _i219 = 0; _i219 < _map216.size; ++_i219)
                 {
-                  _key199 = iprot.readString();
+                  _key217 = 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 _map220 = iprot.readMapBegin();
+                    _val218 = new HashMap<String,Long>(2*_map220.size);
+                    String _key221;
+                    long _val222;
+                    for (int _i223 = 0; _i223 < _map220.size; ++_i223)
                     {
-                      _key203 = iprot.readString();
-                      _val204 = iprot.readI64();
-                      _val200.put(_key203, _val204);
+                      _key221 = iprot.readString();
+                      _val222 = iprot.readI64();
+                      _val218.put(_key221, _val222);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.acked.put(_key199, _val200);
+                  struct.acked.put(_key217, _val218);
                 }
                 iprot.readMapEnd();
               }
@@ -634,27 +634,27 @@
           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 _map224 = iprot.readMapBegin();
+                struct.failed = new HashMap<String,Map<String,Long>>(2*_map224.size);
+                String _key225;
+                Map<String,Long> _val226;
+                for (int _i227 = 0; _i227 < _map224.size; ++_i227)
                 {
-                  _key207 = iprot.readString();
+                  _key225 = 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 _map228 = iprot.readMapBegin();
+                    _val226 = new HashMap<String,Long>(2*_map228.size);
+                    String _key229;
+                    long _val230;
+                    for (int _i231 = 0; _i231 < _map228.size; ++_i231)
                     {
-                      _key211 = iprot.readString();
-                      _val212 = iprot.readI64();
-                      _val208.put(_key211, _val212);
+                      _key229 = iprot.readString();
+                      _val230 = iprot.readI64();
+                      _val226.put(_key229, _val230);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.failed.put(_key207, _val208);
+                  struct.failed.put(_key225, _val226);
                 }
                 iprot.readMapEnd();
               }
@@ -666,27 +666,27 @@
           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 _map232 = iprot.readMapBegin();
+                struct.complete_ms_avg = new HashMap<String,Map<String,Double>>(2*_map232.size);
+                String _key233;
+                Map<String,Double> _val234;
+                for (int _i235 = 0; _i235 < _map232.size; ++_i235)
                 {
-                  _key215 = iprot.readString();
+                  _key233 = 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 _map236 = iprot.readMapBegin();
+                    _val234 = new HashMap<String,Double>(2*_map236.size);
+                    String _key237;
+                    double _val238;
+                    for (int _i239 = 0; _i239 < _map236.size; ++_i239)
                     {
-                      _key219 = iprot.readString();
-                      _val220 = iprot.readDouble();
-                      _val216.put(_key219, _val220);
+                      _key237 = iprot.readString();
+                      _val238 = iprot.readDouble();
+                      _val234.put(_key237, _val238);
                     }
                     iprot.readMapEnd();
                   }
-                  struct.complete_ms_avg.put(_key215, _val216);
+                  struct.complete_ms_avg.put(_key233, _val234);
                 }
                 iprot.readMapEnd();
               }
@@ -712,15 +712,15 @@
         oprot.writeFieldBegin(ACKED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.acked.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter222 : struct.acked.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter240 : struct.acked.entrySet())
           {
-            oprot.writeString(_iter222.getKey());
+            oprot.writeString(_iter240.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, _iter240.getValue().size()));
+              for (Map.Entry<String, Long> _iter241 : _iter240.getValue().entrySet())
               {
-                oprot.writeString(_iter223.getKey());
-                oprot.writeI64(_iter223.getValue());
+                oprot.writeString(_iter241.getKey());
+                oprot.writeI64(_iter241.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -733,15 +733,15 @@
         oprot.writeFieldBegin(FAILED_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.failed.size()));
-          for (Map.Entry<String, Map<String,Long>> _iter224 : struct.failed.entrySet())
+          for (Map.Entry<String, Map<String,Long>> _iter242 : struct.failed.entrySet())
           {
-            oprot.writeString(_iter224.getKey());
+            oprot.writeString(_iter242.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, _iter242.getValue().size()));
+              for (Map.Entry<String, Long> _iter243 : _iter242.getValue().entrySet())
               {
-                oprot.writeString(_iter225.getKey());
-                oprot.writeI64(_iter225.getValue());
+                oprot.writeString(_iter243.getKey());
+                oprot.writeI64(_iter243.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -754,15 +754,15 @@
         oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.complete_ms_avg.size()));
-          for (Map.Entry<String, Map<String,Double>> _iter226 : struct.complete_ms_avg.entrySet())
+          for (Map.Entry<String, Map<String,Double>> _iter244 : struct.complete_ms_avg.entrySet())
           {
-            oprot.writeString(_iter226.getKey());
+            oprot.writeString(_iter244.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, _iter244.getValue().size()));
+              for (Map.Entry<String, Double> _iter245 : _iter244.getValue().entrySet())
               {
-                oprot.writeString(_iter227.getKey());
-                oprot.writeDouble(_iter227.getValue());
+                oprot.writeString(_iter245.getKey());
+                oprot.writeDouble(_iter245.getValue());
               }
               oprot.writeMapEnd();
             }
@@ -790,45 +790,45 @@
       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>> _iter246 : struct.acked.entrySet())
         {
-          oprot.writeString(_iter228.getKey());
+          oprot.writeString(_iter246.getKey());
           {
-            oprot.writeI32(_iter228.getValue().size());
-            for (Map.Entry<String, Long> _iter229 : _iter228.getValue().entrySet())
+            oprot.writeI32(_iter246.getValue().size());
+            for (Map.Entry<String, Long> _iter247 : _iter246.getValue().entrySet())
             {
-              oprot.writeString(_iter229.getKey());
-              oprot.writeI64(_iter229.getValue());
+              oprot.writeString(_iter247.getKey());
+              oprot.writeI64(_iter247.getValue());
             }
           }
         }
       }
       {
         oprot.writeI32(struct.failed.size());
-        for (Map.Entry<String, Map<String,Long>> _iter230 : struct.failed.entrySet())
+        for (Map.Entry<String, Map<String,Long>> _iter248 : struct.failed.entrySet())
         {
-          oprot.writeString(_iter230.getKey());
+          oprot.writeString(_iter248.getKey());
           {
-            oprot.writeI32(_iter230.getValue().size());
-            for (Map.Entry<String, Long> _iter231 : _iter230.getValue().entrySet())
+            oprot.writeI32(_iter248.getValue().size());
+            for (Map.Entry<String, Long> _iter249 : _iter248.getValue().entrySet())
             {
-              oprot.writeString(_iter231.getKey());
-              oprot.writeI64(_iter231.getValue());
+              oprot.writeString(_iter249.getKey());
+              oprot.writeI64(_iter249.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>> _iter250 : struct.complete_ms_avg.entrySet())
         {
-          oprot.writeString(_iter232.getKey());
+          oprot.writeString(_iter250.getKey());
           {
-            oprot.writeI32(_iter232.getValue().size());
-            for (Map.Entry<String, Double> _iter233 : _iter232.getValue().entrySet())
+            oprot.writeI32(_iter250.getValue().size());
+            for (Map.Entry<String, Double> _iter251 : _iter250.getValue().entrySet())
             {
-              oprot.writeString(_iter233.getKey());
-              oprot.writeDouble(_iter233.getValue());
+              oprot.writeString(_iter251.getKey());
+              oprot.writeDouble(_iter251.getValue());
             }
           }
         }
@@ -839,74 +839,74 @@
     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 _map252 = 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*_map252.size);
+        String _key253;
+        Map<String,Long> _val254;
+        for (int _i255 = 0; _i255 < _map252.size; ++_i255)
         {
-          _key235 = iprot.readString();
+          _key253 = 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 _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)
             {
-              _key239 = iprot.readString();
-              _val240 = iprot.readI64();
-              _val236.put(_key239, _val240);
+              _key257 = iprot.readString();
+              _val258 = iprot.readI64();
+              _val254.put(_key257, _val258);
             }
           }
-          struct.acked.put(_key235, _val236);
+          struct.acked.put(_key253, _val254);
         }
       }
       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 _map260 = 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*_map260.size);
+        String _key261;
+        Map<String,Long> _val262;
+        for (int _i263 = 0; _i263 < _map260.size; ++_i263)
         {
-          _key243 = iprot.readString();
+          _key261 = 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 _map264 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, iprot.readI32());
+            _val262 = new HashMap<String,Long>(2*_map264.size);
+            String _key265;
+            long _val266;
+            for (int _i267 = 0; _i267 < _map264.size; ++_i267)
             {
-              _key247 = iprot.readString();
-              _val248 = iprot.readI64();
-              _val244.put(_key247, _val248);
+              _key265 = iprot.readString();
+              _val266 = iprot.readI64();
+              _val262.put(_key265, _val266);
             }
           }
-          struct.failed.put(_key243, _val244);
+          struct.failed.put(_key261, _val262);
         }
       }
       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 _map268 = 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*_map268.size);
+        String _key269;
+        Map<String,Double> _val270;
+        for (int _i271 = 0; _i271 < _map268.size; ++_i271)
         {
-          _key251 = iprot.readString();
+          _key269 = 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 _map272 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32());
+            _val270 = new HashMap<String,Double>(2*_map272.size);
+            String _key273;
+            double _val274;
+            for (int _i275 = 0; _i275 < _map272.size; ++_i275)
             {
-              _key255 = iprot.readString();
-              _val256 = iprot.readDouble();
-              _val252.put(_key255, _val256);
+              _key273 = iprot.readString();
+              _val274 = iprot.readDouble();
+              _val270.put(_key273, _val274);
             }
           }
-          struct.complete_ms_avg.put(_key251, _val252);
+          struct.complete_ms_avg.put(_key269, _val270);
         }
       }
       struct.set_complete_ms_avg_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java b/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java
index 15d34ae..24b3e5c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StateSpoutSpec.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class StateSpoutSpec implements org.apache.thrift.TBase<StateSpoutSpec, StateSpoutSpec._Fields>, java.io.Serializable, Cloneable, Comparable<StateSpoutSpec> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StateSpoutSpec");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormBase.java b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
index 250cc9c..5f80c59 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StormBase.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormBase.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class StormBase implements org.apache.thrift.TBase<StormBase, StormBase._Fields>, java.io.Serializable, Cloneable, Comparable<StormBase> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormBase");
 
@@ -611,13 +611,13 @@
       return get_status();
 
     case NUM_WORKERS:
-      return Integer.valueOf(get_num_workers());
+      return get_num_workers();
 
     case COMPONENT_EXECUTORS:
       return get_component_executors();
 
     case LAUNCH_TIME_SECS:
-      return Integer.valueOf(get_launch_time_secs());
+      return get_launch_time_secs();
 
     case OWNER:
       return get_owner();
@@ -1090,15 +1090,15 @@
           case 4: // COMPONENT_EXECUTORS
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map434 = iprot.readMapBegin();
-                struct.component_executors = new HashMap<String,Integer>(2*_map434.size);
-                String _key435;
-                int _val436;
-                for (int _i437 = 0; _i437 < _map434.size; ++_i437)
+                org.apache.thrift.protocol.TMap _map588 = iprot.readMapBegin();
+                struct.component_executors = new HashMap<String,Integer>(2*_map588.size);
+                String _key589;
+                int _val590;
+                for (int _i591 = 0; _i591 < _map588.size; ++_i591)
                 {
-                  _key435 = iprot.readString();
-                  _val436 = iprot.readI32();
-                  struct.component_executors.put(_key435, _val436);
+                  _key589 = iprot.readString();
+                  _val590 = iprot.readI32();
+                  struct.component_executors.put(_key589, _val590);
                 }
                 iprot.readMapEnd();
               }
@@ -1143,16 +1143,16 @@
           case 9: // COMPONENT_DEBUG
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map438 = iprot.readMapBegin();
-                struct.component_debug = new HashMap<String,DebugOptions>(2*_map438.size);
-                String _key439;
-                DebugOptions _val440;
-                for (int _i441 = 0; _i441 < _map438.size; ++_i441)
+                org.apache.thrift.protocol.TMap _map592 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map592.size);
+                String _key593;
+                DebugOptions _val594;
+                for (int _i595 = 0; _i595 < _map592.size; ++_i595)
                 {
-                  _key439 = iprot.readString();
-                  _val440 = new DebugOptions();
-                  _val440.read(iprot);
-                  struct.component_debug.put(_key439, _val440);
+                  _key593 = iprot.readString();
+                  _val594 = new DebugOptions();
+                  _val594.read(iprot);
+                  struct.component_debug.put(_key593, _val594);
                 }
                 iprot.readMapEnd();
               }
@@ -1192,10 +1192,10 @@
           oprot.writeFieldBegin(COMPONENT_EXECUTORS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.component_executors.size()));
-            for (Map.Entry<String, Integer> _iter442 : struct.component_executors.entrySet())
+            for (Map.Entry<String, Integer> _iter596 : struct.component_executors.entrySet())
             {
-              oprot.writeString(_iter442.getKey());
-              oprot.writeI32(_iter442.getValue());
+              oprot.writeString(_iter596.getKey());
+              oprot.writeI32(_iter596.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1233,10 +1233,10 @@
           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> _iter443 : struct.component_debug.entrySet())
+            for (Map.Entry<String, DebugOptions> _iter597 : struct.component_debug.entrySet())
             {
-              oprot.writeString(_iter443.getKey());
-              _iter443.getValue().write(oprot);
+              oprot.writeString(_iter597.getKey());
+              _iter597.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1286,10 +1286,10 @@
       if (struct.is_set_component_executors()) {
         {
           oprot.writeI32(struct.component_executors.size());
-          for (Map.Entry<String, Integer> _iter444 : struct.component_executors.entrySet())
+          for (Map.Entry<String, Integer> _iter598 : struct.component_executors.entrySet())
           {
-            oprot.writeString(_iter444.getKey());
-            oprot.writeI32(_iter444.getValue());
+            oprot.writeString(_iter598.getKey());
+            oprot.writeI32(_iter598.getValue());
           }
         }
       }
@@ -1308,10 +1308,10 @@
       if (struct.is_set_component_debug()) {
         {
           oprot.writeI32(struct.component_debug.size());
-          for (Map.Entry<String, DebugOptions> _iter445 : struct.component_debug.entrySet())
+          for (Map.Entry<String, DebugOptions> _iter599 : struct.component_debug.entrySet())
           {
-            oprot.writeString(_iter445.getKey());
-            _iter445.getValue().write(oprot);
+            oprot.writeString(_iter599.getKey());
+            _iter599.getValue().write(oprot);
           }
         }
       }
@@ -1329,15 +1329,15 @@
       BitSet incoming = iprot.readBitSet(6);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TMap _map446 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32());
-          struct.component_executors = new HashMap<String,Integer>(2*_map446.size);
-          String _key447;
-          int _val448;
-          for (int _i449 = 0; _i449 < _map446.size; ++_i449)
+          org.apache.thrift.protocol.TMap _map600 = 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*_map600.size);
+          String _key601;
+          int _val602;
+          for (int _i603 = 0; _i603 < _map600.size; ++_i603)
           {
-            _key447 = iprot.readString();
-            _val448 = iprot.readI32();
-            struct.component_executors.put(_key447, _val448);
+            _key601 = iprot.readString();
+            _val602 = iprot.readI32();
+            struct.component_executors.put(_key601, _val602);
           }
         }
         struct.set_component_executors_isSet(true);
@@ -1361,16 +1361,16 @@
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TMap _map450 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.component_debug = new HashMap<String,DebugOptions>(2*_map450.size);
-          String _key451;
-          DebugOptions _val452;
-          for (int _i453 = 0; _i453 < _map450.size; ++_i453)
+          org.apache.thrift.protocol.TMap _map604 = 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*_map604.size);
+          String _key605;
+          DebugOptions _val606;
+          for (int _i607 = 0; _i607 < _map604.size; ++_i607)
           {
-            _key451 = iprot.readString();
-            _val452 = new DebugOptions();
-            _val452.read(iprot);
-            struct.component_debug.put(_key451, _val452);
+            _key605 = iprot.readString();
+            _val606 = new DebugOptions();
+            _val606.read(iprot);
+            struct.component_debug.put(_key605, _val606);
           }
         }
         struct.set_component_debug_isSet(true);
diff --git a/storm-core/src/jvm/backtype/storm/generated/StormTopology.java b/storm-core/src/jvm/backtype/storm/generated/StormTopology.java
index d022e95..352d12d 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StormTopology.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StormTopology.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,13 +51,14 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class StormTopology implements org.apache.thrift.TBase<StormTopology, StormTopology._Fields>, java.io.Serializable, Cloneable, Comparable<StormTopology> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormTopology");
 
   private static final org.apache.thrift.protocol.TField SPOUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("spouts", org.apache.thrift.protocol.TType.MAP, (short)1);
   private static final org.apache.thrift.protocol.TField BOLTS_FIELD_DESC = new org.apache.thrift.protocol.TField("bolts", org.apache.thrift.protocol.TType.MAP, (short)2);
   private static final org.apache.thrift.protocol.TField STATE_SPOUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("state_spouts", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField WORKER_HOOKS_FIELD_DESC = new org.apache.thrift.protocol.TField("worker_hooks", org.apache.thrift.protocol.TType.LIST, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -68,12 +69,14 @@
   private Map<String,SpoutSpec> spouts; // required
   private Map<String,Bolt> bolts; // required
   private Map<String,StateSpoutSpec> state_spouts; // required
+  private List<ByteBuffer> worker_hooks; // 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 {
     SPOUTS((short)1, "spouts"),
     BOLTS((short)2, "bolts"),
-    STATE_SPOUTS((short)3, "state_spouts");
+    STATE_SPOUTS((short)3, "state_spouts"),
+    WORKER_HOOKS((short)4, "worker_hooks");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -94,6 +97,8 @@
           return BOLTS;
         case 3: // STATE_SPOUTS
           return STATE_SPOUTS;
+        case 4: // WORKER_HOOKS
+          return WORKER_HOOKS;
         default:
           return null;
       }
@@ -134,6 +139,7 @@
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.WORKER_HOOKS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -149,6 +155,9 @@
         new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StateSpoutSpec.class))));
+    tmpMap.put(_Fields.WORKER_HOOKS, new org.apache.thrift.meta_data.FieldMetaData("worker_hooks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING            , true))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormTopology.class, metaDataMap);
   }
@@ -216,6 +225,10 @@
       }
       this.state_spouts = __this__state_spouts;
     }
+    if (other.is_set_worker_hooks()) {
+      List<ByteBuffer> __this__worker_hooks = new ArrayList<ByteBuffer>(other.worker_hooks);
+      this.worker_hooks = __this__worker_hooks;
+    }
   }
 
   public StormTopology deepCopy() {
@@ -227,6 +240,7 @@
     this.spouts = null;
     this.bolts = null;
     this.state_spouts = null;
+    this.worker_hooks = null;
   }
 
   public int get_spouts_size() {
@@ -331,6 +345,44 @@
     }
   }
 
+  public int get_worker_hooks_size() {
+    return (this.worker_hooks == null) ? 0 : this.worker_hooks.size();
+  }
+
+  public java.util.Iterator<ByteBuffer> get_worker_hooks_iterator() {
+    return (this.worker_hooks == null) ? null : this.worker_hooks.iterator();
+  }
+
+  public void add_to_worker_hooks(ByteBuffer elem) {
+    if (this.worker_hooks == null) {
+      this.worker_hooks = new ArrayList<ByteBuffer>();
+    }
+    this.worker_hooks.add(elem);
+  }
+
+  public List<ByteBuffer> get_worker_hooks() {
+    return this.worker_hooks;
+  }
+
+  public void set_worker_hooks(List<ByteBuffer> worker_hooks) {
+    this.worker_hooks = worker_hooks;
+  }
+
+  public void unset_worker_hooks() {
+    this.worker_hooks = null;
+  }
+
+  /** Returns true if field worker_hooks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_worker_hooks() {
+    return this.worker_hooks != null;
+  }
+
+  public void set_worker_hooks_isSet(boolean value) {
+    if (!value) {
+      this.worker_hooks = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case SPOUTS:
@@ -357,6 +409,14 @@
       }
       break;
 
+    case WORKER_HOOKS:
+      if (value == null) {
+        unset_worker_hooks();
+      } else {
+        set_worker_hooks((List<ByteBuffer>)value);
+      }
+      break;
+
     }
   }
 
@@ -371,6 +431,9 @@
     case STATE_SPOUTS:
       return get_state_spouts();
 
+    case WORKER_HOOKS:
+      return get_worker_hooks();
+
     }
     throw new IllegalStateException();
   }
@@ -388,6 +451,8 @@
       return is_set_bolts();
     case STATE_SPOUTS:
       return is_set_state_spouts();
+    case WORKER_HOOKS:
+      return is_set_worker_hooks();
     }
     throw new IllegalStateException();
   }
@@ -432,6 +497,15 @@
         return false;
     }
 
+    boolean this_present_worker_hooks = true && this.is_set_worker_hooks();
+    boolean that_present_worker_hooks = true && that.is_set_worker_hooks();
+    if (this_present_worker_hooks || that_present_worker_hooks) {
+      if (!(this_present_worker_hooks && that_present_worker_hooks))
+        return false;
+      if (!this.worker_hooks.equals(that.worker_hooks))
+        return false;
+    }
+
     return true;
   }
 
@@ -454,6 +528,11 @@
     if (present_state_spouts)
       list.add(state_spouts);
 
+    boolean present_worker_hooks = true && (is_set_worker_hooks());
+    list.add(present_worker_hooks);
+    if (present_worker_hooks)
+      list.add(worker_hooks);
+
     return list.hashCode();
   }
 
@@ -495,6 +574,16 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_worker_hooks()).compareTo(other.is_set_worker_hooks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_worker_hooks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.worker_hooks, other.worker_hooks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -538,6 +627,16 @@
       sb.append(this.state_spouts);
     }
     first = false;
+    if (is_set_worker_hooks()) {
+      if (!first) sb.append(", ");
+      sb.append("worker_hooks:");
+      if (this.worker_hooks == null) {
+        sb.append("null");
+      } else {
+        org.apache.thrift.TBaseHelper.toString(this.worker_hooks, sb);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -656,6 +755,24 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // WORKER_HOOKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list56 = iprot.readListBegin();
+                struct.worker_hooks = new ArrayList<ByteBuffer>(_list56.size);
+                ByteBuffer _elem57;
+                for (int _i58 = 0; _i58 < _list56.size; ++_i58)
+                {
+                  _elem57 = iprot.readBinary();
+                  struct.worker_hooks.add(_elem57);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_worker_hooks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -673,10 +790,10 @@
         oprot.writeFieldBegin(SPOUTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.spouts.size()));
-          for (Map.Entry<String, SpoutSpec> _iter56 : struct.spouts.entrySet())
+          for (Map.Entry<String, SpoutSpec> _iter59 : struct.spouts.entrySet())
           {
-            oprot.writeString(_iter56.getKey());
-            _iter56.getValue().write(oprot);
+            oprot.writeString(_iter59.getKey());
+            _iter59.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -686,10 +803,10 @@
         oprot.writeFieldBegin(BOLTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.bolts.size()));
-          for (Map.Entry<String, Bolt> _iter57 : struct.bolts.entrySet())
+          for (Map.Entry<String, Bolt> _iter60 : struct.bolts.entrySet())
           {
-            oprot.writeString(_iter57.getKey());
-            _iter57.getValue().write(oprot);
+            oprot.writeString(_iter60.getKey());
+            _iter60.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -699,15 +816,29 @@
         oprot.writeFieldBegin(STATE_SPOUTS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.state_spouts.size()));
-          for (Map.Entry<String, StateSpoutSpec> _iter58 : struct.state_spouts.entrySet())
+          for (Map.Entry<String, StateSpoutSpec> _iter61 : struct.state_spouts.entrySet())
           {
-            oprot.writeString(_iter58.getKey());
-            _iter58.getValue().write(oprot);
+            oprot.writeString(_iter61.getKey());
+            _iter61.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
         oprot.writeFieldEnd();
       }
+      if (struct.worker_hooks != null) {
+        if (struct.is_set_worker_hooks()) {
+          oprot.writeFieldBegin(WORKER_HOOKS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.worker_hooks.size()));
+            for (ByteBuffer _iter62 : struct.worker_hooks)
+            {
+              oprot.writeBinary(_iter62);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -727,26 +858,40 @@
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.spouts.size());
-        for (Map.Entry<String, SpoutSpec> _iter59 : struct.spouts.entrySet())
+        for (Map.Entry<String, SpoutSpec> _iter63 : struct.spouts.entrySet())
         {
-          oprot.writeString(_iter59.getKey());
-          _iter59.getValue().write(oprot);
+          oprot.writeString(_iter63.getKey());
+          _iter63.getValue().write(oprot);
         }
       }
       {
         oprot.writeI32(struct.bolts.size());
-        for (Map.Entry<String, Bolt> _iter60 : struct.bolts.entrySet())
+        for (Map.Entry<String, Bolt> _iter64 : struct.bolts.entrySet())
         {
-          oprot.writeString(_iter60.getKey());
-          _iter60.getValue().write(oprot);
+          oprot.writeString(_iter64.getKey());
+          _iter64.getValue().write(oprot);
         }
       }
       {
         oprot.writeI32(struct.state_spouts.size());
-        for (Map.Entry<String, StateSpoutSpec> _iter61 : struct.state_spouts.entrySet())
+        for (Map.Entry<String, StateSpoutSpec> _iter65 : struct.state_spouts.entrySet())
         {
-          oprot.writeString(_iter61.getKey());
-          _iter61.getValue().write(oprot);
+          oprot.writeString(_iter65.getKey());
+          _iter65.getValue().write(oprot);
+        }
+      }
+      BitSet optionals = new BitSet();
+      if (struct.is_set_worker_hooks()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_worker_hooks()) {
+        {
+          oprot.writeI32(struct.worker_hooks.size());
+          for (ByteBuffer _iter66 : struct.worker_hooks)
+          {
+            oprot.writeBinary(_iter66);
+          }
         }
       }
     }
@@ -755,47 +900,61 @@
     public void read(org.apache.thrift.protocol.TProtocol prot, StormTopology struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map62 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.spouts = new HashMap<String,SpoutSpec>(2*_map62.size);
-        String _key63;
-        SpoutSpec _val64;
-        for (int _i65 = 0; _i65 < _map62.size; ++_i65)
+        org.apache.thrift.protocol.TMap _map67 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.spouts = new HashMap<String,SpoutSpec>(2*_map67.size);
+        String _key68;
+        SpoutSpec _val69;
+        for (int _i70 = 0; _i70 < _map67.size; ++_i70)
         {
-          _key63 = iprot.readString();
-          _val64 = new SpoutSpec();
-          _val64.read(iprot);
-          struct.spouts.put(_key63, _val64);
+          _key68 = iprot.readString();
+          _val69 = new SpoutSpec();
+          _val69.read(iprot);
+          struct.spouts.put(_key68, _val69);
         }
       }
       struct.set_spouts_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map66 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.bolts = new HashMap<String,Bolt>(2*_map66.size);
-        String _key67;
-        Bolt _val68;
-        for (int _i69 = 0; _i69 < _map66.size; ++_i69)
+        org.apache.thrift.protocol.TMap _map71 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.bolts = new HashMap<String,Bolt>(2*_map71.size);
+        String _key72;
+        Bolt _val73;
+        for (int _i74 = 0; _i74 < _map71.size; ++_i74)
         {
-          _key67 = iprot.readString();
-          _val68 = new Bolt();
-          _val68.read(iprot);
-          struct.bolts.put(_key67, _val68);
+          _key72 = iprot.readString();
+          _val73 = new Bolt();
+          _val73.read(iprot);
+          struct.bolts.put(_key72, _val73);
         }
       }
       struct.set_bolts_isSet(true);
       {
-        org.apache.thrift.protocol.TMap _map70 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.state_spouts = new HashMap<String,StateSpoutSpec>(2*_map70.size);
-        String _key71;
-        StateSpoutSpec _val72;
-        for (int _i73 = 0; _i73 < _map70.size; ++_i73)
+        org.apache.thrift.protocol.TMap _map75 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.state_spouts = new HashMap<String,StateSpoutSpec>(2*_map75.size);
+        String _key76;
+        StateSpoutSpec _val77;
+        for (int _i78 = 0; _i78 < _map75.size; ++_i78)
         {
-          _key71 = iprot.readString();
-          _val72 = new StateSpoutSpec();
-          _val72.read(iprot);
-          struct.state_spouts.put(_key71, _val72);
+          _key76 = iprot.readString();
+          _val77 = new StateSpoutSpec();
+          _val77.read(iprot);
+          struct.state_spouts.put(_key76, _val77);
         }
       }
       struct.set_state_spouts_isSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list79 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.worker_hooks = new ArrayList<ByteBuffer>(_list79.size);
+          ByteBuffer _elem80;
+          for (int _i81 = 0; _i81 < _list79.size; ++_i81)
+          {
+            _elem80 = iprot.readBinary();
+            struct.worker_hooks.add(_elem80);
+          }
+        }
+        struct.set_worker_hooks_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java b/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java
index b9bc75d..caa1887 100644
--- a/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/StreamInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class StreamInfo implements org.apache.thrift.TBase<StreamInfo, StreamInfo._Fields>, java.io.Serializable, Cloneable, Comparable<StreamInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StreamInfo");
 
@@ -266,7 +266,7 @@
       return get_output_fields();
 
     case DIRECT:
-      return Boolean.valueOf(is_direct());
+      return is_direct();
 
     }
     throw new IllegalStateException();
diff --git a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
index 2a68c3b..214ded0 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SubmitOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-2-6")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class SubmitOptions implements org.apache.thrift.TBase<SubmitOptions, SubmitOptions._Fields>, java.io.Serializable, Cloneable, Comparable<SubmitOptions> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SubmitOptions");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
index bb4698c..2ce5eb9 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class SupervisorInfo implements org.apache.thrift.TBase<SupervisorInfo, SupervisorInfo._Fields>, java.io.Serializable, Cloneable, Comparable<SupervisorInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorInfo");
 
@@ -63,6 +63,7 @@
   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 @@
   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 @@
     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 @@
           return UPTIME_SECS;
         case 8: // VERSION
           return VERSION;
+        case 9: // RESOURCES_MAP
+          return RESOURCES_MAP;
         default:
           return null;
       }
@@ -162,7 +167,7 @@
   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 @@
         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 @@
     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 @@
     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 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,13 +588,21 @@
       }
       break;
 
+    case RESOURCES_MAP:
+      if (value == null) {
+        unset_resources_map();
+      } else {
+        set_resources_map((Map<String,Double>)value);
+      }
+      break;
+
     }
   }
 
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case TIME_SECS:
-      return Long.valueOf(get_time_secs());
+      return get_time_secs();
 
     case HOSTNAME:
       return get_hostname();
@@ -564,11 +620,14 @@
       return get_scheduler_meta();
 
     case UPTIME_SECS:
-      return Long.valueOf(get_uptime_secs());
+      return get_uptime_secs();
 
     case VERSION:
       return get_version();
 
+    case RESOURCES_MAP:
+      return get_resources_map();
+
     }
     throw new IllegalStateException();
   }
@@ -596,6 +655,8 @@
       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 @@
         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 @@
     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 @@
         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 @@
       }
       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 @@
           case 4: // USED_PORTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list354 = iprot.readListBegin();
-                struct.used_ports = new ArrayList<Long>(_list354.size);
-                long _elem355;
-                for (int _i356 = 0; _i356 < _list354.size; ++_i356)
+                org.apache.thrift.protocol.TList _list488 = iprot.readListBegin();
+                struct.used_ports = new ArrayList<Long>(_list488.size);
+                long _elem489;
+                for (int _i490 = 0; _i490 < _list488.size; ++_i490)
                 {
-                  _elem355 = iprot.readI64();
-                  struct.used_ports.add(_elem355);
+                  _elem489 = iprot.readI64();
+                  struct.used_ports.add(_elem489);
                 }
                 iprot.readListEnd();
               }
@@ -1008,13 +1103,13 @@
           case 5: // META
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list357 = iprot.readListBegin();
-                struct.meta = new ArrayList<Long>(_list357.size);
-                long _elem358;
-                for (int _i359 = 0; _i359 < _list357.size; ++_i359)
+                org.apache.thrift.protocol.TList _list491 = iprot.readListBegin();
+                struct.meta = new ArrayList<Long>(_list491.size);
+                long _elem492;
+                for (int _i493 = 0; _i493 < _list491.size; ++_i493)
                 {
-                  _elem358 = iprot.readI64();
-                  struct.meta.add(_elem358);
+                  _elem492 = iprot.readI64();
+                  struct.meta.add(_elem492);
                 }
                 iprot.readListEnd();
               }
@@ -1026,15 +1121,15 @@
           case 6: // SCHEDULER_META
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map360 = iprot.readMapBegin();
-                struct.scheduler_meta = new HashMap<String,String>(2*_map360.size);
-                String _key361;
-                String _val362;
-                for (int _i363 = 0; _i363 < _map360.size; ++_i363)
+                org.apache.thrift.protocol.TMap _map494 = iprot.readMapBegin();
+                struct.scheduler_meta = new HashMap<String,String>(2*_map494.size);
+                String _key495;
+                String _val496;
+                for (int _i497 = 0; _i497 < _map494.size; ++_i497)
                 {
-                  _key361 = iprot.readString();
-                  _val362 = iprot.readString();
-                  struct.scheduler_meta.put(_key361, _val362);
+                  _key495 = iprot.readString();
+                  _val496 = iprot.readString();
+                  struct.scheduler_meta.put(_key495, _val496);
                 }
                 iprot.readMapEnd();
               }
@@ -1059,6 +1154,26 @@
               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 _map498 = iprot.readMapBegin();
+                struct.resources_map = new HashMap<String,Double>(2*_map498.size);
+                String _key499;
+                double _val500;
+                for (int _i501 = 0; _i501 < _map498.size; ++_i501)
+                {
+                  _key499 = iprot.readString();
+                  _val500 = iprot.readDouble();
+                  struct.resources_map.put(_key499, _val500);
+                }
+                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 @@
           oprot.writeFieldBegin(USED_PORTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.used_ports.size()));
-            for (long _iter364 : struct.used_ports)
+            for (long _iter502 : struct.used_ports)
             {
-              oprot.writeI64(_iter364);
+              oprot.writeI64(_iter502);
             }
             oprot.writeListEnd();
           }
@@ -1106,9 +1221,9 @@
           oprot.writeFieldBegin(META_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.meta.size()));
-            for (long _iter365 : struct.meta)
+            for (long _iter503 : struct.meta)
             {
-              oprot.writeI64(_iter365);
+              oprot.writeI64(_iter503);
             }
             oprot.writeListEnd();
           }
@@ -1120,10 +1235,10 @@
           oprot.writeFieldBegin(SCHEDULER_META_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.scheduler_meta.size()));
-            for (Map.Entry<String, String> _iter366 : struct.scheduler_meta.entrySet())
+            for (Map.Entry<String, String> _iter504 : struct.scheduler_meta.entrySet())
             {
-              oprot.writeString(_iter366.getKey());
-              oprot.writeString(_iter366.getValue());
+              oprot.writeString(_iter504.getKey());
+              oprot.writeString(_iter504.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -1142,6 +1257,21 @@
           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> _iter505 : struct.resources_map.entrySet())
+            {
+              oprot.writeString(_iter505.getKey());
+              oprot.writeDouble(_iter505.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1180,35 +1310,38 @@
       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 _iter367 : struct.used_ports)
+          for (long _iter506 : struct.used_ports)
           {
-            oprot.writeI64(_iter367);
+            oprot.writeI64(_iter506);
           }
         }
       }
       if (struct.is_set_meta()) {
         {
           oprot.writeI32(struct.meta.size());
-          for (long _iter368 : struct.meta)
+          for (long _iter507 : struct.meta)
           {
-            oprot.writeI64(_iter368);
+            oprot.writeI64(_iter507);
           }
         }
       }
       if (struct.is_set_scheduler_meta()) {
         {
           oprot.writeI32(struct.scheduler_meta.size());
-          for (Map.Entry<String, String> _iter369 : struct.scheduler_meta.entrySet())
+          for (Map.Entry<String, String> _iter508 : struct.scheduler_meta.entrySet())
           {
-            oprot.writeString(_iter369.getKey());
-            oprot.writeString(_iter369.getValue());
+            oprot.writeString(_iter508.getKey());
+            oprot.writeString(_iter508.getValue());
           }
         }
       }
@@ -1218,6 +1351,16 @@
       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> _iter509 : struct.resources_map.entrySet())
+          {
+            oprot.writeString(_iter509.getKey());
+            oprot.writeDouble(_iter509.getValue());
+          }
+        }
+      }
     }
 
     @Override
@@ -1227,48 +1370,48 @@
       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 _list370 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.used_ports = new ArrayList<Long>(_list370.size);
-          long _elem371;
-          for (int _i372 = 0; _i372 < _list370.size; ++_i372)
+          org.apache.thrift.protocol.TList _list510 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.used_ports = new ArrayList<Long>(_list510.size);
+          long _elem511;
+          for (int _i512 = 0; _i512 < _list510.size; ++_i512)
           {
-            _elem371 = iprot.readI64();
-            struct.used_ports.add(_elem371);
+            _elem511 = iprot.readI64();
+            struct.used_ports.add(_elem511);
           }
         }
         struct.set_used_ports_isSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list373 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.meta = new ArrayList<Long>(_list373.size);
-          long _elem374;
-          for (int _i375 = 0; _i375 < _list373.size; ++_i375)
+          org.apache.thrift.protocol.TList _list513 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.meta = new ArrayList<Long>(_list513.size);
+          long _elem514;
+          for (int _i515 = 0; _i515 < _list513.size; ++_i515)
           {
-            _elem374 = iprot.readI64();
-            struct.meta.add(_elem374);
+            _elem514 = iprot.readI64();
+            struct.meta.add(_elem514);
           }
         }
         struct.set_meta_isSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TMap _map376 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.scheduler_meta = new HashMap<String,String>(2*_map376.size);
-          String _key377;
-          String _val378;
-          for (int _i379 = 0; _i379 < _map376.size; ++_i379)
+          org.apache.thrift.protocol.TMap _map516 = 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*_map516.size);
+          String _key517;
+          String _val518;
+          for (int _i519 = 0; _i519 < _map516.size; ++_i519)
           {
-            _key377 = iprot.readString();
-            _val378 = iprot.readString();
-            struct.scheduler_meta.put(_key377, _val378);
+            _key517 = iprot.readString();
+            _val518 = iprot.readString();
+            struct.scheduler_meta.put(_key517, _val518);
           }
         }
         struct.set_scheduler_meta_isSet(true);
@@ -1281,6 +1424,21 @@
         struct.version = iprot.readString();
         struct.set_version_isSet(true);
       }
+      if (incoming.get(6)) {
+        {
+          org.apache.thrift.protocol.TMap _map520 = 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*_map520.size);
+          String _key521;
+          double _val522;
+          for (int _i523 = 0; _i523 < _map520.size; ++_i523)
+          {
+            _key521 = iprot.readString();
+            _val522 = iprot.readDouble();
+            struct.resources_map.put(_key521, _val522);
+          }
+        }
+        struct.set_resources_map_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java b/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
index 9bdf9f7..8bdf289 100644
--- a/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/SupervisorSummary.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 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.3)")
 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,9 @@
   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 org.apache.thrift.protocol.TField USED_MEM_FIELD_DESC = new org.apache.thrift.protocol.TField("used_mem", org.apache.thrift.protocol.TType.DOUBLE, (short)8);
+  private static final org.apache.thrift.protocol.TField USED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("used_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)9);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -74,6 +77,9 @@
   private int num_used_workers; // required
   private String supervisor_id; // required
   private String version; // optional
+  private Map<String,Double> total_resources; // optional
+  private double used_mem; // optional
+  private double used_cpu; // 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 +88,10 @@
     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"),
+    USED_MEM((short)8, "used_mem"),
+    USED_CPU((short)9, "used_cpu");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -109,6 +118,12 @@
           return SUPERVISOR_ID;
         case 6: // VERSION
           return VERSION;
+        case 7: // TOTAL_RESOURCES
+          return TOTAL_RESOURCES;
+        case 8: // USED_MEM
+          return USED_MEM;
+        case 9: // USED_CPU
+          return USED_CPU;
         default:
           return null;
       }
@@ -152,8 +167,10 @@
   private static final int __UPTIME_SECS_ISSET_ID = 0;
   private static final int __NUM_WORKERS_ISSET_ID = 1;
   private static final int __NUM_USED_WORKERS_ISSET_ID = 2;
+  private static final int __USED_MEM_ISSET_ID = 3;
+  private static final int __USED_CPU_ISSET_ID = 4;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.VERSION};
+  private static final _Fields optionals[] = {_Fields.VERSION,_Fields.TOTAL_RESOURCES,_Fields.USED_MEM,_Fields.USED_CPU};
   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 +186,14 @@
         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))));
+    tmpMap.put(_Fields.USED_MEM, new org.apache.thrift.meta_data.FieldMetaData("used_mem", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.USED_CPU, new org.apache.thrift.meta_data.FieldMetaData("used_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap);
   }
@@ -213,6 +238,12 @@
     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;
+    }
+    this.used_mem = other.used_mem;
+    this.used_cpu = other.used_cpu;
   }
 
   public SupervisorSummary deepCopy() {
@@ -231,6 +262,11 @@
     this.supervisor_id = null;
     this.version = "VERSION_NOT_PROVIDED";
 
+    this.total_resources = null;
+    set_used_mem_isSet(false);
+    this.used_mem = 0.0;
+    set_used_cpu_isSet(false);
+    this.used_cpu = 0.0;
   }
 
   public String get_host() {
@@ -368,6 +404,84 @@
     }
   }
 
+  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 double get_used_mem() {
+    return this.used_mem;
+  }
+
+  public void set_used_mem(double used_mem) {
+    this.used_mem = used_mem;
+    set_used_mem_isSet(true);
+  }
+
+  public void unset_used_mem() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USED_MEM_ISSET_ID);
+  }
+
+  /** Returns true if field used_mem is set (has been assigned a value) and false otherwise */
+  public boolean is_set_used_mem() {
+    return EncodingUtils.testBit(__isset_bitfield, __USED_MEM_ISSET_ID);
+  }
+
+  public void set_used_mem_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USED_MEM_ISSET_ID, value);
+  }
+
+  public double get_used_cpu() {
+    return this.used_cpu;
+  }
+
+  public void set_used_cpu(double used_cpu) {
+    this.used_cpu = used_cpu;
+    set_used_cpu_isSet(true);
+  }
+
+  public void unset_used_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field used_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_used_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __USED_CPU_ISSET_ID);
+  }
+
+  public void set_used_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USED_CPU_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case HOST:
@@ -418,6 +532,30 @@
       }
       break;
 
+    case TOTAL_RESOURCES:
+      if (value == null) {
+        unset_total_resources();
+      } else {
+        set_total_resources((Map<String,Double>)value);
+      }
+      break;
+
+    case USED_MEM:
+      if (value == null) {
+        unset_used_mem();
+      } else {
+        set_used_mem((Double)value);
+      }
+      break;
+
+    case USED_CPU:
+      if (value == null) {
+        unset_used_cpu();
+      } else {
+        set_used_cpu((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -427,13 +565,13 @@
       return get_host();
 
     case UPTIME_SECS:
-      return Integer.valueOf(get_uptime_secs());
+      return get_uptime_secs();
 
     case NUM_WORKERS:
-      return Integer.valueOf(get_num_workers());
+      return get_num_workers();
 
     case NUM_USED_WORKERS:
-      return Integer.valueOf(get_num_used_workers());
+      return get_num_used_workers();
 
     case SUPERVISOR_ID:
       return get_supervisor_id();
@@ -441,6 +579,15 @@
     case VERSION:
       return get_version();
 
+    case TOTAL_RESOURCES:
+      return get_total_resources();
+
+    case USED_MEM:
+      return get_used_mem();
+
+    case USED_CPU:
+      return get_used_cpu();
+
     }
     throw new IllegalStateException();
   }
@@ -464,6 +611,12 @@
       return is_set_supervisor_id();
     case VERSION:
       return is_set_version();
+    case TOTAL_RESOURCES:
+      return is_set_total_resources();
+    case USED_MEM:
+      return is_set_used_mem();
+    case USED_CPU:
+      return is_set_used_cpu();
     }
     throw new IllegalStateException();
   }
@@ -535,6 +688,33 @@
         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;
+    }
+
+    boolean this_present_used_mem = true && this.is_set_used_mem();
+    boolean that_present_used_mem = true && that.is_set_used_mem();
+    if (this_present_used_mem || that_present_used_mem) {
+      if (!(this_present_used_mem && that_present_used_mem))
+        return false;
+      if (this.used_mem != that.used_mem)
+        return false;
+    }
+
+    boolean this_present_used_cpu = true && this.is_set_used_cpu();
+    boolean that_present_used_cpu = true && that.is_set_used_cpu();
+    if (this_present_used_cpu || that_present_used_cpu) {
+      if (!(this_present_used_cpu && that_present_used_cpu))
+        return false;
+      if (this.used_cpu != that.used_cpu)
+        return false;
+    }
+
     return true;
   }
 
@@ -572,6 +752,21 @@
     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);
+
+    boolean present_used_mem = true && (is_set_used_mem());
+    list.add(present_used_mem);
+    if (present_used_mem)
+      list.add(used_mem);
+
+    boolean present_used_cpu = true && (is_set_used_cpu());
+    list.add(present_used_cpu);
+    if (present_used_cpu)
+      list.add(used_cpu);
+
     return list.hashCode();
   }
 
@@ -643,6 +838,36 @@
         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;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_used_mem()).compareTo(other.is_set_used_mem());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_used_mem()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_mem, other.used_mem);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_used_cpu()).compareTo(other.is_set_used_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_used_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_cpu, other.used_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -700,6 +925,28 @@
       }
       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;
+    }
+    if (is_set_used_mem()) {
+      if (!first) sb.append(", ");
+      sb.append("used_mem:");
+      sb.append(this.used_mem);
+      first = false;
+    }
+    if (is_set_used_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("used_cpu:");
+      sb.append(this.used_cpu);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -813,6 +1060,42 @@
               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 _map82 = iprot.readMapBegin();
+                struct.total_resources = new HashMap<String,Double>(2*_map82.size);
+                String _key83;
+                double _val84;
+                for (int _i85 = 0; _i85 < _map82.size; ++_i85)
+                {
+                  _key83 = iprot.readString();
+                  _val84 = iprot.readDouble();
+                  struct.total_resources.put(_key83, _val84);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_total_resources_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // USED_MEM
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.used_mem = iprot.readDouble();
+              struct.set_used_mem_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // USED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.used_cpu = iprot.readDouble();
+              struct.set_used_cpu_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 +1135,31 @@
           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> _iter86 : struct.total_resources.entrySet())
+            {
+              oprot.writeString(_iter86.getKey());
+              oprot.writeDouble(_iter86.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_used_mem()) {
+        oprot.writeFieldBegin(USED_MEM_FIELD_DESC);
+        oprot.writeDouble(struct.used_mem);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_used_cpu()) {
+        oprot.writeFieldBegin(USED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.used_cpu);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -878,10 +1186,35 @@
       if (struct.is_set_version()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_total_resources()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_used_mem()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_used_cpu()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
       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> _iter87 : struct.total_resources.entrySet())
+          {
+            oprot.writeString(_iter87.getKey());
+            oprot.writeDouble(_iter87.getValue());
+          }
+        }
+      }
+      if (struct.is_set_used_mem()) {
+        oprot.writeDouble(struct.used_mem);
+      }
+      if (struct.is_set_used_cpu()) {
+        oprot.writeDouble(struct.used_cpu);
+      }
     }
 
     @Override
@@ -897,11 +1230,34 @@
       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(4);
       if (incoming.get(0)) {
         struct.version = iprot.readString();
         struct.set_version_isSet(true);
       }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TMap _map88 = 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*_map88.size);
+          String _key89;
+          double _val90;
+          for (int _i91 = 0; _i91 < _map88.size; ++_i91)
+          {
+            _key89 = iprot.readString();
+            _val90 = iprot.readDouble();
+            struct.total_resources.put(_key89, _val90);
+          }
+        }
+        struct.set_total_resources_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.used_mem = iprot.readDouble();
+        struct.set_used_mem_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.used_cpu = iprot.readDouble();
+        struct.set_used_cpu_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java b/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java
index 6810669..810bea8 100644
--- a/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java
+++ b/storm-core/src/jvm/backtype/storm/generated/ThriftSerializedObject.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-10")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class ThriftSerializedObject implements org.apache.thrift.TBase<ThriftSerializedObject, ThriftSerializedObject._Fields>, java.io.Serializable, Cloneable, Comparable<ThriftSerializedObject> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ThriftSerializedObject");
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java b/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
index c6fea05..6e6d704 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyActionOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
new file mode 100644
index 0000000..cced456
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyHistoryInfo.java
@@ -0,0 +1,461 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class TopologyHistoryInfo implements org.apache.thrift.TBase<TopologyHistoryInfo, TopologyHistoryInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyHistoryInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyHistoryInfo");
+
+  private static final org.apache.thrift.protocol.TField TOPO_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("topo_ids", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologyHistoryInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologyHistoryInfoTupleSchemeFactory());
+  }
+
+  private List<String> topo_ids; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TOPO_IDS((short)1, "topo_ids");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TOPO_IDS
+          return TOPO_IDS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TOPO_IDS, new org.apache.thrift.meta_data.FieldMetaData("topo_ids", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyHistoryInfo.class, metaDataMap);
+  }
+
+  public TopologyHistoryInfo() {
+  }
+
+  public TopologyHistoryInfo(
+    List<String> topo_ids)
+  {
+    this();
+    this.topo_ids = topo_ids;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyHistoryInfo(TopologyHistoryInfo other) {
+    if (other.is_set_topo_ids()) {
+      List<String> __this__topo_ids = new ArrayList<String>(other.topo_ids);
+      this.topo_ids = __this__topo_ids;
+    }
+  }
+
+  public TopologyHistoryInfo deepCopy() {
+    return new TopologyHistoryInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.topo_ids = null;
+  }
+
+  public int get_topo_ids_size() {
+    return (this.topo_ids == null) ? 0 : this.topo_ids.size();
+  }
+
+  public java.util.Iterator<String> get_topo_ids_iterator() {
+    return (this.topo_ids == null) ? null : this.topo_ids.iterator();
+  }
+
+  public void add_to_topo_ids(String elem) {
+    if (this.topo_ids == null) {
+      this.topo_ids = new ArrayList<String>();
+    }
+    this.topo_ids.add(elem);
+  }
+
+  public List<String> get_topo_ids() {
+    return this.topo_ids;
+  }
+
+  public void set_topo_ids(List<String> topo_ids) {
+    this.topo_ids = topo_ids;
+  }
+
+  public void unset_topo_ids() {
+    this.topo_ids = null;
+  }
+
+  /** Returns true if field topo_ids is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topo_ids() {
+    return this.topo_ids != null;
+  }
+
+  public void set_topo_ids_isSet(boolean value) {
+    if (!value) {
+      this.topo_ids = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TOPO_IDS:
+      if (value == null) {
+        unset_topo_ids();
+      } else {
+        set_topo_ids((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TOPO_IDS:
+      return get_topo_ids();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TOPO_IDS:
+      return is_set_topo_ids();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyHistoryInfo)
+      return this.equals((TopologyHistoryInfo)that);
+    return false;
+  }
+
+  public boolean equals(TopologyHistoryInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_topo_ids = true && this.is_set_topo_ids();
+    boolean that_present_topo_ids = true && that.is_set_topo_ids();
+    if (this_present_topo_ids || that_present_topo_ids) {
+      if (!(this_present_topo_ids && that_present_topo_ids))
+        return false;
+      if (!this.topo_ids.equals(that.topo_ids))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_topo_ids = true && (is_set_topo_ids());
+    list.add(present_topo_ids);
+    if (present_topo_ids)
+      list.add(topo_ids);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologyHistoryInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_topo_ids()).compareTo(other.is_set_topo_ids());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topo_ids()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topo_ids, other.topo_ids);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyHistoryInfo(");
+    boolean first = true;
+
+    sb.append("topo_ids:");
+    if (this.topo_ids == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topo_ids);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologyHistoryInfoStandardSchemeFactory implements SchemeFactory {
+    public TopologyHistoryInfoStandardScheme getScheme() {
+      return new TopologyHistoryInfoStandardScheme();
+    }
+  }
+
+  private static class TopologyHistoryInfoStandardScheme extends StandardScheme<TopologyHistoryInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TOPO_IDS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list698 = iprot.readListBegin();
+                struct.topo_ids = new ArrayList<String>(_list698.size);
+                String _elem699;
+                for (int _i700 = 0; _i700 < _list698.size; ++_i700)
+                {
+                  _elem699 = iprot.readString();
+                  struct.topo_ids.add(_elem699);
+                }
+                iprot.readListEnd();
+              }
+              struct.set_topo_ids_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.topo_ids != null) {
+        oprot.writeFieldBegin(TOPO_IDS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.topo_ids.size()));
+          for (String _iter701 : struct.topo_ids)
+          {
+            oprot.writeString(_iter701);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TopologyHistoryInfoTupleSchemeFactory implements SchemeFactory {
+    public TopologyHistoryInfoTupleScheme getScheme() {
+      return new TopologyHistoryInfoTupleScheme();
+    }
+  }
+
+  private static class TopologyHistoryInfoTupleScheme extends TupleScheme<TopologyHistoryInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_topo_ids()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.is_set_topo_ids()) {
+        {
+          oprot.writeI32(struct.topo_ids.size());
+          for (String _iter702 : struct.topo_ids)
+          {
+            oprot.writeString(_iter702);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TopologyHistoryInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list703 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.topo_ids = new ArrayList<String>(_list703.size);
+          String _elem704;
+          for (int _i705 = 0; _i705 < _list703.size; ++_i705)
+          {
+            _elem704 = iprot.readString();
+            struct.topo_ids.add(_elem704);
+          }
+        }
+        struct.set_topo_ids_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
index 6c2cc69..be1d706 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-9-3")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class TopologyInfo implements org.apache.thrift.TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable, Comparable<TopologyInfo> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo");
 
@@ -65,6 +65,12 @@
   private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
   private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)514);
   private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)515);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -82,6 +88,12 @@
   private String sched_status; // optional
   private String owner; // optional
   private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // 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 {
@@ -94,7 +106,13 @@
     COMPONENT_DEBUG((short)7, "component_debug"),
     SCHED_STATUS((short)513, "sched_status"),
     OWNER((short)514, "owner"),
-    REPLICATION_COUNT((short)515, "replication_count");
+    REPLICATION_COUNT((short)515, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -129,6 +147,18 @@
           return OWNER;
         case 515: // REPLICATION_COUNT
           return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
         default:
           return null;
       }
@@ -171,8 +201,14 @@
   // isset id assignments
   private static final int __UPTIME_SECS_ISSET_ID = 0;
   private static final int __REPLICATION_COUNT_ISSET_ID = 1;
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 2;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 3;
+  private static final int __REQUESTED_CPU_ISSET_ID = 4;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 7;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT};
+  private static final _Fields optionals[] = {_Fields.COMPONENT_DEBUG,_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
   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);
@@ -195,13 +231,25 @@
     tmpMap.put(_Fields.COMPONENT_DEBUG, new org.apache.thrift.meta_data.FieldMetaData("component_debug", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
-            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT            , "DebugOptions"))));
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class))));
     tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap);
   }
@@ -276,7 +324,7 @@
 
         String __this__component_debug_copy_key = other_element_key;
 
-        DebugOptions __this__component_debug_copy_value = other_element_value;
+        DebugOptions __this__component_debug_copy_value = new DebugOptions(other_element_value);
 
         __this__component_debug.put(__this__component_debug_copy_key, __this__component_debug_copy_value);
       }
@@ -289,6 +337,12 @@
       this.owner = other.owner;
     }
     this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
   }
 
   public TopologyInfo deepCopy() {
@@ -309,6 +363,18 @@
     this.owner = null;
     set_replication_count_isSet(false);
     this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
   }
 
   public String get_id() {
@@ -576,6 +642,138 @@
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
   }
 
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -658,6 +856,54 @@
       }
       break;
 
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -670,7 +916,7 @@
       return get_name();
 
     case UPTIME_SECS:
-      return Integer.valueOf(get_uptime_secs());
+      return get_uptime_secs();
 
     case EXECUTORS:
       return get_executors();
@@ -691,7 +937,25 @@
       return get_owner();
 
     case REPLICATION_COUNT:
-      return Integer.valueOf(get_replication_count());
+      return get_replication_count();
+
+    case REQUESTED_MEMONHEAP:
+      return get_requested_memonheap();
+
+    case REQUESTED_MEMOFFHEAP:
+      return get_requested_memoffheap();
+
+    case REQUESTED_CPU:
+      return get_requested_cpu();
+
+    case ASSIGNED_MEMONHEAP:
+      return get_assigned_memonheap();
+
+    case ASSIGNED_MEMOFFHEAP:
+      return get_assigned_memoffheap();
+
+    case ASSIGNED_CPU:
+      return get_assigned_cpu();
 
     }
     throw new IllegalStateException();
@@ -724,6 +988,18 @@
       return is_set_owner();
     case REPLICATION_COUNT:
       return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
     }
     throw new IllegalStateException();
   }
@@ -831,6 +1107,60 @@
         return false;
     }
 
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
     return true;
   }
 
@@ -888,6 +1218,36 @@
     if (present_replication_count)
       list.add(replication_count);
 
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
     return list.hashCode();
   }
 
@@ -999,6 +1359,66 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1098,6 +1518,42 @@
       sb.append(this.replication_count);
       first = false;
     }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1194,14 +1650,14 @@
           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 _list316 = iprot.readListBegin();
+                struct.executors = new ArrayList<ExecutorSummary>(_list316.size);
+                ExecutorSummary _elem317;
+                for (int _i318 = 0; _i318 < _list316.size; ++_i318)
                 {
-                  _elem299 = new ExecutorSummary();
-                  _elem299.read(iprot);
-                  struct.executors.add(_elem299);
+                  _elem317 = new ExecutorSummary();
+                  _elem317.read(iprot);
+                  struct.executors.add(_elem317);
                 }
                 iprot.readListEnd();
               }
@@ -1221,26 +1677,26 @@
           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 _map319 = iprot.readMapBegin();
+                struct.errors = new HashMap<String,List<ErrorInfo>>(2*_map319.size);
+                String _key320;
+                List<ErrorInfo> _val321;
+                for (int _i322 = 0; _i322 < _map319.size; ++_i322)
                 {
-                  _key302 = iprot.readString();
+                  _key320 = 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 _list323 = iprot.readListBegin();
+                    _val321 = new ArrayList<ErrorInfo>(_list323.size);
+                    ErrorInfo _elem324;
+                    for (int _i325 = 0; _i325 < _list323.size; ++_i325)
                     {
-                      _elem306 = new ErrorInfo();
-                      _elem306.read(iprot);
-                      _val303.add(_elem306);
+                      _elem324 = new ErrorInfo();
+                      _elem324.read(iprot);
+                      _val321.add(_elem324);
                     }
                     iprot.readListEnd();
                   }
-                  struct.errors.put(_key302, _val303);
+                  struct.errors.put(_key320, _val321);
                 }
                 iprot.readMapEnd();
               }
@@ -1252,16 +1708,16 @@
           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 _map326 = iprot.readMapBegin();
+                struct.component_debug = new HashMap<String,DebugOptions>(2*_map326.size);
+                String _key327;
+                DebugOptions _val328;
+                for (int _i329 = 0; _i329 < _map326.size; ++_i329)
                 {
-                  _key309 = iprot.readString();
-                  _val310 = new DebugOptions();
-                  _val310.read(iprot);
-                  struct.component_debug.put(_key309, _val310);
+                  _key327 = iprot.readString();
+                  _val328 = new DebugOptions();
+                  _val328.read(iprot);
+                  struct.component_debug.put(_key327, _val328);
                 }
                 iprot.readMapEnd();
               }
@@ -1294,6 +1750,54 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1324,9 +1828,9 @@
         oprot.writeFieldBegin(EXECUTORS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.executors.size()));
-          for (ExecutorSummary _iter312 : struct.executors)
+          for (ExecutorSummary _iter330 : struct.executors)
           {
-            _iter312.write(oprot);
+            _iter330.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1341,14 +1845,14 @@
         oprot.writeFieldBegin(ERRORS_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.LIST, struct.errors.size()));
-          for (Map.Entry<String, List<ErrorInfo>> _iter313 : struct.errors.entrySet())
+          for (Map.Entry<String, List<ErrorInfo>> _iter331 : struct.errors.entrySet())
           {
-            oprot.writeString(_iter313.getKey());
+            oprot.writeString(_iter331.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, _iter331.getValue().size()));
+              for (ErrorInfo _iter332 : _iter331.getValue())
               {
-                _iter314.write(oprot);
+                _iter332.write(oprot);
               }
               oprot.writeListEnd();
             }
@@ -1362,10 +1866,10 @@
           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> _iter333 : struct.component_debug.entrySet())
             {
-              oprot.writeString(_iter315.getKey());
-              _iter315.getValue().write(oprot);
+              oprot.writeString(_iter333.getKey());
+              _iter333.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -1391,6 +1895,36 @@
         oprot.writeI32(struct.replication_count);
         oprot.writeFieldEnd();
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1413,22 +1947,22 @@
       oprot.writeI32(struct.uptime_secs);
       {
         oprot.writeI32(struct.executors.size());
-        for (ExecutorSummary _iter316 : struct.executors)
+        for (ExecutorSummary _iter334 : struct.executors)
         {
-          _iter316.write(oprot);
+          _iter334.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>> _iter335 : struct.errors.entrySet())
         {
-          oprot.writeString(_iter317.getKey());
+          oprot.writeString(_iter335.getKey());
           {
-            oprot.writeI32(_iter317.getValue().size());
-            for (ErrorInfo _iter318 : _iter317.getValue())
+            oprot.writeI32(_iter335.getValue().size());
+            for (ErrorInfo _iter336 : _iter335.getValue())
             {
-              _iter318.write(oprot);
+              _iter336.write(oprot);
             }
           }
         }
@@ -1446,14 +1980,32 @@
       if (struct.is_set_replication_count()) {
         optionals.set(3);
       }
-      oprot.writeBitSet(optionals, 4);
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(8);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(9);
+      }
+      oprot.writeBitSet(optionals, 10);
       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> _iter337 : struct.component_debug.entrySet())
           {
-            oprot.writeString(_iter319.getKey());
-            _iter319.getValue().write(oprot);
+            oprot.writeString(_iter337.getKey());
+            _iter337.getValue().write(oprot);
           }
         }
       }
@@ -1466,6 +2018,24 @@
       if (struct.is_set_replication_count()) {
         oprot.writeI32(struct.replication_count);
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
     }
 
     @Override
@@ -1478,55 +2048,55 @@
       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 _list338 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.executors = new ArrayList<ExecutorSummary>(_list338.size);
+        ExecutorSummary _elem339;
+        for (int _i340 = 0; _i340 < _list338.size; ++_i340)
         {
-          _elem321 = new ExecutorSummary();
-          _elem321.read(iprot);
-          struct.executors.add(_elem321);
+          _elem339 = new ExecutorSummary();
+          _elem339.read(iprot);
+          struct.executors.add(_elem339);
         }
       }
       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 _map341 = 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*_map341.size);
+        String _key342;
+        List<ErrorInfo> _val343;
+        for (int _i344 = 0; _i344 < _map341.size; ++_i344)
         {
-          _key324 = iprot.readString();
+          _key342 = 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 _list345 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            _val343 = new ArrayList<ErrorInfo>(_list345.size);
+            ErrorInfo _elem346;
+            for (int _i347 = 0; _i347 < _list345.size; ++_i347)
             {
-              _elem328 = new ErrorInfo();
-              _elem328.read(iprot);
-              _val325.add(_elem328);
+              _elem346 = new ErrorInfo();
+              _elem346.read(iprot);
+              _val343.add(_elem346);
             }
           }
-          struct.errors.put(_key324, _val325);
+          struct.errors.put(_key342, _val343);
         }
       }
       struct.set_errors_isSet(true);
-      BitSet incoming = iprot.readBitSet(4);
+      BitSet incoming = iprot.readBitSet(10);
       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 _map348 = 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*_map348.size);
+          String _key349;
+          DebugOptions _val350;
+          for (int _i351 = 0; _i351 < _map348.size; ++_i351)
           {
-            _key331 = iprot.readString();
-            _val332 = new DebugOptions();
-            _val332.read(iprot);
-            struct.component_debug.put(_key331, _val332);
+            _key349 = iprot.readString();
+            _val350 = new DebugOptions();
+            _val350.read(iprot);
+            struct.component_debug.put(_key349, _val350);
           }
         }
         struct.set_component_debug_isSet(true);
@@ -1543,6 +2113,30 @@
         struct.replication_count = iprot.readI32();
         struct.set_replication_count_isSet(true);
       }
+      if (incoming.get(4)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyInitialStatus.java b/storm-core/src/jvm/backtype/storm/generated/TopologyInitialStatus.java
index a8d6c9e..7117bd4 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyInitialStatus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyInitialStatus.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
new file mode 100644
index 0000000..c943cac
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyPageInfo.java
@@ -0,0 +1,2597 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)3);
+  private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField NUM_EXECUTORS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_executors", org.apache.thrift.protocol.TType.I32, (short)7);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_conf", org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField ID_TO_SPOUT_AGG_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("id_to_spout_agg_stats", org.apache.thrift.protocol.TType.MAP, (short)9);
+  private static final org.apache.thrift.protocol.TField ID_TO_BOLT_AGG_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("id_to_bolt_agg_stats", org.apache.thrift.protocol.TType.MAP, (short)10);
+  private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)11);
+  private static final org.apache.thrift.protocol.TField TOPOLOGY_STATS_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_stats", org.apache.thrift.protocol.TType.STRUCT, (short)12);
+  private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)13);
+  private static final org.apache.thrift.protocol.TField DEBUG_OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("debug_options", org.apache.thrift.protocol.TType.STRUCT, (short)14);
+  private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)15);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologyPageInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologyPageInfoTupleSchemeFactory());
+  }
+
+  private String id; // required
+  private String name; // optional
+  private int uptime_secs; // optional
+  private String status; // optional
+  private int num_tasks; // optional
+  private int num_workers; // optional
+  private int num_executors; // optional
+  private String topology_conf; // optional
+  private Map<String,ComponentAggregateStats> id_to_spout_agg_stats; // optional
+  private Map<String,ComponentAggregateStats> id_to_bolt_agg_stats; // optional
+  private String sched_status; // optional
+  private TopologyStats topology_stats; // optional
+  private String owner; // optional
+  private DebugOptions debug_options; // optional
+  private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    ID((short)1, "id"),
+    NAME((short)2, "name"),
+    UPTIME_SECS((short)3, "uptime_secs"),
+    STATUS((short)4, "status"),
+    NUM_TASKS((short)5, "num_tasks"),
+    NUM_WORKERS((short)6, "num_workers"),
+    NUM_EXECUTORS((short)7, "num_executors"),
+    TOPOLOGY_CONF((short)8, "topology_conf"),
+    ID_TO_SPOUT_AGG_STATS((short)9, "id_to_spout_agg_stats"),
+    ID_TO_BOLT_AGG_STATS((short)10, "id_to_bolt_agg_stats"),
+    SCHED_STATUS((short)11, "sched_status"),
+    TOPOLOGY_STATS((short)12, "topology_stats"),
+    OWNER((short)13, "owner"),
+    DEBUG_OPTIONS((short)14, "debug_options"),
+    REPLICATION_COUNT((short)15, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // NAME
+          return NAME;
+        case 3: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 4: // STATUS
+          return STATUS;
+        case 5: // NUM_TASKS
+          return NUM_TASKS;
+        case 6: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 7: // NUM_EXECUTORS
+          return NUM_EXECUTORS;
+        case 8: // TOPOLOGY_CONF
+          return TOPOLOGY_CONF;
+        case 9: // ID_TO_SPOUT_AGG_STATS
+          return ID_TO_SPOUT_AGG_STATS;
+        case 10: // ID_TO_BOLT_AGG_STATS
+          return ID_TO_BOLT_AGG_STATS;
+        case 11: // SCHED_STATUS
+          return SCHED_STATUS;
+        case 12: // TOPOLOGY_STATS
+          return TOPOLOGY_STATS;
+        case 13: // OWNER
+          return OWNER;
+        case 14: // DEBUG_OPTIONS
+          return DEBUG_OPTIONS;
+        case 15: // REPLICATION_COUNT
+          return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __UPTIME_SECS_ISSET_ID = 0;
+  private static final int __NUM_TASKS_ISSET_ID = 1;
+  private static final int __NUM_WORKERS_ISSET_ID = 2;
+  private static final int __NUM_EXECUTORS_ISSET_ID = 3;
+  private static final int __REPLICATION_COUNT_ISSET_ID = 4;
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __REQUESTED_CPU_ISSET_ID = 7;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 8;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 9;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 10;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.NAME,_Fields.UPTIME_SECS,_Fields.STATUS,_Fields.NUM_TASKS,_Fields.NUM_WORKERS,_Fields.NUM_EXECUTORS,_Fields.TOPOLOGY_CONF,_Fields.ID_TO_SPOUT_AGG_STATS,_Fields.ID_TO_BOLT_AGG_STATS,_Fields.SCHED_STATUS,_Fields.TOPOLOGY_STATS,_Fields.OWNER,_Fields.DEBUG_OPTIONS,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NUM_EXECUTORS, new org.apache.thrift.meta_data.FieldMetaData("num_executors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.TOPOLOGY_CONF, new org.apache.thrift.meta_data.FieldMetaData("topology_conf", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.ID_TO_SPOUT_AGG_STATS, new org.apache.thrift.meta_data.FieldMetaData("id_to_spout_agg_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.ID_TO_BOLT_AGG_STATS, new org.apache.thrift.meta_data.FieldMetaData("id_to_bolt_agg_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentAggregateStats.class))));
+    tmpMap.put(_Fields.SCHED_STATUS, new org.apache.thrift.meta_data.FieldMetaData("sched_status", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TOPOLOGY_STATS, new org.apache.thrift.meta_data.FieldMetaData("topology_stats", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyStats.class)));
+    tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DEBUG_OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("debug_options", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DebugOptions.class)));
+    tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyPageInfo.class, metaDataMap);
+  }
+
+  public TopologyPageInfo() {
+  }
+
+  public TopologyPageInfo(
+    String id)
+  {
+    this();
+    this.id = id;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyPageInfo(TopologyPageInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_status()) {
+      this.status = other.status;
+    }
+    this.num_tasks = other.num_tasks;
+    this.num_workers = other.num_workers;
+    this.num_executors = other.num_executors;
+    if (other.is_set_topology_conf()) {
+      this.topology_conf = other.topology_conf;
+    }
+    if (other.is_set_id_to_spout_agg_stats()) {
+      Map<String,ComponentAggregateStats> __this__id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(other.id_to_spout_agg_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.id_to_spout_agg_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__id_to_spout_agg_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__id_to_spout_agg_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__id_to_spout_agg_stats.put(__this__id_to_spout_agg_stats_copy_key, __this__id_to_spout_agg_stats_copy_value);
+      }
+      this.id_to_spout_agg_stats = __this__id_to_spout_agg_stats;
+    }
+    if (other.is_set_id_to_bolt_agg_stats()) {
+      Map<String,ComponentAggregateStats> __this__id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(other.id_to_bolt_agg_stats.size());
+      for (Map.Entry<String, ComponentAggregateStats> other_element : other.id_to_bolt_agg_stats.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        ComponentAggregateStats other_element_value = other_element.getValue();
+
+        String __this__id_to_bolt_agg_stats_copy_key = other_element_key;
+
+        ComponentAggregateStats __this__id_to_bolt_agg_stats_copy_value = new ComponentAggregateStats(other_element_value);
+
+        __this__id_to_bolt_agg_stats.put(__this__id_to_bolt_agg_stats_copy_key, __this__id_to_bolt_agg_stats_copy_value);
+      }
+      this.id_to_bolt_agg_stats = __this__id_to_bolt_agg_stats;
+    }
+    if (other.is_set_sched_status()) {
+      this.sched_status = other.sched_status;
+    }
+    if (other.is_set_topology_stats()) {
+      this.topology_stats = new TopologyStats(other.topology_stats);
+    }
+    if (other.is_set_owner()) {
+      this.owner = other.owner;
+    }
+    if (other.is_set_debug_options()) {
+      this.debug_options = new DebugOptions(other.debug_options);
+    }
+    this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
+  }
+
+  public TopologyPageInfo deepCopy() {
+    return new TopologyPageInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    this.id = null;
+    this.name = null;
+    set_uptime_secs_isSet(false);
+    this.uptime_secs = 0;
+    this.status = null;
+    set_num_tasks_isSet(false);
+    this.num_tasks = 0;
+    set_num_workers_isSet(false);
+    this.num_workers = 0;
+    set_num_executors_isSet(false);
+    this.num_executors = 0;
+    this.topology_conf = null;
+    this.id_to_spout_agg_stats = null;
+    this.id_to_bolt_agg_stats = null;
+    this.sched_status = null;
+    this.topology_stats = null;
+    this.owner = null;
+    this.debug_options = null;
+    set_replication_count_isSet(false);
+    this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public String get_status() {
+    return this.status;
+  }
+
+  public void set_status(String status) {
+    this.status = status;
+  }
+
+  public void unset_status() {
+    this.status = null;
+  }
+
+  /** Returns true if field status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_status() {
+    return this.status != null;
+  }
+
+  public void set_status_isSet(boolean value) {
+    if (!value) {
+      this.status = null;
+    }
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value);
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_num_executors() {
+    return this.num_executors;
+  }
+
+  public void set_num_executors(int num_executors) {
+    this.num_executors = num_executors;
+    set_num_executors_isSet(true);
+  }
+
+  public void unset_num_executors() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  /** Returns true if field num_executors is set (has been assigned a value) and false otherwise */
+  public boolean is_set_num_executors() {
+    return EncodingUtils.testBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID);
+  }
+
+  public void set_num_executors_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_EXECUTORS_ISSET_ID, value);
+  }
+
+  public String get_topology_conf() {
+    return this.topology_conf;
+  }
+
+  public void set_topology_conf(String topology_conf) {
+    this.topology_conf = topology_conf;
+  }
+
+  public void unset_topology_conf() {
+    this.topology_conf = null;
+  }
+
+  /** Returns true if field topology_conf is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_conf() {
+    return this.topology_conf != null;
+  }
+
+  public void set_topology_conf_isSet(boolean value) {
+    if (!value) {
+      this.topology_conf = null;
+    }
+  }
+
+  public int get_id_to_spout_agg_stats_size() {
+    return (this.id_to_spout_agg_stats == null) ? 0 : this.id_to_spout_agg_stats.size();
+  }
+
+  public void put_to_id_to_spout_agg_stats(String key, ComponentAggregateStats val) {
+    if (this.id_to_spout_agg_stats == null) {
+      this.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.id_to_spout_agg_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_id_to_spout_agg_stats() {
+    return this.id_to_spout_agg_stats;
+  }
+
+  public void set_id_to_spout_agg_stats(Map<String,ComponentAggregateStats> id_to_spout_agg_stats) {
+    this.id_to_spout_agg_stats = id_to_spout_agg_stats;
+  }
+
+  public void unset_id_to_spout_agg_stats() {
+    this.id_to_spout_agg_stats = null;
+  }
+
+  /** Returns true if field id_to_spout_agg_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id_to_spout_agg_stats() {
+    return this.id_to_spout_agg_stats != null;
+  }
+
+  public void set_id_to_spout_agg_stats_isSet(boolean value) {
+    if (!value) {
+      this.id_to_spout_agg_stats = null;
+    }
+  }
+
+  public int get_id_to_bolt_agg_stats_size() {
+    return (this.id_to_bolt_agg_stats == null) ? 0 : this.id_to_bolt_agg_stats.size();
+  }
+
+  public void put_to_id_to_bolt_agg_stats(String key, ComponentAggregateStats val) {
+    if (this.id_to_bolt_agg_stats == null) {
+      this.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>();
+    }
+    this.id_to_bolt_agg_stats.put(key, val);
+  }
+
+  public Map<String,ComponentAggregateStats> get_id_to_bolt_agg_stats() {
+    return this.id_to_bolt_agg_stats;
+  }
+
+  public void set_id_to_bolt_agg_stats(Map<String,ComponentAggregateStats> id_to_bolt_agg_stats) {
+    this.id_to_bolt_agg_stats = id_to_bolt_agg_stats;
+  }
+
+  public void unset_id_to_bolt_agg_stats() {
+    this.id_to_bolt_agg_stats = null;
+  }
+
+  /** Returns true if field id_to_bolt_agg_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_id_to_bolt_agg_stats() {
+    return this.id_to_bolt_agg_stats != null;
+  }
+
+  public void set_id_to_bolt_agg_stats_isSet(boolean value) {
+    if (!value) {
+      this.id_to_bolt_agg_stats = null;
+    }
+  }
+
+  public String get_sched_status() {
+    return this.sched_status;
+  }
+
+  public void set_sched_status(String sched_status) {
+    this.sched_status = sched_status;
+  }
+
+  public void unset_sched_status() {
+    this.sched_status = null;
+  }
+
+  /** Returns true if field sched_status is set (has been assigned a value) and false otherwise */
+  public boolean is_set_sched_status() {
+    return this.sched_status != null;
+  }
+
+  public void set_sched_status_isSet(boolean value) {
+    if (!value) {
+      this.sched_status = null;
+    }
+  }
+
+  public TopologyStats get_topology_stats() {
+    return this.topology_stats;
+  }
+
+  public void set_topology_stats(TopologyStats topology_stats) {
+    this.topology_stats = topology_stats;
+  }
+
+  public void unset_topology_stats() {
+    this.topology_stats = null;
+  }
+
+  /** Returns true if field topology_stats is set (has been assigned a value) and false otherwise */
+  public boolean is_set_topology_stats() {
+    return this.topology_stats != null;
+  }
+
+  public void set_topology_stats_isSet(boolean value) {
+    if (!value) {
+      this.topology_stats = null;
+    }
+  }
+
+  public String get_owner() {
+    return this.owner;
+  }
+
+  public void set_owner(String owner) {
+    this.owner = owner;
+  }
+
+  public void unset_owner() {
+    this.owner = null;
+  }
+
+  /** Returns true if field owner is set (has been assigned a value) and false otherwise */
+  public boolean is_set_owner() {
+    return this.owner != null;
+  }
+
+  public void set_owner_isSet(boolean value) {
+    if (!value) {
+      this.owner = null;
+    }
+  }
+
+  public DebugOptions get_debug_options() {
+    return this.debug_options;
+  }
+
+  public void set_debug_options(DebugOptions debug_options) {
+    this.debug_options = debug_options;
+  }
+
+  public void unset_debug_options() {
+    this.debug_options = null;
+  }
+
+  /** Returns true if field debug_options is set (has been assigned a value) and false otherwise */
+  public boolean is_set_debug_options() {
+    return this.debug_options != null;
+  }
+
+  public void set_debug_options_isSet(boolean value) {
+    if (!value) {
+      this.debug_options = null;
+    }
+  }
+
+  public int get_replication_count() {
+    return this.replication_count;
+  }
+
+  public void set_replication_count(int replication_count) {
+    this.replication_count = replication_count;
+    set_replication_count_isSet(true);
+  }
+
+  public void unset_replication_count() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  /** Returns true if field replication_count is set (has been assigned a value) and false otherwise */
+  public boolean is_set_replication_count() {
+    return EncodingUtils.testBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID);
+  }
+
+  public void set_replication_count_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
+  }
+
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case STATUS:
+      if (value == null) {
+        unset_status();
+      } else {
+        set_status((String)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case NUM_EXECUTORS:
+      if (value == null) {
+        unset_num_executors();
+      } else {
+        set_num_executors((Integer)value);
+      }
+      break;
+
+    case TOPOLOGY_CONF:
+      if (value == null) {
+        unset_topology_conf();
+      } else {
+        set_topology_conf((String)value);
+      }
+      break;
+
+    case ID_TO_SPOUT_AGG_STATS:
+      if (value == null) {
+        unset_id_to_spout_agg_stats();
+      } else {
+        set_id_to_spout_agg_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case ID_TO_BOLT_AGG_STATS:
+      if (value == null) {
+        unset_id_to_bolt_agg_stats();
+      } else {
+        set_id_to_bolt_agg_stats((Map<String,ComponentAggregateStats>)value);
+      }
+      break;
+
+    case SCHED_STATUS:
+      if (value == null) {
+        unset_sched_status();
+      } else {
+        set_sched_status((String)value);
+      }
+      break;
+
+    case TOPOLOGY_STATS:
+      if (value == null) {
+        unset_topology_stats();
+      } else {
+        set_topology_stats((TopologyStats)value);
+      }
+      break;
+
+    case OWNER:
+      if (value == null) {
+        unset_owner();
+      } else {
+        set_owner((String)value);
+      }
+      break;
+
+    case DEBUG_OPTIONS:
+      if (value == null) {
+        unset_debug_options();
+      } else {
+        set_debug_options((DebugOptions)value);
+      }
+      break;
+
+    case REPLICATION_COUNT:
+      if (value == null) {
+        unset_replication_count();
+      } else {
+        set_replication_count((Integer)value);
+      }
+      break;
+
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case NAME:
+      return get_name();
+
+    case UPTIME_SECS:
+      return get_uptime_secs();
+
+    case STATUS:
+      return get_status();
+
+    case NUM_TASKS:
+      return get_num_tasks();
+
+    case NUM_WORKERS:
+      return get_num_workers();
+
+    case NUM_EXECUTORS:
+      return get_num_executors();
+
+    case TOPOLOGY_CONF:
+      return get_topology_conf();
+
+    case ID_TO_SPOUT_AGG_STATS:
+      return get_id_to_spout_agg_stats();
+
+    case ID_TO_BOLT_AGG_STATS:
+      return get_id_to_bolt_agg_stats();
+
+    case SCHED_STATUS:
+      return get_sched_status();
+
+    case TOPOLOGY_STATS:
+      return get_topology_stats();
+
+    case OWNER:
+      return get_owner();
+
+    case DEBUG_OPTIONS:
+      return get_debug_options();
+
+    case REPLICATION_COUNT:
+      return get_replication_count();
+
+    case REQUESTED_MEMONHEAP:
+      return get_requested_memonheap();
+
+    case REQUESTED_MEMOFFHEAP:
+      return get_requested_memoffheap();
+
+    case REQUESTED_CPU:
+      return get_requested_cpu();
+
+    case ASSIGNED_MEMONHEAP:
+      return get_assigned_memonheap();
+
+    case ASSIGNED_MEMOFFHEAP:
+      return get_assigned_memoffheap();
+
+    case ASSIGNED_CPU:
+      return get_assigned_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case NAME:
+      return is_set_name();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case STATUS:
+      return is_set_status();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case NUM_EXECUTORS:
+      return is_set_num_executors();
+    case TOPOLOGY_CONF:
+      return is_set_topology_conf();
+    case ID_TO_SPOUT_AGG_STATS:
+      return is_set_id_to_spout_agg_stats();
+    case ID_TO_BOLT_AGG_STATS:
+      return is_set_id_to_bolt_agg_stats();
+    case SCHED_STATUS:
+      return is_set_sched_status();
+    case TOPOLOGY_STATS:
+      return is_set_topology_stats();
+    case OWNER:
+      return is_set_owner();
+    case DEBUG_OPTIONS:
+      return is_set_debug_options();
+    case REPLICATION_COUNT:
+      return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyPageInfo)
+      return this.equals((TopologyPageInfo)that);
+    return false;
+  }
+
+  public boolean equals(TopologyPageInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true && this.is_set_uptime_secs();
+    boolean that_present_uptime_secs = true && that.is_set_uptime_secs();
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_status = true && this.is_set_status();
+    boolean that_present_status = true && that.is_set_status();
+    if (this_present_status || that_present_status) {
+      if (!(this_present_status && that_present_status))
+        return false;
+      if (!this.status.equals(that.status))
+        return false;
+    }
+
+    boolean this_present_num_tasks = true && this.is_set_num_tasks();
+    boolean that_present_num_tasks = true && that.is_set_num_tasks();
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_num_workers = true && this.is_set_num_workers();
+    boolean that_present_num_workers = true && that.is_set_num_workers();
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_num_executors = true && this.is_set_num_executors();
+    boolean that_present_num_executors = true && that.is_set_num_executors();
+    if (this_present_num_executors || that_present_num_executors) {
+      if (!(this_present_num_executors && that_present_num_executors))
+        return false;
+      if (this.num_executors != that.num_executors)
+        return false;
+    }
+
+    boolean this_present_topology_conf = true && this.is_set_topology_conf();
+    boolean that_present_topology_conf = true && that.is_set_topology_conf();
+    if (this_present_topology_conf || that_present_topology_conf) {
+      if (!(this_present_topology_conf && that_present_topology_conf))
+        return false;
+      if (!this.topology_conf.equals(that.topology_conf))
+        return false;
+    }
+
+    boolean this_present_id_to_spout_agg_stats = true && this.is_set_id_to_spout_agg_stats();
+    boolean that_present_id_to_spout_agg_stats = true && that.is_set_id_to_spout_agg_stats();
+    if (this_present_id_to_spout_agg_stats || that_present_id_to_spout_agg_stats) {
+      if (!(this_present_id_to_spout_agg_stats && that_present_id_to_spout_agg_stats))
+        return false;
+      if (!this.id_to_spout_agg_stats.equals(that.id_to_spout_agg_stats))
+        return false;
+    }
+
+    boolean this_present_id_to_bolt_agg_stats = true && this.is_set_id_to_bolt_agg_stats();
+    boolean that_present_id_to_bolt_agg_stats = true && that.is_set_id_to_bolt_agg_stats();
+    if (this_present_id_to_bolt_agg_stats || that_present_id_to_bolt_agg_stats) {
+      if (!(this_present_id_to_bolt_agg_stats && that_present_id_to_bolt_agg_stats))
+        return false;
+      if (!this.id_to_bolt_agg_stats.equals(that.id_to_bolt_agg_stats))
+        return false;
+    }
+
+    boolean this_present_sched_status = true && this.is_set_sched_status();
+    boolean that_present_sched_status = true && that.is_set_sched_status();
+    if (this_present_sched_status || that_present_sched_status) {
+      if (!(this_present_sched_status && that_present_sched_status))
+        return false;
+      if (!this.sched_status.equals(that.sched_status))
+        return false;
+    }
+
+    boolean this_present_topology_stats = true && this.is_set_topology_stats();
+    boolean that_present_topology_stats = true && that.is_set_topology_stats();
+    if (this_present_topology_stats || that_present_topology_stats) {
+      if (!(this_present_topology_stats && that_present_topology_stats))
+        return false;
+      if (!this.topology_stats.equals(that.topology_stats))
+        return false;
+    }
+
+    boolean this_present_owner = true && this.is_set_owner();
+    boolean that_present_owner = true && that.is_set_owner();
+    if (this_present_owner || that_present_owner) {
+      if (!(this_present_owner && that_present_owner))
+        return false;
+      if (!this.owner.equals(that.owner))
+        return false;
+    }
+
+    boolean this_present_debug_options = true && this.is_set_debug_options();
+    boolean that_present_debug_options = true && that.is_set_debug_options();
+    if (this_present_debug_options || that_present_debug_options) {
+      if (!(this_present_debug_options && that_present_debug_options))
+        return false;
+      if (!this.debug_options.equals(that.debug_options))
+        return false;
+    }
+
+    boolean this_present_replication_count = true && this.is_set_replication_count();
+    boolean that_present_replication_count = true && that.is_set_replication_count();
+    if (this_present_replication_count || that_present_replication_count) {
+      if (!(this_present_replication_count && that_present_replication_count))
+        return false;
+      if (this.replication_count != that.replication_count)
+        return false;
+    }
+
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_id = true && (is_set_id());
+    list.add(present_id);
+    if (present_id)
+      list.add(id);
+
+    boolean present_name = true && (is_set_name());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_uptime_secs = true && (is_set_uptime_secs());
+    list.add(present_uptime_secs);
+    if (present_uptime_secs)
+      list.add(uptime_secs);
+
+    boolean present_status = true && (is_set_status());
+    list.add(present_status);
+    if (present_status)
+      list.add(status);
+
+    boolean present_num_tasks = true && (is_set_num_tasks());
+    list.add(present_num_tasks);
+    if (present_num_tasks)
+      list.add(num_tasks);
+
+    boolean present_num_workers = true && (is_set_num_workers());
+    list.add(present_num_workers);
+    if (present_num_workers)
+      list.add(num_workers);
+
+    boolean present_num_executors = true && (is_set_num_executors());
+    list.add(present_num_executors);
+    if (present_num_executors)
+      list.add(num_executors);
+
+    boolean present_topology_conf = true && (is_set_topology_conf());
+    list.add(present_topology_conf);
+    if (present_topology_conf)
+      list.add(topology_conf);
+
+    boolean present_id_to_spout_agg_stats = true && (is_set_id_to_spout_agg_stats());
+    list.add(present_id_to_spout_agg_stats);
+    if (present_id_to_spout_agg_stats)
+      list.add(id_to_spout_agg_stats);
+
+    boolean present_id_to_bolt_agg_stats = true && (is_set_id_to_bolt_agg_stats());
+    list.add(present_id_to_bolt_agg_stats);
+    if (present_id_to_bolt_agg_stats)
+      list.add(id_to_bolt_agg_stats);
+
+    boolean present_sched_status = true && (is_set_sched_status());
+    list.add(present_sched_status);
+    if (present_sched_status)
+      list.add(sched_status);
+
+    boolean present_topology_stats = true && (is_set_topology_stats());
+    list.add(present_topology_stats);
+    if (present_topology_stats)
+      list.add(topology_stats);
+
+    boolean present_owner = true && (is_set_owner());
+    list.add(present_owner);
+    if (present_owner)
+      list.add(owner);
+
+    boolean present_debug_options = true && (is_set_debug_options());
+    list.add(present_debug_options);
+    if (present_debug_options)
+      list.add(debug_options);
+
+    boolean present_replication_count = true && (is_set_replication_count());
+    list.add(present_replication_count);
+    if (present_replication_count)
+      list.add(replication_count);
+
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologyPageInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_executors()).compareTo(other.is_set_num_executors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_executors()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_executors, other.num_executors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_conf()).compareTo(other.is_set_topology_conf());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_conf()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_conf, other.topology_conf);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_id_to_spout_agg_stats()).compareTo(other.is_set_id_to_spout_agg_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id_to_spout_agg_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id_to_spout_agg_stats, other.id_to_spout_agg_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_id_to_bolt_agg_stats()).compareTo(other.is_set_id_to_bolt_agg_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id_to_bolt_agg_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id_to_bolt_agg_stats, other.id_to_bolt_agg_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_sched_status()).compareTo(other.is_set_sched_status());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_sched_status()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sched_status, other.sched_status);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topology_stats()).compareTo(other.is_set_topology_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topology_stats()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_stats, other.topology_stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_owner()).compareTo(other.is_set_owner());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_owner()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_debug_options()).compareTo(other.is_set_debug_options());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_debug_options()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.debug_options, other.debug_options);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_replication_count()).compareTo(other.is_set_replication_count());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_replication_count()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.replication_count, other.replication_count);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyPageInfo(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (is_set_name()) {
+      if (!first) sb.append(", ");
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+    }
+    if (is_set_uptime_secs()) {
+      if (!first) sb.append(", ");
+      sb.append("uptime_secs:");
+      sb.append(this.uptime_secs);
+      first = false;
+    }
+    if (is_set_status()) {
+      if (!first) sb.append(", ");
+      sb.append("status:");
+      if (this.status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.status);
+      }
+      first = false;
+    }
+    if (is_set_num_tasks()) {
+      if (!first) sb.append(", ");
+      sb.append("num_tasks:");
+      sb.append(this.num_tasks);
+      first = false;
+    }
+    if (is_set_num_workers()) {
+      if (!first) sb.append(", ");
+      sb.append("num_workers:");
+      sb.append(this.num_workers);
+      first = false;
+    }
+    if (is_set_num_executors()) {
+      if (!first) sb.append(", ");
+      sb.append("num_executors:");
+      sb.append(this.num_executors);
+      first = false;
+    }
+    if (is_set_topology_conf()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_conf:");
+      if (this.topology_conf == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_conf);
+      }
+      first = false;
+    }
+    if (is_set_id_to_spout_agg_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("id_to_spout_agg_stats:");
+      if (this.id_to_spout_agg_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id_to_spout_agg_stats);
+      }
+      first = false;
+    }
+    if (is_set_id_to_bolt_agg_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("id_to_bolt_agg_stats:");
+      if (this.id_to_bolt_agg_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id_to_bolt_agg_stats);
+      }
+      first = false;
+    }
+    if (is_set_sched_status()) {
+      if (!first) sb.append(", ");
+      sb.append("sched_status:");
+      if (this.sched_status == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.sched_status);
+      }
+      first = false;
+    }
+    if (is_set_topology_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("topology_stats:");
+      if (this.topology_stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology_stats);
+      }
+      first = false;
+    }
+    if (is_set_owner()) {
+      if (!first) sb.append(", ");
+      sb.append("owner:");
+      if (this.owner == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.owner);
+      }
+      first = false;
+    }
+    if (is_set_debug_options()) {
+      if (!first) sb.append(", ");
+      sb.append("debug_options:");
+      if (this.debug_options == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.debug_options);
+      }
+      first = false;
+    }
+    if (is_set_replication_count()) {
+      if (!first) sb.append(", ");
+      sb.append("replication_count:");
+      sb.append(this.replication_count);
+      first = false;
+    }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (topology_stats != null) {
+      topology_stats.validate();
+    }
+    if (debug_options != null) {
+      debug_options.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologyPageInfoStandardSchemeFactory implements SchemeFactory {
+    public TopologyPageInfoStandardScheme getScheme() {
+      return new TopologyPageInfoStandardScheme();
+    }
+  }
+
+  private static class TopologyPageInfoStandardScheme extends StandardScheme<TopologyPageInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyPageInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.id = iprot.readString();
+              struct.set_id_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.set_name_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // UPTIME_SECS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.uptime_secs = iprot.readI32();
+              struct.set_uptime_secs_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.status = iprot.readString();
+              struct.set_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // NUM_TASKS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_tasks = iprot.readI32();
+              struct.set_num_tasks_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // NUM_WORKERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_workers = iprot.readI32();
+              struct.set_num_workers_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // NUM_EXECUTORS
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.num_executors = iprot.readI32();
+              struct.set_num_executors_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // TOPOLOGY_CONF
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.topology_conf = iprot.readString();
+              struct.set_topology_conf_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // ID_TO_SPOUT_AGG_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map402 = iprot.readMapBegin();
+                struct.id_to_spout_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map402.size);
+                String _key403;
+                ComponentAggregateStats _val404;
+                for (int _i405 = 0; _i405 < _map402.size; ++_i405)
+                {
+                  _key403 = iprot.readString();
+                  _val404 = new ComponentAggregateStats();
+                  _val404.read(iprot);
+                  struct.id_to_spout_agg_stats.put(_key403, _val404);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_id_to_spout_agg_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // ID_TO_BOLT_AGG_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map406 = iprot.readMapBegin();
+                struct.id_to_bolt_agg_stats = new HashMap<String,ComponentAggregateStats>(2*_map406.size);
+                String _key407;
+                ComponentAggregateStats _val408;
+                for (int _i409 = 0; _i409 < _map406.size; ++_i409)
+                {
+                  _key407 = iprot.readString();
+                  _val408 = new ComponentAggregateStats();
+                  _val408.read(iprot);
+                  struct.id_to_bolt_agg_stats.put(_key407, _val408);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_id_to_bolt_agg_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 11: // SCHED_STATUS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.sched_status = iprot.readString();
+              struct.set_sched_status_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 12: // TOPOLOGY_STATS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.topology_stats = new TopologyStats();
+              struct.topology_stats.read(iprot);
+              struct.set_topology_stats_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 13: // OWNER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.owner = iprot.readString();
+              struct.set_owner_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 14: // DEBUG_OPTIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.debug_options = new DebugOptions();
+              struct.debug_options.read(iprot);
+              struct.set_debug_options_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 15: // REPLICATION_COUNT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.replication_count = iprot.readI32();
+              struct.set_replication_count_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyPageInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(struct.id);
+        oprot.writeFieldEnd();
+      }
+      if (struct.name != null) {
+        if (struct.is_set_name()) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+        oprot.writeI32(struct.uptime_secs);
+        oprot.writeFieldEnd();
+      }
+      if (struct.status != null) {
+        if (struct.is_set_status()) {
+          oprot.writeFieldBegin(STATUS_FIELD_DESC);
+          oprot.writeString(struct.status);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+        oprot.writeI32(struct.num_tasks);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_workers()) {
+        oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+        oprot.writeI32(struct.num_workers);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeFieldBegin(NUM_EXECUTORS_FIELD_DESC);
+        oprot.writeI32(struct.num_executors);
+        oprot.writeFieldEnd();
+      }
+      if (struct.topology_conf != null) {
+        if (struct.is_set_topology_conf()) {
+          oprot.writeFieldBegin(TOPOLOGY_CONF_FIELD_DESC);
+          oprot.writeString(struct.topology_conf);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.id_to_spout_agg_stats != null) {
+        if (struct.is_set_id_to_spout_agg_stats()) {
+          oprot.writeFieldBegin(ID_TO_SPOUT_AGG_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.id_to_spout_agg_stats.size()));
+            for (Map.Entry<String, ComponentAggregateStats> _iter410 : struct.id_to_spout_agg_stats.entrySet())
+            {
+              oprot.writeString(_iter410.getKey());
+              _iter410.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.id_to_bolt_agg_stats != null) {
+        if (struct.is_set_id_to_bolt_agg_stats()) {
+          oprot.writeFieldBegin(ID_TO_BOLT_AGG_STATS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.id_to_bolt_agg_stats.size()));
+            for (Map.Entry<String, ComponentAggregateStats> _iter411 : struct.id_to_bolt_agg_stats.entrySet())
+            {
+              oprot.writeString(_iter411.getKey());
+              _iter411.getValue().write(oprot);
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.sched_status != null) {
+        if (struct.is_set_sched_status()) {
+          oprot.writeFieldBegin(SCHED_STATUS_FIELD_DESC);
+          oprot.writeString(struct.sched_status);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.topology_stats != null) {
+        if (struct.is_set_topology_stats()) {
+          oprot.writeFieldBegin(TOPOLOGY_STATS_FIELD_DESC);
+          struct.topology_stats.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.owner != null) {
+        if (struct.is_set_owner()) {
+          oprot.writeFieldBegin(OWNER_FIELD_DESC);
+          oprot.writeString(struct.owner);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.debug_options != null) {
+        if (struct.is_set_debug_options()) {
+          oprot.writeFieldBegin(DEBUG_OPTIONS_FIELD_DESC);
+          struct.debug_options.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.is_set_replication_count()) {
+        oprot.writeFieldBegin(REPLICATION_COUNT_FIELD_DESC);
+        oprot.writeI32(struct.replication_count);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TopologyPageInfoTupleSchemeFactory implements SchemeFactory {
+    public TopologyPageInfoTupleScheme getScheme() {
+      return new TopologyPageInfoTupleScheme();
+    }
+  }
+
+  private static class TopologyPageInfoTupleScheme extends TupleScheme<TopologyPageInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TopologyPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.id);
+      BitSet optionals = new BitSet();
+      if (struct.is_set_name()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_uptime_secs()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_status()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_num_tasks()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_num_workers()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_num_executors()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_topology_conf()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_id_to_spout_agg_stats()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_id_to_bolt_agg_stats()) {
+        optionals.set(8);
+      }
+      if (struct.is_set_sched_status()) {
+        optionals.set(9);
+      }
+      if (struct.is_set_topology_stats()) {
+        optionals.set(10);
+      }
+      if (struct.is_set_owner()) {
+        optionals.set(11);
+      }
+      if (struct.is_set_debug_options()) {
+        optionals.set(12);
+      }
+      if (struct.is_set_replication_count()) {
+        optionals.set(13);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(14);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(15);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(16);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(17);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(18);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(19);
+      }
+      oprot.writeBitSet(optionals, 20);
+      if (struct.is_set_name()) {
+        oprot.writeString(struct.name);
+      }
+      if (struct.is_set_uptime_secs()) {
+        oprot.writeI32(struct.uptime_secs);
+      }
+      if (struct.is_set_status()) {
+        oprot.writeString(struct.status);
+      }
+      if (struct.is_set_num_tasks()) {
+        oprot.writeI32(struct.num_tasks);
+      }
+      if (struct.is_set_num_workers()) {
+        oprot.writeI32(struct.num_workers);
+      }
+      if (struct.is_set_num_executors()) {
+        oprot.writeI32(struct.num_executors);
+      }
+      if (struct.is_set_topology_conf()) {
+        oprot.writeString(struct.topology_conf);
+      }
+      if (struct.is_set_id_to_spout_agg_stats()) {
+        {
+          oprot.writeI32(struct.id_to_spout_agg_stats.size());
+          for (Map.Entry<String, ComponentAggregateStats> _iter412 : struct.id_to_spout_agg_stats.entrySet())
+          {
+            oprot.writeString(_iter412.getKey());
+            _iter412.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> _iter413 : struct.id_to_bolt_agg_stats.entrySet())
+          {
+            oprot.writeString(_iter413.getKey());
+            _iter413.getValue().write(oprot);
+          }
+        }
+      }
+      if (struct.is_set_sched_status()) {
+        oprot.writeString(struct.sched_status);
+      }
+      if (struct.is_set_topology_stats()) {
+        struct.topology_stats.write(oprot);
+      }
+      if (struct.is_set_owner()) {
+        oprot.writeString(struct.owner);
+      }
+      if (struct.is_set_debug_options()) {
+        struct.debug_options.write(oprot);
+      }
+      if (struct.is_set_replication_count()) {
+        oprot.writeI32(struct.replication_count);
+      }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TopologyPageInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.id = iprot.readString();
+      struct.set_id_isSet(true);
+      BitSet incoming = iprot.readBitSet(20);
+      if (incoming.get(0)) {
+        struct.name = iprot.readString();
+        struct.set_name_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.uptime_secs = iprot.readI32();
+        struct.set_uptime_secs_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.status = iprot.readString();
+        struct.set_status_isSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.num_tasks = iprot.readI32();
+        struct.set_num_tasks_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.num_workers = iprot.readI32();
+        struct.set_num_workers_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.num_executors = iprot.readI32();
+        struct.set_num_executors_isSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.topology_conf = iprot.readString();
+        struct.set_topology_conf_isSet(true);
+      }
+      if (incoming.get(7)) {
+        {
+          org.apache.thrift.protocol.TMap _map414 = 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*_map414.size);
+          String _key415;
+          ComponentAggregateStats _val416;
+          for (int _i417 = 0; _i417 < _map414.size; ++_i417)
+          {
+            _key415 = iprot.readString();
+            _val416 = new ComponentAggregateStats();
+            _val416.read(iprot);
+            struct.id_to_spout_agg_stats.put(_key415, _val416);
+          }
+        }
+        struct.set_id_to_spout_agg_stats_isSet(true);
+      }
+      if (incoming.get(8)) {
+        {
+          org.apache.thrift.protocol.TMap _map418 = 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*_map418.size);
+          String _key419;
+          ComponentAggregateStats _val420;
+          for (int _i421 = 0; _i421 < _map418.size; ++_i421)
+          {
+            _key419 = iprot.readString();
+            _val420 = new ComponentAggregateStats();
+            _val420.read(iprot);
+            struct.id_to_bolt_agg_stats.put(_key419, _val420);
+          }
+        }
+        struct.set_id_to_bolt_agg_stats_isSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.sched_status = iprot.readString();
+        struct.set_sched_status_isSet(true);
+      }
+      if (incoming.get(10)) {
+        struct.topology_stats = new TopologyStats();
+        struct.topology_stats.read(iprot);
+        struct.set_topology_stats_isSet(true);
+      }
+      if (incoming.get(11)) {
+        struct.owner = iprot.readString();
+        struct.set_owner_isSet(true);
+      }
+      if (incoming.get(12)) {
+        struct.debug_options = new DebugOptions();
+        struct.debug_options.read(iprot);
+        struct.set_debug_options_isSet(true);
+      }
+      if (incoming.get(13)) {
+        struct.replication_count = iprot.readI32();
+        struct.set_replication_count_isSet(true);
+      }
+      if (incoming.get(14)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(15)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(16)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(17)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(18)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(19)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
new file mode 100644
index 0000000..aa598e4
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyStats.java
@@ -0,0 +1,1094 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+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");
+
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_EMITTED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_emitted", org.apache.thrift.protocol.TType.MAP, (short)1);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_TRANSFERRED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_transferred", org.apache.thrift.protocol.TType.MAP, (short)2);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_COMPLETE_LATENCIES_MS_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_complete_latencies_ms", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_ACKED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_acked", org.apache.thrift.protocol.TType.MAP, (short)4);
+  private static final org.apache.thrift.protocol.TField WINDOW_TO_FAILED_FIELD_DESC = new org.apache.thrift.protocol.TField("window_to_failed", org.apache.thrift.protocol.TType.MAP, (short)5);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TopologyStatsStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TopologyStatsTupleSchemeFactory());
+  }
+
+  private Map<String,Long> window_to_emitted; // optional
+  private Map<String,Long> window_to_transferred; // optional
+  private Map<String,Double> window_to_complete_latencies_ms; // optional
+  private Map<String,Long> window_to_acked; // optional
+  private Map<String,Long> window_to_failed; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    WINDOW_TO_EMITTED((short)1, "window_to_emitted"),
+    WINDOW_TO_TRANSFERRED((short)2, "window_to_transferred"),
+    WINDOW_TO_COMPLETE_LATENCIES_MS((short)3, "window_to_complete_latencies_ms"),
+    WINDOW_TO_ACKED((short)4, "window_to_acked"),
+    WINDOW_TO_FAILED((short)5, "window_to_failed");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // WINDOW_TO_EMITTED
+          return WINDOW_TO_EMITTED;
+        case 2: // WINDOW_TO_TRANSFERRED
+          return WINDOW_TO_TRANSFERRED;
+        case 3: // WINDOW_TO_COMPLETE_LATENCIES_MS
+          return WINDOW_TO_COMPLETE_LATENCIES_MS;
+        case 4: // WINDOW_TO_ACKED
+          return WINDOW_TO_ACKED;
+        case 5: // WINDOW_TO_FAILED
+          return WINDOW_TO_FAILED;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.WINDOW_TO_EMITTED,_Fields.WINDOW_TO_TRANSFERRED,_Fields.WINDOW_TO_COMPLETE_LATENCIES_MS,_Fields.WINDOW_TO_ACKED,_Fields.WINDOW_TO_FAILED};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.WINDOW_TO_EMITTED, new org.apache.thrift.meta_data.FieldMetaData("window_to_emitted", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WINDOW_TO_TRANSFERRED, new org.apache.thrift.meta_data.FieldMetaData("window_to_transferred", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WINDOW_TO_COMPLETE_LATENCIES_MS, new org.apache.thrift.meta_data.FieldMetaData("window_to_complete_latencies_ms", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))));
+    tmpMap.put(_Fields.WINDOW_TO_ACKED, new org.apache.thrift.meta_data.FieldMetaData("window_to_acked", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    tmpMap.put(_Fields.WINDOW_TO_FAILED, new org.apache.thrift.meta_data.FieldMetaData("window_to_failed", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyStats.class, metaDataMap);
+  }
+
+  public TopologyStats() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyStats(TopologyStats other) {
+    if (other.is_set_window_to_emitted()) {
+      Map<String,Long> __this__window_to_emitted = new HashMap<String,Long>(other.window_to_emitted);
+      this.window_to_emitted = __this__window_to_emitted;
+    }
+    if (other.is_set_window_to_transferred()) {
+      Map<String,Long> __this__window_to_transferred = new HashMap<String,Long>(other.window_to_transferred);
+      this.window_to_transferred = __this__window_to_transferred;
+    }
+    if (other.is_set_window_to_complete_latencies_ms()) {
+      Map<String,Double> __this__window_to_complete_latencies_ms = new HashMap<String,Double>(other.window_to_complete_latencies_ms);
+      this.window_to_complete_latencies_ms = __this__window_to_complete_latencies_ms;
+    }
+    if (other.is_set_window_to_acked()) {
+      Map<String,Long> __this__window_to_acked = new HashMap<String,Long>(other.window_to_acked);
+      this.window_to_acked = __this__window_to_acked;
+    }
+    if (other.is_set_window_to_failed()) {
+      Map<String,Long> __this__window_to_failed = new HashMap<String,Long>(other.window_to_failed);
+      this.window_to_failed = __this__window_to_failed;
+    }
+  }
+
+  public TopologyStats deepCopy() {
+    return new TopologyStats(this);
+  }
+
+  @Override
+  public void clear() {
+    this.window_to_emitted = null;
+    this.window_to_transferred = null;
+    this.window_to_complete_latencies_ms = null;
+    this.window_to_acked = null;
+    this.window_to_failed = null;
+  }
+
+  public int get_window_to_emitted_size() {
+    return (this.window_to_emitted == null) ? 0 : this.window_to_emitted.size();
+  }
+
+  public void put_to_window_to_emitted(String key, long val) {
+    if (this.window_to_emitted == null) {
+      this.window_to_emitted = new HashMap<String,Long>();
+    }
+    this.window_to_emitted.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_emitted() {
+    return this.window_to_emitted;
+  }
+
+  public void set_window_to_emitted(Map<String,Long> window_to_emitted) {
+    this.window_to_emitted = window_to_emitted;
+  }
+
+  public void unset_window_to_emitted() {
+    this.window_to_emitted = null;
+  }
+
+  /** Returns true if field window_to_emitted is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_emitted() {
+    return this.window_to_emitted != null;
+  }
+
+  public void set_window_to_emitted_isSet(boolean value) {
+    if (!value) {
+      this.window_to_emitted = null;
+    }
+  }
+
+  public int get_window_to_transferred_size() {
+    return (this.window_to_transferred == null) ? 0 : this.window_to_transferred.size();
+  }
+
+  public void put_to_window_to_transferred(String key, long val) {
+    if (this.window_to_transferred == null) {
+      this.window_to_transferred = new HashMap<String,Long>();
+    }
+    this.window_to_transferred.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_transferred() {
+    return this.window_to_transferred;
+  }
+
+  public void set_window_to_transferred(Map<String,Long> window_to_transferred) {
+    this.window_to_transferred = window_to_transferred;
+  }
+
+  public void unset_window_to_transferred() {
+    this.window_to_transferred = null;
+  }
+
+  /** Returns true if field window_to_transferred is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_transferred() {
+    return this.window_to_transferred != null;
+  }
+
+  public void set_window_to_transferred_isSet(boolean value) {
+    if (!value) {
+      this.window_to_transferred = null;
+    }
+  }
+
+  public int get_window_to_complete_latencies_ms_size() {
+    return (this.window_to_complete_latencies_ms == null) ? 0 : this.window_to_complete_latencies_ms.size();
+  }
+
+  public void put_to_window_to_complete_latencies_ms(String key, double val) {
+    if (this.window_to_complete_latencies_ms == null) {
+      this.window_to_complete_latencies_ms = new HashMap<String,Double>();
+    }
+    this.window_to_complete_latencies_ms.put(key, val);
+  }
+
+  public Map<String,Double> get_window_to_complete_latencies_ms() {
+    return this.window_to_complete_latencies_ms;
+  }
+
+  public void set_window_to_complete_latencies_ms(Map<String,Double> window_to_complete_latencies_ms) {
+    this.window_to_complete_latencies_ms = window_to_complete_latencies_ms;
+  }
+
+  public void unset_window_to_complete_latencies_ms() {
+    this.window_to_complete_latencies_ms = null;
+  }
+
+  /** Returns true if field window_to_complete_latencies_ms is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_complete_latencies_ms() {
+    return this.window_to_complete_latencies_ms != null;
+  }
+
+  public void set_window_to_complete_latencies_ms_isSet(boolean value) {
+    if (!value) {
+      this.window_to_complete_latencies_ms = null;
+    }
+  }
+
+  public int get_window_to_acked_size() {
+    return (this.window_to_acked == null) ? 0 : this.window_to_acked.size();
+  }
+
+  public void put_to_window_to_acked(String key, long val) {
+    if (this.window_to_acked == null) {
+      this.window_to_acked = new HashMap<String,Long>();
+    }
+    this.window_to_acked.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_acked() {
+    return this.window_to_acked;
+  }
+
+  public void set_window_to_acked(Map<String,Long> window_to_acked) {
+    this.window_to_acked = window_to_acked;
+  }
+
+  public void unset_window_to_acked() {
+    this.window_to_acked = null;
+  }
+
+  /** Returns true if field window_to_acked is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_acked() {
+    return this.window_to_acked != null;
+  }
+
+  public void set_window_to_acked_isSet(boolean value) {
+    if (!value) {
+      this.window_to_acked = null;
+    }
+  }
+
+  public int get_window_to_failed_size() {
+    return (this.window_to_failed == null) ? 0 : this.window_to_failed.size();
+  }
+
+  public void put_to_window_to_failed(String key, long val) {
+    if (this.window_to_failed == null) {
+      this.window_to_failed = new HashMap<String,Long>();
+    }
+    this.window_to_failed.put(key, val);
+  }
+
+  public Map<String,Long> get_window_to_failed() {
+    return this.window_to_failed;
+  }
+
+  public void set_window_to_failed(Map<String,Long> window_to_failed) {
+    this.window_to_failed = window_to_failed;
+  }
+
+  public void unset_window_to_failed() {
+    this.window_to_failed = null;
+  }
+
+  /** Returns true if field window_to_failed is set (has been assigned a value) and false otherwise */
+  public boolean is_set_window_to_failed() {
+    return this.window_to_failed != null;
+  }
+
+  public void set_window_to_failed_isSet(boolean value) {
+    if (!value) {
+      this.window_to_failed = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case WINDOW_TO_EMITTED:
+      if (value == null) {
+        unset_window_to_emitted();
+      } else {
+        set_window_to_emitted((Map<String,Long>)value);
+      }
+      break;
+
+    case WINDOW_TO_TRANSFERRED:
+      if (value == null) {
+        unset_window_to_transferred();
+      } else {
+        set_window_to_transferred((Map<String,Long>)value);
+      }
+      break;
+
+    case WINDOW_TO_COMPLETE_LATENCIES_MS:
+      if (value == null) {
+        unset_window_to_complete_latencies_ms();
+      } else {
+        set_window_to_complete_latencies_ms((Map<String,Double>)value);
+      }
+      break;
+
+    case WINDOW_TO_ACKED:
+      if (value == null) {
+        unset_window_to_acked();
+      } else {
+        set_window_to_acked((Map<String,Long>)value);
+      }
+      break;
+
+    case WINDOW_TO_FAILED:
+      if (value == null) {
+        unset_window_to_failed();
+      } else {
+        set_window_to_failed((Map<String,Long>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case WINDOW_TO_EMITTED:
+      return get_window_to_emitted();
+
+    case WINDOW_TO_TRANSFERRED:
+      return get_window_to_transferred();
+
+    case WINDOW_TO_COMPLETE_LATENCIES_MS:
+      return get_window_to_complete_latencies_ms();
+
+    case WINDOW_TO_ACKED:
+      return get_window_to_acked();
+
+    case WINDOW_TO_FAILED:
+      return get_window_to_failed();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case WINDOW_TO_EMITTED:
+      return is_set_window_to_emitted();
+    case WINDOW_TO_TRANSFERRED:
+      return is_set_window_to_transferred();
+    case WINDOW_TO_COMPLETE_LATENCIES_MS:
+      return is_set_window_to_complete_latencies_ms();
+    case WINDOW_TO_ACKED:
+      return is_set_window_to_acked();
+    case WINDOW_TO_FAILED:
+      return is_set_window_to_failed();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyStats)
+      return this.equals((TopologyStats)that);
+    return false;
+  }
+
+  public boolean equals(TopologyStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_window_to_emitted = true && this.is_set_window_to_emitted();
+    boolean that_present_window_to_emitted = true && that.is_set_window_to_emitted();
+    if (this_present_window_to_emitted || that_present_window_to_emitted) {
+      if (!(this_present_window_to_emitted && that_present_window_to_emitted))
+        return false;
+      if (!this.window_to_emitted.equals(that.window_to_emitted))
+        return false;
+    }
+
+    boolean this_present_window_to_transferred = true && this.is_set_window_to_transferred();
+    boolean that_present_window_to_transferred = true && that.is_set_window_to_transferred();
+    if (this_present_window_to_transferred || that_present_window_to_transferred) {
+      if (!(this_present_window_to_transferred && that_present_window_to_transferred))
+        return false;
+      if (!this.window_to_transferred.equals(that.window_to_transferred))
+        return false;
+    }
+
+    boolean this_present_window_to_complete_latencies_ms = true && this.is_set_window_to_complete_latencies_ms();
+    boolean that_present_window_to_complete_latencies_ms = true && that.is_set_window_to_complete_latencies_ms();
+    if (this_present_window_to_complete_latencies_ms || that_present_window_to_complete_latencies_ms) {
+      if (!(this_present_window_to_complete_latencies_ms && that_present_window_to_complete_latencies_ms))
+        return false;
+      if (!this.window_to_complete_latencies_ms.equals(that.window_to_complete_latencies_ms))
+        return false;
+    }
+
+    boolean this_present_window_to_acked = true && this.is_set_window_to_acked();
+    boolean that_present_window_to_acked = true && that.is_set_window_to_acked();
+    if (this_present_window_to_acked || that_present_window_to_acked) {
+      if (!(this_present_window_to_acked && that_present_window_to_acked))
+        return false;
+      if (!this.window_to_acked.equals(that.window_to_acked))
+        return false;
+    }
+
+    boolean this_present_window_to_failed = true && this.is_set_window_to_failed();
+    boolean that_present_window_to_failed = true && that.is_set_window_to_failed();
+    if (this_present_window_to_failed || that_present_window_to_failed) {
+      if (!(this_present_window_to_failed && that_present_window_to_failed))
+        return false;
+      if (!this.window_to_failed.equals(that.window_to_failed))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_window_to_emitted = true && (is_set_window_to_emitted());
+    list.add(present_window_to_emitted);
+    if (present_window_to_emitted)
+      list.add(window_to_emitted);
+
+    boolean present_window_to_transferred = true && (is_set_window_to_transferred());
+    list.add(present_window_to_transferred);
+    if (present_window_to_transferred)
+      list.add(window_to_transferred);
+
+    boolean present_window_to_complete_latencies_ms = true && (is_set_window_to_complete_latencies_ms());
+    list.add(present_window_to_complete_latencies_ms);
+    if (present_window_to_complete_latencies_ms)
+      list.add(window_to_complete_latencies_ms);
+
+    boolean present_window_to_acked = true && (is_set_window_to_acked());
+    list.add(present_window_to_acked);
+    if (present_window_to_acked)
+      list.add(window_to_acked);
+
+    boolean present_window_to_failed = true && (is_set_window_to_failed());
+    list.add(present_window_to_failed);
+    if (present_window_to_failed)
+      list.add(window_to_failed);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(TopologyStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_window_to_emitted()).compareTo(other.is_set_window_to_emitted());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_emitted()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_emitted, other.window_to_emitted);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_transferred()).compareTo(other.is_set_window_to_transferred());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_transferred()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_transferred, other.window_to_transferred);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_complete_latencies_ms()).compareTo(other.is_set_window_to_complete_latencies_ms());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_complete_latencies_ms()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_complete_latencies_ms, other.window_to_complete_latencies_ms);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_acked()).compareTo(other.is_set_window_to_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_acked()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_acked, other.window_to_acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_window_to_failed()).compareTo(other.is_set_window_to_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_window_to_failed()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.window_to_failed, other.window_to_failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyStats(");
+    boolean first = true;
+
+    if (is_set_window_to_emitted()) {
+      sb.append("window_to_emitted:");
+      if (this.window_to_emitted == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_emitted);
+      }
+      first = false;
+    }
+    if (is_set_window_to_transferred()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_transferred:");
+      if (this.window_to_transferred == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_transferred);
+      }
+      first = false;
+    }
+    if (is_set_window_to_complete_latencies_ms()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_complete_latencies_ms:");
+      if (this.window_to_complete_latencies_ms == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_complete_latencies_ms);
+      }
+      first = false;
+    }
+    if (is_set_window_to_acked()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_acked:");
+      if (this.window_to_acked == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_acked);
+      }
+      first = false;
+    }
+    if (is_set_window_to_failed()) {
+      if (!first) sb.append(", ");
+      sb.append("window_to_failed:");
+      if (this.window_to_failed == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.window_to_failed);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TopologyStatsStandardSchemeFactory implements SchemeFactory {
+    public TopologyStatsStandardScheme getScheme() {
+      return new TopologyStatsStandardScheme();
+    }
+  }
+
+  private static class TopologyStatsStandardScheme extends StandardScheme<TopologyStats> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyStats struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // WINDOW_TO_EMITTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map352 = iprot.readMapBegin();
+                struct.window_to_emitted = new HashMap<String,Long>(2*_map352.size);
+                String _key353;
+                long _val354;
+                for (int _i355 = 0; _i355 < _map352.size; ++_i355)
+                {
+                  _key353 = iprot.readString();
+                  _val354 = iprot.readI64();
+                  struct.window_to_emitted.put(_key353, _val354);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_emitted_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // WINDOW_TO_TRANSFERRED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map356 = iprot.readMapBegin();
+                struct.window_to_transferred = new HashMap<String,Long>(2*_map356.size);
+                String _key357;
+                long _val358;
+                for (int _i359 = 0; _i359 < _map356.size; ++_i359)
+                {
+                  _key357 = iprot.readString();
+                  _val358 = iprot.readI64();
+                  struct.window_to_transferred.put(_key357, _val358);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_transferred_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // WINDOW_TO_COMPLETE_LATENCIES_MS
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map360 = iprot.readMapBegin();
+                struct.window_to_complete_latencies_ms = new HashMap<String,Double>(2*_map360.size);
+                String _key361;
+                double _val362;
+                for (int _i363 = 0; _i363 < _map360.size; ++_i363)
+                {
+                  _key361 = iprot.readString();
+                  _val362 = iprot.readDouble();
+                  struct.window_to_complete_latencies_ms.put(_key361, _val362);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_complete_latencies_ms_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // WINDOW_TO_ACKED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map364 = iprot.readMapBegin();
+                struct.window_to_acked = new HashMap<String,Long>(2*_map364.size);
+                String _key365;
+                long _val366;
+                for (int _i367 = 0; _i367 < _map364.size; ++_i367)
+                {
+                  _key365 = iprot.readString();
+                  _val366 = iprot.readI64();
+                  struct.window_to_acked.put(_key365, _val366);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_acked_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // WINDOW_TO_FAILED
+            if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
+              {
+                org.apache.thrift.protocol.TMap _map368 = iprot.readMapBegin();
+                struct.window_to_failed = new HashMap<String,Long>(2*_map368.size);
+                String _key369;
+                long _val370;
+                for (int _i371 = 0; _i371 < _map368.size; ++_i371)
+                {
+                  _key369 = iprot.readString();
+                  _val370 = iprot.readI64();
+                  struct.window_to_failed.put(_key369, _val370);
+                }
+                iprot.readMapEnd();
+              }
+              struct.set_window_to_failed_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyStats struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.window_to_emitted != null) {
+        if (struct.is_set_window_to_emitted()) {
+          oprot.writeFieldBegin(WINDOW_TO_EMITTED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_emitted.size()));
+            for (Map.Entry<String, Long> _iter372 : struct.window_to_emitted.entrySet())
+            {
+              oprot.writeString(_iter372.getKey());
+              oprot.writeI64(_iter372.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_transferred != null) {
+        if (struct.is_set_window_to_transferred()) {
+          oprot.writeFieldBegin(WINDOW_TO_TRANSFERRED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_transferred.size()));
+            for (Map.Entry<String, Long> _iter373 : struct.window_to_transferred.entrySet())
+            {
+              oprot.writeString(_iter373.getKey());
+              oprot.writeI64(_iter373.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_complete_latencies_ms != null) {
+        if (struct.is_set_window_to_complete_latencies_ms()) {
+          oprot.writeFieldBegin(WINDOW_TO_COMPLETE_LATENCIES_MS_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.DOUBLE, struct.window_to_complete_latencies_ms.size()));
+            for (Map.Entry<String, Double> _iter374 : struct.window_to_complete_latencies_ms.entrySet())
+            {
+              oprot.writeString(_iter374.getKey());
+              oprot.writeDouble(_iter374.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_acked != null) {
+        if (struct.is_set_window_to_acked()) {
+          oprot.writeFieldBegin(WINDOW_TO_ACKED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_acked.size()));
+            for (Map.Entry<String, Long> _iter375 : struct.window_to_acked.entrySet())
+            {
+              oprot.writeString(_iter375.getKey());
+              oprot.writeI64(_iter375.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.window_to_failed != null) {
+        if (struct.is_set_window_to_failed()) {
+          oprot.writeFieldBegin(WINDOW_TO_FAILED_FIELD_DESC);
+          {
+            oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I64, struct.window_to_failed.size()));
+            for (Map.Entry<String, Long> _iter376 : struct.window_to_failed.entrySet())
+            {
+              oprot.writeString(_iter376.getKey());
+              oprot.writeI64(_iter376.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TopologyStatsTupleSchemeFactory implements SchemeFactory {
+    public TopologyStatsTupleScheme getScheme() {
+      return new TopologyStatsTupleScheme();
+    }
+  }
+
+  private static class TopologyStatsTupleScheme extends TupleScheme<TopologyStats> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TopologyStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_window_to_emitted()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_window_to_transferred()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_window_to_complete_latencies_ms()) {
+        optionals.set(2);
+      }
+      if (struct.is_set_window_to_acked()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_window_to_failed()) {
+        optionals.set(4);
+      }
+      oprot.writeBitSet(optionals, 5);
+      if (struct.is_set_window_to_emitted()) {
+        {
+          oprot.writeI32(struct.window_to_emitted.size());
+          for (Map.Entry<String, Long> _iter377 : struct.window_to_emitted.entrySet())
+          {
+            oprot.writeString(_iter377.getKey());
+            oprot.writeI64(_iter377.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_transferred()) {
+        {
+          oprot.writeI32(struct.window_to_transferred.size());
+          for (Map.Entry<String, Long> _iter378 : struct.window_to_transferred.entrySet())
+          {
+            oprot.writeString(_iter378.getKey());
+            oprot.writeI64(_iter378.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_complete_latencies_ms()) {
+        {
+          oprot.writeI32(struct.window_to_complete_latencies_ms.size());
+          for (Map.Entry<String, Double> _iter379 : struct.window_to_complete_latencies_ms.entrySet())
+          {
+            oprot.writeString(_iter379.getKey());
+            oprot.writeDouble(_iter379.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_acked()) {
+        {
+          oprot.writeI32(struct.window_to_acked.size());
+          for (Map.Entry<String, Long> _iter380 : struct.window_to_acked.entrySet())
+          {
+            oprot.writeString(_iter380.getKey());
+            oprot.writeI64(_iter380.getValue());
+          }
+        }
+      }
+      if (struct.is_set_window_to_failed()) {
+        {
+          oprot.writeI32(struct.window_to_failed.size());
+          for (Map.Entry<String, Long> _iter381 : struct.window_to_failed.entrySet())
+          {
+            oprot.writeString(_iter381.getKey());
+            oprot.writeI64(_iter381.getValue());
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TopologyStats struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(5);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TMap _map382 = 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*_map382.size);
+          String _key383;
+          long _val384;
+          for (int _i385 = 0; _i385 < _map382.size; ++_i385)
+          {
+            _key383 = iprot.readString();
+            _val384 = iprot.readI64();
+            struct.window_to_emitted.put(_key383, _val384);
+          }
+        }
+        struct.set_window_to_emitted_isSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          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_transferred = new HashMap<String,Long>(2*_map386.size);
+          String _key387;
+          long _val388;
+          for (int _i389 = 0; _i389 < _map386.size; ++_i389)
+          {
+            _key387 = iprot.readString();
+            _val388 = iprot.readI64();
+            struct.window_to_transferred.put(_key387, _val388);
+          }
+        }
+        struct.set_window_to_transferred_isSet(true);
+      }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TMap _map390 = 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*_map390.size);
+          String _key391;
+          double _val392;
+          for (int _i393 = 0; _i393 < _map390.size; ++_i393)
+          {
+            _key391 = iprot.readString();
+            _val392 = iprot.readDouble();
+            struct.window_to_complete_latencies_ms.put(_key391, _val392);
+          }
+        }
+        struct.set_window_to_complete_latencies_ms_isSet(true);
+      }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TMap _map394 = 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*_map394.size);
+          String _key395;
+          long _val396;
+          for (int _i397 = 0; _i397 < _map394.size; ++_i397)
+          {
+            _key395 = iprot.readString();
+            _val396 = iprot.readI64();
+            struct.window_to_acked.put(_key395, _val396);
+          }
+        }
+        struct.set_window_to_acked_isSet(true);
+      }
+      if (incoming.get(4)) {
+        {
+          org.apache.thrift.protocol.TMap _map398 = 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*_map398.size);
+          String _key399;
+          long _val400;
+          for (int _i401 = 0; _i401 < _map398.size; ++_i401)
+          {
+            _key399 = iprot.readString();
+            _val400 = iprot.readI64();
+            struct.window_to_failed.put(_key399, _val400);
+          }
+        }
+        struct.set_window_to_failed_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java b/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java
index 8183ab2..3866a8c 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologyStatus.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
diff --git a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
index 4e6ff61..5387e62 100644
--- a/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
+++ b/storm-core/src/jvm/backtype/storm/generated/TopologySummary.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 /**
- * Autogenerated by Thrift Compiler (0.9.2)
+ * Autogenerated by Thrift Compiler (0.9.3)
  *
  * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  *  @generated
@@ -51,7 +51,7 @@
 import org.slf4j.LoggerFactory;
 
 @SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-4-17")
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 public class TopologySummary implements org.apache.thrift.TBase<TopologySummary, TopologySummary._Fields>, java.io.Serializable, Cloneable, Comparable<TopologySummary> {
   private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologySummary");
 
@@ -65,6 +65,12 @@
   private static final org.apache.thrift.protocol.TField SCHED_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("sched_status", org.apache.thrift.protocol.TType.STRING, (short)513);
   private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)514);
   private static final org.apache.thrift.protocol.TField REPLICATION_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("replication_count", org.apache.thrift.protocol.TType.I32, (short)515);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)521);
+  private static final org.apache.thrift.protocol.TField REQUESTED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)522);
+  private static final org.apache.thrift.protocol.TField REQUESTED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("requested_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)523);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMONHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memonheap", org.apache.thrift.protocol.TType.DOUBLE, (short)524);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_MEMOFFHEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_memoffheap", org.apache.thrift.protocol.TType.DOUBLE, (short)525);
+  private static final org.apache.thrift.protocol.TField ASSIGNED_CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("assigned_cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)526);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -82,6 +88,12 @@
   private String sched_status; // optional
   private String owner; // optional
   private int replication_count; // optional
+  private double requested_memonheap; // optional
+  private double requested_memoffheap; // optional
+  private double requested_cpu; // optional
+  private double assigned_memonheap; // optional
+  private double assigned_memoffheap; // optional
+  private double assigned_cpu; // 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 {
@@ -94,7 +106,13 @@
     STATUS((short)7, "status"),
     SCHED_STATUS((short)513, "sched_status"),
     OWNER((short)514, "owner"),
-    REPLICATION_COUNT((short)515, "replication_count");
+    REPLICATION_COUNT((short)515, "replication_count"),
+    REQUESTED_MEMONHEAP((short)521, "requested_memonheap"),
+    REQUESTED_MEMOFFHEAP((short)522, "requested_memoffheap"),
+    REQUESTED_CPU((short)523, "requested_cpu"),
+    ASSIGNED_MEMONHEAP((short)524, "assigned_memonheap"),
+    ASSIGNED_MEMOFFHEAP((short)525, "assigned_memoffheap"),
+    ASSIGNED_CPU((short)526, "assigned_cpu");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -129,6 +147,18 @@
           return OWNER;
         case 515: // REPLICATION_COUNT
           return REPLICATION_COUNT;
+        case 521: // REQUESTED_MEMONHEAP
+          return REQUESTED_MEMONHEAP;
+        case 522: // REQUESTED_MEMOFFHEAP
+          return REQUESTED_MEMOFFHEAP;
+        case 523: // REQUESTED_CPU
+          return REQUESTED_CPU;
+        case 524: // ASSIGNED_MEMONHEAP
+          return ASSIGNED_MEMONHEAP;
+        case 525: // ASSIGNED_MEMOFFHEAP
+          return ASSIGNED_MEMOFFHEAP;
+        case 526: // ASSIGNED_CPU
+          return ASSIGNED_CPU;
         default:
           return null;
       }
@@ -174,8 +204,14 @@
   private static final int __NUM_WORKERS_ISSET_ID = 2;
   private static final int __UPTIME_SECS_ISSET_ID = 3;
   private static final int __REPLICATION_COUNT_ISSET_ID = 4;
-  private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT};
+  private static final int __REQUESTED_MEMONHEAP_ISSET_ID = 5;
+  private static final int __REQUESTED_MEMOFFHEAP_ISSET_ID = 6;
+  private static final int __REQUESTED_CPU_ISSET_ID = 7;
+  private static final int __ASSIGNED_MEMONHEAP_ISSET_ID = 8;
+  private static final int __ASSIGNED_MEMOFFHEAP_ISSET_ID = 9;
+  private static final int __ASSIGNED_CPU_ISSET_ID = 10;
+  private short __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.SCHED_STATUS,_Fields.OWNER,_Fields.REPLICATION_COUNT,_Fields.REQUESTED_MEMONHEAP,_Fields.REQUESTED_MEMOFFHEAP,_Fields.REQUESTED_CPU,_Fields.ASSIGNED_MEMONHEAP,_Fields.ASSIGNED_MEMOFFHEAP,_Fields.ASSIGNED_CPU};
   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);
@@ -199,6 +235,18 @@
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.REPLICATION_COUNT, new org.apache.thrift.meta_data.FieldMetaData("replication_count", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.REQUESTED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("requested_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.REQUESTED_CPU, new org.apache.thrift.meta_data.FieldMetaData("requested_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMONHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memonheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_MEMOFFHEAP, new org.apache.thrift.meta_data.FieldMetaData("assigned_memoffheap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.ASSIGNED_CPU, new org.apache.thrift.meta_data.FieldMetaData("assigned_cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap);
   }
@@ -254,6 +302,12 @@
       this.owner = other.owner;
     }
     this.replication_count = other.replication_count;
+    this.requested_memonheap = other.requested_memonheap;
+    this.requested_memoffheap = other.requested_memoffheap;
+    this.requested_cpu = other.requested_cpu;
+    this.assigned_memonheap = other.assigned_memonheap;
+    this.assigned_memoffheap = other.assigned_memoffheap;
+    this.assigned_cpu = other.assigned_cpu;
   }
 
   public TopologySummary deepCopy() {
@@ -277,6 +331,18 @@
     this.owner = null;
     set_replication_count_isSet(false);
     this.replication_count = 0;
+    set_requested_memonheap_isSet(false);
+    this.requested_memonheap = 0.0;
+    set_requested_memoffheap_isSet(false);
+    this.requested_memoffheap = 0.0;
+    set_requested_cpu_isSet(false);
+    this.requested_cpu = 0.0;
+    set_assigned_memonheap_isSet(false);
+    this.assigned_memonheap = 0.0;
+    set_assigned_memoffheap_isSet(false);
+    this.assigned_memoffheap = 0.0;
+    set_assigned_cpu_isSet(false);
+    this.assigned_cpu = 0.0;
   }
 
   public String get_id() {
@@ -504,6 +570,138 @@
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REPLICATION_COUNT_ISSET_ID, value);
   }
 
+  public double get_requested_memonheap() {
+    return this.requested_memonheap;
+  }
+
+  public void set_requested_memonheap(double requested_memonheap) {
+    this.requested_memonheap = requested_memonheap;
+    set_requested_memonheap_isSet(true);
+  }
+
+  public void unset_requested_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_memoffheap() {
+    return this.requested_memoffheap;
+  }
+
+  public void set_requested_memoffheap(double requested_memoffheap) {
+    this.requested_memoffheap = requested_memoffheap;
+    set_requested_memoffheap_isSet(true);
+  }
+
+  public void unset_requested_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field requested_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_requested_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_requested_cpu() {
+    return this.requested_cpu;
+  }
+
+  public void set_requested_cpu(double requested_cpu) {
+    this.requested_cpu = requested_cpu;
+    set_requested_cpu_isSet(true);
+  }
+
+  public void unset_requested_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field requested_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_requested_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID);
+  }
+
+  public void set_requested_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTED_CPU_ISSET_ID, value);
+  }
+
+  public double get_assigned_memonheap() {
+    return this.assigned_memonheap;
+  }
+
+  public void set_assigned_memonheap(double assigned_memonheap) {
+    this.assigned_memonheap = assigned_memonheap;
+    set_assigned_memonheap_isSet(true);
+  }
+
+  public void unset_assigned_memonheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memonheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memonheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memonheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMONHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_memoffheap() {
+    return this.assigned_memoffheap;
+  }
+
+  public void set_assigned_memoffheap(double assigned_memoffheap) {
+    this.assigned_memoffheap = assigned_memoffheap;
+    set_assigned_memoffheap_isSet(true);
+  }
+
+  public void unset_assigned_memoffheap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_memoffheap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_memoffheap() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID);
+  }
+
+  public void set_assigned_memoffheap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_MEMOFFHEAP_ISSET_ID, value);
+  }
+
+  public double get_assigned_cpu() {
+    return this.assigned_cpu;
+  }
+
+  public void set_assigned_cpu(double assigned_cpu) {
+    this.assigned_cpu = assigned_cpu;
+    set_assigned_cpu_isSet(true);
+  }
+
+  public void unset_assigned_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  /** Returns true if field assigned_cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_assigned_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID);
+  }
+
+  public void set_assigned_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ASSIGNED_CPU_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case ID:
@@ -586,6 +784,54 @@
       }
       break;
 
+    case REQUESTED_MEMONHEAP:
+      if (value == null) {
+        unset_requested_memonheap();
+      } else {
+        set_requested_memonheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_MEMOFFHEAP:
+      if (value == null) {
+        unset_requested_memoffheap();
+      } else {
+        set_requested_memoffheap((Double)value);
+      }
+      break;
+
+    case REQUESTED_CPU:
+      if (value == null) {
+        unset_requested_cpu();
+      } else {
+        set_requested_cpu((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMONHEAP:
+      if (value == null) {
+        unset_assigned_memonheap();
+      } else {
+        set_assigned_memonheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_MEMOFFHEAP:
+      if (value == null) {
+        unset_assigned_memoffheap();
+      } else {
+        set_assigned_memoffheap((Double)value);
+      }
+      break;
+
+    case ASSIGNED_CPU:
+      if (value == null) {
+        unset_assigned_cpu();
+      } else {
+        set_assigned_cpu((Double)value);
+      }
+      break;
+
     }
   }
 
@@ -598,16 +844,16 @@
       return get_name();
 
     case NUM_TASKS:
-      return Integer.valueOf(get_num_tasks());
+      return get_num_tasks();
 
     case NUM_EXECUTORS:
-      return Integer.valueOf(get_num_executors());
+      return get_num_executors();
 
     case NUM_WORKERS:
-      return Integer.valueOf(get_num_workers());
+      return get_num_workers();
 
     case UPTIME_SECS:
-      return Integer.valueOf(get_uptime_secs());
+      return get_uptime_secs();
 
     case STATUS:
       return get_status();
@@ -619,7 +865,25 @@
       return get_owner();
 
     case REPLICATION_COUNT:
-      return Integer.valueOf(get_replication_count());
+      return get_replication_count();
+
+    case REQUESTED_MEMONHEAP:
+      return get_requested_memonheap();
+
+    case REQUESTED_MEMOFFHEAP:
+      return get_requested_memoffheap();
+
+    case REQUESTED_CPU:
+      return get_requested_cpu();
+
+    case ASSIGNED_MEMONHEAP:
+      return get_assigned_memonheap();
+
+    case ASSIGNED_MEMOFFHEAP:
+      return get_assigned_memoffheap();
+
+    case ASSIGNED_CPU:
+      return get_assigned_cpu();
 
     }
     throw new IllegalStateException();
@@ -652,6 +916,18 @@
       return is_set_owner();
     case REPLICATION_COUNT:
       return is_set_replication_count();
+    case REQUESTED_MEMONHEAP:
+      return is_set_requested_memonheap();
+    case REQUESTED_MEMOFFHEAP:
+      return is_set_requested_memoffheap();
+    case REQUESTED_CPU:
+      return is_set_requested_cpu();
+    case ASSIGNED_MEMONHEAP:
+      return is_set_assigned_memonheap();
+    case ASSIGNED_MEMOFFHEAP:
+      return is_set_assigned_memoffheap();
+    case ASSIGNED_CPU:
+      return is_set_assigned_cpu();
     }
     throw new IllegalStateException();
   }
@@ -759,6 +1035,60 @@
         return false;
     }
 
+    boolean this_present_requested_memonheap = true && this.is_set_requested_memonheap();
+    boolean that_present_requested_memonheap = true && that.is_set_requested_memonheap();
+    if (this_present_requested_memonheap || that_present_requested_memonheap) {
+      if (!(this_present_requested_memonheap && that_present_requested_memonheap))
+        return false;
+      if (this.requested_memonheap != that.requested_memonheap)
+        return false;
+    }
+
+    boolean this_present_requested_memoffheap = true && this.is_set_requested_memoffheap();
+    boolean that_present_requested_memoffheap = true && that.is_set_requested_memoffheap();
+    if (this_present_requested_memoffheap || that_present_requested_memoffheap) {
+      if (!(this_present_requested_memoffheap && that_present_requested_memoffheap))
+        return false;
+      if (this.requested_memoffheap != that.requested_memoffheap)
+        return false;
+    }
+
+    boolean this_present_requested_cpu = true && this.is_set_requested_cpu();
+    boolean that_present_requested_cpu = true && that.is_set_requested_cpu();
+    if (this_present_requested_cpu || that_present_requested_cpu) {
+      if (!(this_present_requested_cpu && that_present_requested_cpu))
+        return false;
+      if (this.requested_cpu != that.requested_cpu)
+        return false;
+    }
+
+    boolean this_present_assigned_memonheap = true && this.is_set_assigned_memonheap();
+    boolean that_present_assigned_memonheap = true && that.is_set_assigned_memonheap();
+    if (this_present_assigned_memonheap || that_present_assigned_memonheap) {
+      if (!(this_present_assigned_memonheap && that_present_assigned_memonheap))
+        return false;
+      if (this.assigned_memonheap != that.assigned_memonheap)
+        return false;
+    }
+
+    boolean this_present_assigned_memoffheap = true && this.is_set_assigned_memoffheap();
+    boolean that_present_assigned_memoffheap = true && that.is_set_assigned_memoffheap();
+    if (this_present_assigned_memoffheap || that_present_assigned_memoffheap) {
+      if (!(this_present_assigned_memoffheap && that_present_assigned_memoffheap))
+        return false;
+      if (this.assigned_memoffheap != that.assigned_memoffheap)
+        return false;
+    }
+
+    boolean this_present_assigned_cpu = true && this.is_set_assigned_cpu();
+    boolean that_present_assigned_cpu = true && that.is_set_assigned_cpu();
+    if (this_present_assigned_cpu || that_present_assigned_cpu) {
+      if (!(this_present_assigned_cpu && that_present_assigned_cpu))
+        return false;
+      if (this.assigned_cpu != that.assigned_cpu)
+        return false;
+    }
+
     return true;
   }
 
@@ -816,6 +1146,36 @@
     if (present_replication_count)
       list.add(replication_count);
 
+    boolean present_requested_memonheap = true && (is_set_requested_memonheap());
+    list.add(present_requested_memonheap);
+    if (present_requested_memonheap)
+      list.add(requested_memonheap);
+
+    boolean present_requested_memoffheap = true && (is_set_requested_memoffheap());
+    list.add(present_requested_memoffheap);
+    if (present_requested_memoffheap)
+      list.add(requested_memoffheap);
+
+    boolean present_requested_cpu = true && (is_set_requested_cpu());
+    list.add(present_requested_cpu);
+    if (present_requested_cpu)
+      list.add(requested_cpu);
+
+    boolean present_assigned_memonheap = true && (is_set_assigned_memonheap());
+    list.add(present_assigned_memonheap);
+    if (present_assigned_memonheap)
+      list.add(assigned_memonheap);
+
+    boolean present_assigned_memoffheap = true && (is_set_assigned_memoffheap());
+    list.add(present_assigned_memoffheap);
+    if (present_assigned_memoffheap)
+      list.add(assigned_memoffheap);
+
+    boolean present_assigned_cpu = true && (is_set_assigned_cpu());
+    list.add(present_assigned_cpu);
+    if (present_assigned_cpu)
+      list.add(assigned_cpu);
+
     return list.hashCode();
   }
 
@@ -927,6 +1287,66 @@
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(is_set_requested_memonheap()).compareTo(other.is_set_requested_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memonheap, other.requested_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_memoffheap()).compareTo(other.is_set_requested_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_memoffheap, other.requested_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_requested_cpu()).compareTo(other.is_set_requested_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_requested_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requested_cpu, other.requested_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memonheap()).compareTo(other.is_set_assigned_memonheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memonheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memonheap, other.assigned_memonheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_memoffheap()).compareTo(other.is_set_assigned_memoffheap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_memoffheap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_memoffheap, other.assigned_memoffheap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_assigned_cpu()).compareTo(other.is_set_assigned_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_assigned_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.assigned_cpu, other.assigned_cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1012,6 +1432,42 @@
       sb.append(this.replication_count);
       first = false;
     }
+    if (is_set_requested_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memonheap:");
+      sb.append(this.requested_memonheap);
+      first = false;
+    }
+    if (is_set_requested_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_memoffheap:");
+      sb.append(this.requested_memoffheap);
+      first = false;
+    }
+    if (is_set_requested_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("requested_cpu:");
+      sb.append(this.requested_cpu);
+      first = false;
+    }
+    if (is_set_assigned_memonheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memonheap:");
+      sb.append(this.assigned_memonheap);
+      first = false;
+    }
+    if (is_set_assigned_memoffheap()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_memoffheap:");
+      sb.append(this.assigned_memoffheap);
+      first = false;
+    }
+    if (is_set_assigned_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("assigned_cpu:");
+      sb.append(this.assigned_cpu);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1165,6 +1621,54 @@
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 521: // REQUESTED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memonheap = iprot.readDouble();
+              struct.set_requested_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 522: // REQUESTED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_memoffheap = iprot.readDouble();
+              struct.set_requested_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 523: // REQUESTED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.requested_cpu = iprot.readDouble();
+              struct.set_requested_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 524: // ASSIGNED_MEMONHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memonheap = iprot.readDouble();
+              struct.set_assigned_memonheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 525: // ASSIGNED_MEMOFFHEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_memoffheap = iprot.readDouble();
+              struct.set_assigned_memoffheap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 526: // ASSIGNED_CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.assigned_cpu = iprot.readDouble();
+              struct.set_assigned_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -1224,6 +1728,36 @@
         oprot.writeI32(struct.replication_count);
         oprot.writeFieldEnd();
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeFieldBegin(REQUESTED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.requested_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeFieldBegin(REQUESTED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.requested_cpu);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMONHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memonheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeFieldBegin(ASSIGNED_MEMOFFHEAP_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_memoffheap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeFieldBegin(ASSIGNED_CPU_FIELD_DESC);
+        oprot.writeDouble(struct.assigned_cpu);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -1258,7 +1792,25 @@
       if (struct.is_set_replication_count()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_requested_memonheap()) {
+        optionals.set(3);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        optionals.set(4);
+      }
+      if (struct.is_set_requested_cpu()) {
+        optionals.set(5);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        optionals.set(6);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        optionals.set(7);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        optionals.set(8);
+      }
+      oprot.writeBitSet(optionals, 9);
       if (struct.is_set_sched_status()) {
         oprot.writeString(struct.sched_status);
       }
@@ -1268,6 +1820,24 @@
       if (struct.is_set_replication_count()) {
         oprot.writeI32(struct.replication_count);
       }
+      if (struct.is_set_requested_memonheap()) {
+        oprot.writeDouble(struct.requested_memonheap);
+      }
+      if (struct.is_set_requested_memoffheap()) {
+        oprot.writeDouble(struct.requested_memoffheap);
+      }
+      if (struct.is_set_requested_cpu()) {
+        oprot.writeDouble(struct.requested_cpu);
+      }
+      if (struct.is_set_assigned_memonheap()) {
+        oprot.writeDouble(struct.assigned_memonheap);
+      }
+      if (struct.is_set_assigned_memoffheap()) {
+        oprot.writeDouble(struct.assigned_memoffheap);
+      }
+      if (struct.is_set_assigned_cpu()) {
+        oprot.writeDouble(struct.assigned_cpu);
+      }
     }
 
     @Override
@@ -1287,7 +1857,7 @@
       struct.set_uptime_secs_isSet(true);
       struct.status = iprot.readString();
       struct.set_status_isSet(true);
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(9);
       if (incoming.get(0)) {
         struct.sched_status = iprot.readString();
         struct.set_sched_status_isSet(true);
@@ -1300,6 +1870,30 @@
         struct.replication_count = iprot.readI32();
         struct.set_replication_count_isSet(true);
       }
+      if (incoming.get(3)) {
+        struct.requested_memonheap = iprot.readDouble();
+        struct.set_requested_memonheap_isSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.requested_memoffheap = iprot.readDouble();
+        struct.set_requested_memoffheap_isSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.requested_cpu = iprot.readDouble();
+        struct.set_requested_cpu_isSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.assigned_memonheap = iprot.readDouble();
+        struct.set_assigned_memonheap_isSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.assigned_memoffheap = iprot.readDouble();
+        struct.set_assigned_memoffheap_isSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.assigned_cpu = iprot.readDouble();
+        struct.set_assigned_cpu_isSet(true);
+      }
     }
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java b/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
new file mode 100644
index 0000000..3af22fb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/generated/WorkerResources.java
@@ -0,0 +1,605 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package backtype.storm.generated;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class WorkerResources implements org.apache.thrift.TBase<WorkerResources, WorkerResources._Fields>, java.io.Serializable, Cloneable, Comparable<WorkerResources> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerResources");
+
+  private static final org.apache.thrift.protocol.TField MEM_ON_HEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("mem_on_heap", org.apache.thrift.protocol.TType.DOUBLE, (short)1);
+  private static final org.apache.thrift.protocol.TField MEM_OFF_HEAP_FIELD_DESC = new org.apache.thrift.protocol.TField("mem_off_heap", org.apache.thrift.protocol.TType.DOUBLE, (short)2);
+  private static final org.apache.thrift.protocol.TField CPU_FIELD_DESC = new org.apache.thrift.protocol.TField("cpu", org.apache.thrift.protocol.TType.DOUBLE, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WorkerResourcesStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WorkerResourcesTupleSchemeFactory());
+  }
+
+  private double mem_on_heap; // optional
+  private double mem_off_heap; // optional
+  private double cpu; // 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 {
+    MEM_ON_HEAP((short)1, "mem_on_heap"),
+    MEM_OFF_HEAP((short)2, "mem_off_heap"),
+    CPU((short)3, "cpu");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MEM_ON_HEAP
+          return MEM_ON_HEAP;
+        case 2: // MEM_OFF_HEAP
+          return MEM_OFF_HEAP;
+        case 3: // CPU
+          return CPU;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __MEM_ON_HEAP_ISSET_ID = 0;
+  private static final int __MEM_OFF_HEAP_ISSET_ID = 1;
+  private static final int __CPU_ISSET_ID = 2;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.MEM_ON_HEAP,_Fields.MEM_OFF_HEAP,_Fields.CPU};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MEM_ON_HEAP, new org.apache.thrift.meta_data.FieldMetaData("mem_on_heap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.MEM_OFF_HEAP, new org.apache.thrift.meta_data.FieldMetaData("mem_off_heap", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    tmpMap.put(_Fields.CPU, new org.apache.thrift.meta_data.FieldMetaData("cpu", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerResources.class, metaDataMap);
+  }
+
+  public WorkerResources() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WorkerResources(WorkerResources other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.mem_on_heap = other.mem_on_heap;
+    this.mem_off_heap = other.mem_off_heap;
+    this.cpu = other.cpu;
+  }
+
+  public WorkerResources deepCopy() {
+    return new WorkerResources(this);
+  }
+
+  @Override
+  public void clear() {
+    set_mem_on_heap_isSet(false);
+    this.mem_on_heap = 0.0;
+    set_mem_off_heap_isSet(false);
+    this.mem_off_heap = 0.0;
+    set_cpu_isSet(false);
+    this.cpu = 0.0;
+  }
+
+  public double get_mem_on_heap() {
+    return this.mem_on_heap;
+  }
+
+  public void set_mem_on_heap(double mem_on_heap) {
+    this.mem_on_heap = mem_on_heap;
+    set_mem_on_heap_isSet(true);
+  }
+
+  public void unset_mem_on_heap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MEM_ON_HEAP_ISSET_ID);
+  }
+
+  /** Returns true if field mem_on_heap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_mem_on_heap() {
+    return EncodingUtils.testBit(__isset_bitfield, __MEM_ON_HEAP_ISSET_ID);
+  }
+
+  public void set_mem_on_heap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MEM_ON_HEAP_ISSET_ID, value);
+  }
+
+  public double get_mem_off_heap() {
+    return this.mem_off_heap;
+  }
+
+  public void set_mem_off_heap(double mem_off_heap) {
+    this.mem_off_heap = mem_off_heap;
+    set_mem_off_heap_isSet(true);
+  }
+
+  public void unset_mem_off_heap() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MEM_OFF_HEAP_ISSET_ID);
+  }
+
+  /** Returns true if field mem_off_heap is set (has been assigned a value) and false otherwise */
+  public boolean is_set_mem_off_heap() {
+    return EncodingUtils.testBit(__isset_bitfield, __MEM_OFF_HEAP_ISSET_ID);
+  }
+
+  public void set_mem_off_heap_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MEM_OFF_HEAP_ISSET_ID, value);
+  }
+
+  public double get_cpu() {
+    return this.cpu;
+  }
+
+  public void set_cpu(double cpu) {
+    this.cpu = cpu;
+    set_cpu_isSet(true);
+  }
+
+  public void unset_cpu() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CPU_ISSET_ID);
+  }
+
+  /** Returns true if field cpu is set (has been assigned a value) and false otherwise */
+  public boolean is_set_cpu() {
+    return EncodingUtils.testBit(__isset_bitfield, __CPU_ISSET_ID);
+  }
+
+  public void set_cpu_isSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CPU_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MEM_ON_HEAP:
+      if (value == null) {
+        unset_mem_on_heap();
+      } else {
+        set_mem_on_heap((Double)value);
+      }
+      break;
+
+    case MEM_OFF_HEAP:
+      if (value == null) {
+        unset_mem_off_heap();
+      } else {
+        set_mem_off_heap((Double)value);
+      }
+      break;
+
+    case CPU:
+      if (value == null) {
+        unset_cpu();
+      } else {
+        set_cpu((Double)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MEM_ON_HEAP:
+      return get_mem_on_heap();
+
+    case MEM_OFF_HEAP:
+      return get_mem_off_heap();
+
+    case CPU:
+      return get_cpu();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case MEM_ON_HEAP:
+      return is_set_mem_on_heap();
+    case MEM_OFF_HEAP:
+      return is_set_mem_off_heap();
+    case CPU:
+      return is_set_cpu();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WorkerResources)
+      return this.equals((WorkerResources)that);
+    return false;
+  }
+
+  public boolean equals(WorkerResources that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_mem_on_heap = true && this.is_set_mem_on_heap();
+    boolean that_present_mem_on_heap = true && that.is_set_mem_on_heap();
+    if (this_present_mem_on_heap || that_present_mem_on_heap) {
+      if (!(this_present_mem_on_heap && that_present_mem_on_heap))
+        return false;
+      if (this.mem_on_heap != that.mem_on_heap)
+        return false;
+    }
+
+    boolean this_present_mem_off_heap = true && this.is_set_mem_off_heap();
+    boolean that_present_mem_off_heap = true && that.is_set_mem_off_heap();
+    if (this_present_mem_off_heap || that_present_mem_off_heap) {
+      if (!(this_present_mem_off_heap && that_present_mem_off_heap))
+        return false;
+      if (this.mem_off_heap != that.mem_off_heap)
+        return false;
+    }
+
+    boolean this_present_cpu = true && this.is_set_cpu();
+    boolean that_present_cpu = true && that.is_set_cpu();
+    if (this_present_cpu || that_present_cpu) {
+      if (!(this_present_cpu && that_present_cpu))
+        return false;
+      if (this.cpu != that.cpu)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_mem_on_heap = true && (is_set_mem_on_heap());
+    list.add(present_mem_on_heap);
+    if (present_mem_on_heap)
+      list.add(mem_on_heap);
+
+    boolean present_mem_off_heap = true && (is_set_mem_off_heap());
+    list.add(present_mem_off_heap);
+    if (present_mem_off_heap)
+      list.add(mem_off_heap);
+
+    boolean present_cpu = true && (is_set_cpu());
+    list.add(present_cpu);
+    if (present_cpu)
+      list.add(cpu);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WorkerResources other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(is_set_mem_on_heap()).compareTo(other.is_set_mem_on_heap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_mem_on_heap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mem_on_heap, other.mem_on_heap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_mem_off_heap()).compareTo(other.is_set_mem_off_heap());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_mem_off_heap()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.mem_off_heap, other.mem_off_heap);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_cpu()).compareTo(other.is_set_cpu());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_cpu()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cpu, other.cpu);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WorkerResources(");
+    boolean first = true;
+
+    if (is_set_mem_on_heap()) {
+      sb.append("mem_on_heap:");
+      sb.append(this.mem_on_heap);
+      first = false;
+    }
+    if (is_set_mem_off_heap()) {
+      if (!first) sb.append(", ");
+      sb.append("mem_off_heap:");
+      sb.append(this.mem_off_heap);
+      first = false;
+    }
+    if (is_set_cpu()) {
+      if (!first) sb.append(", ");
+      sb.append("cpu:");
+      sb.append(this.cpu);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WorkerResourcesStandardSchemeFactory implements SchemeFactory {
+    public WorkerResourcesStandardScheme getScheme() {
+      return new WorkerResourcesStandardScheme();
+    }
+  }
+
+  private static class WorkerResourcesStandardScheme extends StandardScheme<WorkerResources> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerResources struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // MEM_ON_HEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.mem_on_heap = iprot.readDouble();
+              struct.set_mem_on_heap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // MEM_OFF_HEAP
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.mem_off_heap = iprot.readDouble();
+              struct.set_mem_off_heap_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // CPU
+            if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) {
+              struct.cpu = iprot.readDouble();
+              struct.set_cpu_isSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerResources struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.is_set_mem_on_heap()) {
+        oprot.writeFieldBegin(MEM_ON_HEAP_FIELD_DESC);
+        oprot.writeDouble(struct.mem_on_heap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_mem_off_heap()) {
+        oprot.writeFieldBegin(MEM_OFF_HEAP_FIELD_DESC);
+        oprot.writeDouble(struct.mem_off_heap);
+        oprot.writeFieldEnd();
+      }
+      if (struct.is_set_cpu()) {
+        oprot.writeFieldBegin(CPU_FIELD_DESC);
+        oprot.writeDouble(struct.cpu);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WorkerResourcesTupleSchemeFactory implements SchemeFactory {
+    public WorkerResourcesTupleScheme getScheme() {
+      return new WorkerResourcesTupleScheme();
+    }
+  }
+
+  private static class WorkerResourcesTupleScheme extends TupleScheme<WorkerResources> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WorkerResources struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.is_set_mem_on_heap()) {
+        optionals.set(0);
+      }
+      if (struct.is_set_mem_off_heap()) {
+        optionals.set(1);
+      }
+      if (struct.is_set_cpu()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.is_set_mem_on_heap()) {
+        oprot.writeDouble(struct.mem_on_heap);
+      }
+      if (struct.is_set_mem_off_heap()) {
+        oprot.writeDouble(struct.mem_off_heap);
+      }
+      if (struct.is_set_cpu()) {
+        oprot.writeDouble(struct.cpu);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WorkerResources struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.mem_on_heap = iprot.readDouble();
+        struct.set_mem_on_heap_isSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.mem_off_heap = iprot.readDouble();
+        struct.set_mem_off_heap_isSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.cpu = iprot.readDouble();
+        struct.set_cpu_isSet(true);
+      }
+    }
+  }
+
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/grouping/Load.java b/storm-core/src/jvm/backtype/storm/grouping/Load.java
new file mode 100644
index 0000000..a10c791
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/grouping/Load.java
@@ -0,0 +1,77 @@
+/**
+ * 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.grouping;
+
+import java.io.Serializable;
+
+/**
+ * Represents the load that a Bolt is currently under to help in
+ * deciding where to route a tuple, to help balance the load.
+ */
+public class Load {
+    private boolean hasMetrics = false;
+    private double boltLoad = 0.0; //0 no load to 1 fully loaded
+    private double connectionLoad = 0.0; //0 no load to 1 fully loaded
+
+    /**
+     * Create a new load
+     * @param hasMetrics have metrics been reported yet?
+     * @param boltLoad the load as reported by the bolt 0.0 no load 1.0 fully loaded
+     * @param connectionLoad the load as reported by the connection to the bolt 0.0 no load 1.0 fully loaded.
+     */
+    public Load(boolean hasMetrics, double boltLoad, double connectionLoad) {
+        this.hasMetrics = hasMetrics;
+        this.boltLoad = boltLoad;
+        this.connectionLoad = connectionLoad;
+    }
+
+    /**
+     * @return true if metrics have been reported so far.
+     */
+    public boolean hasMetrics() {
+        return hasMetrics;
+    }
+
+    /**
+     * @return the load as reported by the bolt.
+     */
+    public double getBoltLoad() {
+        return boltLoad;
+    }
+
+    /**
+     * @return the load as reported by the connection
+     */
+    public double getConnectionLoad() {
+        return connectionLoad;
+    }
+
+    /**
+     * @return the load that is a combination of sub loads.
+     */
+    public double getLoad() {
+        if (!hasMetrics) {
+            return 1.0;
+        }
+        return connectionLoad > boltLoad ? connectionLoad : boltLoad;
+    }
+
+    public String toString() {
+        return "[:load "+boltLoad+" "+connectionLoad+"]";
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/grouping/LoadAwareCustomStreamGrouping.java b/storm-core/src/jvm/backtype/storm/grouping/LoadAwareCustomStreamGrouping.java
new file mode 100644
index 0000000..f39273f
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/grouping/LoadAwareCustomStreamGrouping.java
@@ -0,0 +1,24 @@
+/**
+ * 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.grouping;
+
+import java.util.List;
+
+public interface LoadAwareCustomStreamGrouping extends CustomStreamGrouping {
+   List<Integer> chooseTasks(int taskId, List<Object> values, LoadMapping load);
+}
diff --git a/storm-core/src/jvm/backtype/storm/grouping/LoadAwareShuffleGrouping.java b/storm-core/src/jvm/backtype/storm/grouping/LoadAwareShuffleGrouping.java
new file mode 100644
index 0000000..a19b585
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/grouping/LoadAwareShuffleGrouping.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.grouping;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.task.WorkerTopologyContext;
+
+public class LoadAwareShuffleGrouping implements LoadAwareCustomStreamGrouping, Serializable {
+    private Random random;
+    private List<Integer>[] rets;
+    private int[] targets;
+    private int[] loads;
+    private int total;
+    private long lastUpdate = 0;
+
+    @Override
+    public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+        random = new Random();
+        rets = (List<Integer>[])new List<?>[targetTasks.size()];
+        targets = new int[targetTasks.size()];
+        for (int i = 0; i < targets.length; i++) {
+            rets[i] = Arrays.asList(targetTasks.get(i));
+            targets[i] = targetTasks.get(i);
+        }
+        loads = new int[targets.length];
+    }
+
+    @Override
+    public List<Integer> chooseTasks(int taskId, List<Object> values) {
+        throw new RuntimeException("NOT IMPLEMENTED");
+    }
+
+    @Override
+    public List<Integer> chooseTasks(int taskId, List<Object> values, LoadMapping load) {
+        if ((lastUpdate + 1000) < System.currentTimeMillis()) {
+            int local_total = 0;
+            for (int i = 0; i < targets.length; i++) {
+                int val = (int)(101 - (load.get(targets[i]) * 100));
+                loads[i] = val;
+                local_total += val;
+            }
+            total = local_total;
+            lastUpdate = System.currentTimeMillis();
+        }
+        int selected = random.nextInt(total);
+        int sum = 0;
+        for (int i = 0; i < targets.length; i++) {
+            sum += loads[i];
+            if (selected < sum) {
+                return rets[i];
+            }
+        }
+        return rets[rets.length-1];
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/grouping/LoadMapping.java b/storm-core/src/jvm/backtype/storm/grouping/LoadMapping.java
new file mode 100644
index 0000000..abd8983
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/grouping/LoadMapping.java
@@ -0,0 +1,64 @@
+/**
+ * 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.grouping;
+
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.Map;
+import java.util.HashMap;
+
+/**
+ * Holds a list of the current loads
+ */
+public class LoadMapping {
+    private static final Load NOT_CONNECTED = new Load(false, 1.0, 1.0);
+    private final AtomicReference<Map<Integer,Load>> _local = new AtomicReference<Map<Integer,Load>>(new HashMap<Integer,Load>());
+    private final AtomicReference<Map<Integer,Load>> _remote = new AtomicReference<Map<Integer,Load>>(new HashMap<Integer,Load>());
+
+    public void setLocal(Map<Integer, Double> local) {
+        Map<Integer, Load> newLocal = new HashMap<Integer, Load>();
+        if (local != null) {
+          for (Map.Entry<Integer, Double> entry: local.entrySet()) {
+            newLocal.put(entry.getKey(), new Load(true, entry.getValue(), 0.0));
+          }
+        }
+        _local.set(newLocal);
+    }
+
+    public void setRemote(Map<Integer, Load> remote) {
+        if (remote != null) {
+          _remote.set(new HashMap<Integer, Load>(remote));
+        } else {
+          _remote.set(new HashMap<Integer, Load>());
+        }
+    }
+
+    public Load getLoad(int task) {
+        Load ret = _local.get().get(task);
+        if (ret == null) {
+          ret = _remote.get().get(task);
+        }
+        if (ret == null) {
+          ret = NOT_CONNECTED;
+        }
+        return ret;
+    }
+
+    public double get(int task) {
+        return getLoad(task).getLoad();
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java b/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
index 43ad5a0..e92156e 100644
--- a/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
+++ b/storm-core/src/jvm/backtype/storm/grouping/PartialKeyGrouping.java
@@ -24,7 +24,6 @@
 import java.util.List;
 
 import backtype.storm.generated.GlobalStreamId;
-import backtype.storm.grouping.CustomStreamGrouping;
 import backtype.storm.task.WorkerTopologyContext;
 import backtype.storm.tuple.Fields;
 
@@ -59,9 +58,9 @@
 
     @Override
     public List<Integer> chooseTasks(int taskId, List<Object> values) {
-        List<Integer> boltIds = new ArrayList<Integer>(1);
+        List<Integer> boltIds = new ArrayList<>(1);
         if (values.size() > 0) {
-            byte[] raw = null;
+            byte[] raw;
             if (fields != null) {
                 List<Object> selectedFields = outFields.select(fields, values);
                 ByteBuffer out = ByteBuffer.allocate(selectedFields.size() * 4);
diff --git a/storm-core/src/jvm/backtype/storm/grouping/ShuffleGrouping.java b/storm-core/src/jvm/backtype/storm/grouping/ShuffleGrouping.java
new file mode 100644
index 0000000..01755bb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/grouping/ShuffleGrouping.java
@@ -0,0 +1,65 @@
+/**
+ * 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.grouping;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.task.WorkerTopologyContext;
+
+public class ShuffleGrouping implements CustomStreamGrouping, Serializable {
+    private Random random;
+    private ArrayList<List<Integer>> choices;
+    private AtomicInteger current;
+
+    @Override
+    public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targetTasks) {
+        random = new Random();
+        choices = new ArrayList<List<Integer>>(targetTasks.size());
+        for (Integer i: targetTasks) {
+            choices.add(Arrays.asList(i));
+        }
+        Collections.shuffle(choices, random);
+        current = new AtomicInteger(0);
+    }
+
+    @Override
+    public List<Integer> chooseTasks(int taskId, List<Object> values) {
+        int rightNow;
+        int size = choices.size();
+        while (true) {
+            rightNow = current.incrementAndGet();
+            if (rightNow < size) {
+                return choices.get(rightNow);
+            } else if (rightNow == size) {
+                current.set(0);
+                //This should be thread safe so long as ArrayList does not have any internal state that can be messed up by multi-treaded access.
+                Collections.shuffle(choices, random);
+                return choices.get(0);
+            }
+            //race condition with another thread, and we lost
+            // try again
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/hooks/BaseWorkerHook.java b/storm-core/src/jvm/backtype/storm/hooks/BaseWorkerHook.java
new file mode 100644
index 0000000..c146ac2
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/hooks/BaseWorkerHook.java
@@ -0,0 +1,51 @@
+/**
+ * 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.hooks;
+
+import backtype.storm.task.WorkerTopologyContext;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * A BaseWorkerHook is a noop implementation of IWorkerHook. You
+ * may extends this class and implement any and/or all methods you
+ * need for your workers.
+ */
+public class BaseWorkerHook implements IWorkerHook, Serializable {
+    private static final long serialVersionUID = 2589466485198339529L;
+
+    /**
+     * This method is called when a worker is started
+     *
+     * @param stormConf The Storm configuration for this worker
+     * @param context This object can be used to get information about this worker's place within the topology
+     */
+    @Override
+    public void start(Map stormConf, WorkerTopologyContext context) {
+        // NOOP
+    }
+
+    /**
+     * This method is called right before a worker shuts down
+     */
+    @Override
+    public void shutdown() {
+        // NOOP
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/hooks/IWorkerHook.java b/storm-core/src/jvm/backtype/storm/hooks/IWorkerHook.java
new file mode 100644
index 0000000..6fb3946
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/hooks/IWorkerHook.java
@@ -0,0 +1,44 @@
+/**
+ * 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.hooks;
+
+import backtype.storm.task.WorkerTopologyContext;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * An IWorkerHook represents a topology component that can be executed
+ * when a worker starts, and when a worker shuts down. It can be useful
+ * when you want to execute operations before topology processing starts,
+ * or cleanup operations before your workers shut down.
+ */
+public interface IWorkerHook extends Serializable {
+    /**
+     * This method is called when a worker is started
+     *
+     * @param stormConf The Storm configuration for this worker
+     * @param context This object can be used to get information about this worker's place within the topology
+     */
+    void start(Map stormConf, WorkerTopologyContext context);
+
+    /**
+     * This method is called right before a worker shuts down
+     */
+    void shutdown();
+}
\ No newline at end of file
diff --git a/storm-core/src/jvm/backtype/storm/logging/ThriftAccessLogger.java b/storm-core/src/jvm/backtype/storm/logging/ThriftAccessLogger.java
new file mode 100644
index 0000000..3374af0
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/logging/ThriftAccessLogger.java
@@ -0,0 +1,27 @@
+/**
+ * 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.logging;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ThriftAccessLogger {
+  private static final Logger LOG = LoggerFactory.getLogger(ThriftAccessLogger.class);
+  public void log(String logMessage) {
+    LOG.info(logMessage);
+  }
+}
diff --git a/storm-core/src/jvm/backtype/storm/logging/filters/AccessLoggingFilter.java b/storm-core/src/jvm/backtype/storm/logging/filters/AccessLoggingFilter.java
new file mode 100644
index 0000000..6c368f1
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/logging/filters/AccessLoggingFilter.java
@@ -0,0 +1,52 @@
+/**
+ * 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.logging.filters;
+import java.io.IOException;
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AccessLoggingFilter implements Filter {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AccessLoggingFilter.class);
+  public void init(FilterConfig config) throws ServletException {
+  }
+
+  public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) throws IOException, ServletException {
+    handle((HttpServletRequest)request, (HttpServletResponse)response, chain);
+  }
+
+  public void handle(HttpServletRequest request, HttpServletResponse response, FilterChain chain) throws IOException, ServletException{
+    if (request != null) {
+      LOG.info("Access from: {} url: {} principal: {}", request.getRemoteAddr(), request.getRequestURL(),
+              (request.getUserPrincipal() == null ? "" : request.getUserPrincipal().getName()));
+    }
+    chain.doFilter(request, response);
+  }
+
+  public void destroy() {
+  }
+}
diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java b/storm-core/src/jvm/backtype/storm/messaging/AddressedTuple.java
similarity index 60%
copy from examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java
copy to storm-core/src/jvm/backtype/storm/messaging/AddressedTuple.java
index b253350..de9a3e6 100644
--- a/examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/AddressedTuple.java
@@ -15,26 +15,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package storm.starter.tools;
+package backtype.storm.messaging;
 
-import backtype.storm.Constants;
 import backtype.storm.tuple.Tuple;
 
-import static org.mockito.Mockito.*;
+/**
+ * A Tuple that is addressed to a destination.
+ */
+public class AddressedTuple {
+    public final Tuple tuple;
+    public final int dest;
 
-public final class MockTupleHelpers {
+    public AddressedTuple(int dest, Tuple tuple) {
+        this.dest = dest;
+        this.tuple = tuple;
+    }
 
-  private MockTupleHelpers() {
-  }
+    public Tuple getTuple() {
+        return tuple;
+    }
 
-  public static Tuple mockTickTuple() {
-    return mockTuple(Constants.SYSTEM_COMPONENT_ID, Constants.SYSTEM_TICK_STREAM_ID);
-  }
+    public int getDest() {
+        return dest;
+    }
 
-  public static Tuple mockTuple(String componentId, String streamId) {
-    Tuple tuple = mock(Tuple.class);
-    when(tuple.getSourceComponent()).thenReturn(componentId);
-    when(tuple.getSourceStreamId()).thenReturn(streamId);
-    return tuple;
-  }
+    @Override
+    public String toString() {
+        return "[dest: "+dest+" tuple: "+tuple+"]";
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java b/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java
index 37981ca..b92b0d6 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java
@@ -39,9 +39,9 @@
      * data sending or receiving. All data sending request will be dropped.
      */
     Closed
-  };
+  }
 
-  /**
+    /**
    * whether this connection is available to transfer data
    */
   public abstract Status status();
diff --git a/storm-core/src/jvm/backtype/storm/messaging/DeserializingConnectionCallback.java b/storm-core/src/jvm/backtype/storm/messaging/DeserializingConnectionCallback.java
new file mode 100644
index 0000000..1e2d3aa
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/DeserializingConnectionCallback.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging;
+
+import backtype.storm.task.GeneralTopologyContext;
+import backtype.storm.tuple.AddressedTuple;
+import backtype.storm.serialization.KryoTupleDeserializer;
+
+import clojure.lang.IFn;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class that is called when a TaskMessage arrives.
+ */
+public class DeserializingConnectionCallback implements IConnectionCallback {
+    private final IFn _cb;
+    private final Map _conf;
+    private final GeneralTopologyContext _context;
+    private final ThreadLocal<KryoTupleDeserializer> _des =
+         new ThreadLocal<KryoTupleDeserializer>() {
+             @Override
+             protected KryoTupleDeserializer initialValue() {
+                 return new KryoTupleDeserializer(_conf, _context);
+             }
+         };
+
+    public DeserializingConnectionCallback(final Map conf, final GeneralTopologyContext context, IFn callback) {
+        _conf = conf;
+        _context = context;
+        _cb = callback;
+    }
+
+    @Override
+    public void recv(List<TaskMessage> batch) {
+        KryoTupleDeserializer des = _des.get();
+        ArrayList<AddressedTuple> ret = new ArrayList<>(batch.size());
+        for (TaskMessage message: batch) {
+            ret.add(new AddressedTuple(message.task(), des.deserialize(message.message())));
+        }
+        _cb.invoke(ret);
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/IConnection.java b/storm-core/src/jvm/backtype/storm/messaging/IConnection.java
index ead4935..a03b3a2 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/IConnection.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/IConnection.java
@@ -17,16 +17,23 @@
  */
 package backtype.storm.messaging;
 
+import backtype.storm.grouping.Load;
+import java.util.Collection;
 import java.util.Iterator;
+import java.util.Map;
 
-public interface IConnection {   
-    
+public interface IConnection {
     /**
-     * receive a batch message iterator (consists taskId and payload)
-     * @param flags 0: block, 1: non-block
-     * @return
+     * Register a callback to be notified when data is ready to be processed.
+     * @param cb the callback to process the messages.
      */
-    public Iterator<TaskMessage> recv(int flags, int clientId);
+    public void registerRecv(IConnectionCallback cb);
+
+    /**
+     * Send load metrics to all downstream connections.
+     * @param taskToLoad a map from the task id to the load for that task.
+     */
+    public void sendLoadMetrics(Map<Integer, Double> taskToLoad);
     
     /**
      * send a message with taskId and payload
@@ -43,6 +50,13 @@
     public void send(Iterator<TaskMessage> msgs);
     
     /**
+     * Get the current load for the given tasks
+     * @param tasks the tasks to look for.
+     * @return a Load for each of the tasks it knows about.
+     */
+    public Map<Integer, Load> getLoad(Collection<Integer> tasks);
+
+    /**
      * close this connection
      */
     public void close();
diff --git a/storm-core/src/jvm/backtype/storm/messaging/IConnectionCallback.java b/storm-core/src/jvm/backtype/storm/messaging/IConnectionCallback.java
new file mode 100644
index 0000000..ecf0828
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/IConnectionCallback.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging;
+
+import java.util.List;
+
+/**
+ * A class that is called when a TaskMessage arrives.
+ */
+public interface IConnectionCallback {
+    /**
+     * A batch of new messages have arrived to be processed
+     * @param batch the messages to be processed
+     */
+    public void recv(List<TaskMessage> batch);
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/IContext.java b/storm-core/src/jvm/backtype/storm/messaging/IContext.java
index 8645a6f..42f59bd 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/IContext.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/IContext.java
@@ -56,4 +56,4 @@
      * @return client side connection
      */
     public IConnection connect(String storm_id, String host, int port);
-};
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/TransportFactory.java b/storm-core/src/jvm/backtype/storm/messaging/TransportFactory.java
index 656b323..c551e0d 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/TransportFactory.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/TransportFactory.java
@@ -32,7 +32,7 @@
         String transport_plugin_klassName = (String)storm_conf.get(Config.STORM_MESSAGING_TRANSPORT);
         LOG.info("Storm peer transport plugin:"+transport_plugin_klassName);
 
-        IContext transport = null;
+        IContext transport;
         try {
             //create a factory class
             Class klass = Class.forName(transport_plugin_klassName);
diff --git a/storm-core/src/jvm/backtype/storm/messaging/local/Context.java b/storm-core/src/jvm/backtype/storm/messaging/local/Context.java
new file mode 100644
index 0000000..968fe64
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/local/Context.java
@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.local;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import backtype.storm.Config;
+import backtype.storm.grouping.Load;
+import backtype.storm.messaging.IConnection;
+import backtype.storm.messaging.TaskMessage;
+import backtype.storm.messaging.IConnectionCallback;
+import backtype.storm.messaging.IContext;
+
+public class Context implements IContext {
+    private static final Logger LOG = LoggerFactory.getLogger(Context.class);
+
+    private static class LocalServer implements IConnection {
+        IConnectionCallback _cb;
+        final ConcurrentHashMap<Integer, Double> _load = new ConcurrentHashMap<>();
+
+        @Override
+        public void registerRecv(IConnectionCallback cb) {
+            _cb = cb;
+        }
+
+        @Override
+        public void send(int taskId,  byte[] payload) {
+            throw new IllegalArgumentException("SHOULD NOT HAPPEN");
+        }
+ 
+        @Override
+        public void send(Iterator<TaskMessage> msgs) {
+            throw new IllegalArgumentException("SHOULD NOT HAPPEN");
+        }
+
+        @Override
+        public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
+            Map<Integer, Load> ret = new HashMap<>();
+            for (Integer task : tasks) {
+                Double found = _load.get(task);
+                if (found != null) {
+                    ret.put(task, new Load(true, found, 0));
+                }
+            }
+            return ret; 
+        }
+
+        @Override
+        public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
+            _load.putAll(taskToLoad);
+        }
+ 
+        @Override
+        public void close() {
+            //NOOP
+        }
+    };
+
+    private static class LocalClient implements IConnection {
+        private final LocalServer _server;
+
+        public LocalClient(LocalServer server) {
+            _server = server;
+        }
+
+        @Override
+        public void registerRecv(IConnectionCallback cb) {
+            throw new IllegalArgumentException("SHOULD NOT HAPPEN");
+        }
+
+        @Override
+        public void send(int taskId,  byte[] payload) {
+            if (_server._cb != null) {
+                _server._cb.recv(Arrays.asList(new TaskMessage(taskId, payload)));
+            }
+        }
+ 
+        @Override
+        public void send(Iterator<TaskMessage> msgs) {
+            if (_server._cb != null) {
+                ArrayList<TaskMessage> ret = new ArrayList<>();
+                while (msgs.hasNext()) {
+                    ret.add(msgs.next());
+                }
+                _server._cb.recv(ret);
+            }
+        }
+
+        @Override
+        public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
+            return _server.getLoad(tasks);
+        }
+
+        @Override
+        public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
+            _server.sendLoadMetrics(taskToLoad);
+        }
+ 
+        @Override
+        public void close() {
+            //NOOP
+        }
+    };
+
+    private static ConcurrentHashMap<String, LocalServer> _registry = new ConcurrentHashMap<>();
+    private static LocalServer getLocalServer(String nodeId, int port) {
+        String key = nodeId + "-" + port;
+        LocalServer ret = _registry.get(key);
+        if (ret == null) {
+            ret = new LocalServer();
+            LocalServer tmp = _registry.putIfAbsent(key, ret);
+            if (tmp != null) {
+                ret = tmp;
+            }
+        }
+        return ret;
+    }
+        
+    @SuppressWarnings("rawtypes")
+    @Override
+    public void prepare(Map storm_conf) {
+        //NOOP
+    }
+
+    @Override
+    public IConnection bind(String storm_id, int port) {
+        return getLocalServer(storm_id, port);
+    }
+
+    @Override
+    public IConnection connect(String storm_id, String host, int port) {
+        return new LocalClient(getLocalServer(storm_id, port));
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public void term() {
+        //NOOP
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java
index 8697907..4f813ba 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java
@@ -17,9 +17,22 @@
  */
 package backtype.storm.messaging.netty;
 
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Iterator;
+import java.util.Collection;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.lang.InterruptedException;
+
 import backtype.storm.Config;
+import backtype.storm.grouping.Load;
 import backtype.storm.messaging.ConnectionWithStatus;
 import backtype.storm.messaging.TaskMessage;
+import backtype.storm.messaging.IConnectionCallback;
 import backtype.storm.metric.api.IStatefulObject;
 import backtype.storm.utils.StormBoundedExponentialBackoffRetry;
 import backtype.storm.utils.Utils;
@@ -34,17 +47,11 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
+
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
 
 import static com.google.common.base.Preconditions.checkState;
 
@@ -60,7 +67,7 @@
  *     - Note: The current implementation drops any messages that are being enqueued for sending if the connection to
  *       the remote destination is currently unavailable.
  */
-public class Client extends ConnectionWithStatus implements IStatefulObject {
+public class Client extends ConnectionWithStatus implements IStatefulObject, ISaslClient {
     private static final long PENDING_MESSAGES_FLUSH_TIMEOUT_MS = 600000L;
     private static final long PENDING_MESSAGES_FLUSH_INTERVAL_MS = 1000L;
 
@@ -73,17 +80,12 @@
     private final ClientBootstrap bootstrap;
     private final InetSocketAddress dstAddress;
     protected final String dstAddressPrefixedName;
+    private volatile Map<Integer, Double> serverLoad = null;
 
     /**
      * The channel used for all write operations from this client to the remote destination.
      */
-    private final AtomicReference<Channel> channelRef = new AtomicReference<Channel>();
-
-
-    /**
-     * Maximum number of reconnection attempts we will perform after a disconnect before giving up.
-     */
-    private final int maxReconnectionAttempts;
+    private final AtomicReference<Channel> channelRef = new AtomicReference<>();
 
     /**
      * Total number of connection attempts.
@@ -110,6 +112,10 @@
      */
     private final AtomicLong pendingMessages = new AtomicLong(0);
 
+    /**
+     * Whether the SASL channel is ready.
+     */
+    private final AtomicBoolean saslChannelReady = new AtomicBoolean(false);
 
     /**
      * This flag is set to true if and only if a client instance is being closed.
@@ -131,28 +137,30 @@
         this.scheduler = scheduler;
         this.context = context;
         int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
+        // if SASL authentication is disabled, saslChannelReady is initialized as true; otherwise false
+        saslChannelReady.set(!Utils.getBoolean(stormConf.get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION), false));
         LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize);
         int messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144);
 
-        maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
+        int maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
         int minWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS));
         int maxWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS));
         retryPolicy = new StormBoundedExponentialBackoffRetry(minWaitMs, maxWaitMs, maxReconnectionAttempts);
 
         // Initiate connection to remote destination
-        bootstrap = createClientBootstrap(factory, bufferSize);
+        bootstrap = createClientBootstrap(factory, bufferSize, stormConf);
         dstAddress = new InetSocketAddress(host, port);
         dstAddressPrefixedName = prefixedName(dstAddress);
         scheduleConnect(NO_DELAY_MS);
         batcher = new MessageBuffer(messageBatchSize);
     }
 
-    private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) {
+    private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize, Map stormConf) {
         ClientBootstrap bootstrap = new ClientBootstrap(factory);
         bootstrap.setOption("tcpNoDelay", true);
         bootstrap.setOption("sendBufferSize", bufferSize);
         bootstrap.setOption("keepAlive", true);
-        bootstrap.setPipelineFactory(new StormClientPipelineFactory(this));
+        bootstrap.setPipelineFactory(new StormClientPipelineFactory(this, stormConf));
         return bootstrap;
     }
 
@@ -164,14 +172,14 @@
     }
 
     /**
-     * We will retry connection with exponential back-off policy
+     * Enqueue a task message to be sent to server
      */
     private void scheduleConnect(long delayMs) {
         scheduler.newTimeout(new Connect(dstAddress), delayMs, TimeUnit.MILLISECONDS);
     }
 
     private boolean reconnectingAllowed() {
-        return !closing && connectionAttempts.get() <= (maxReconnectionAttempts + 1);
+        return !closing;
     }
 
     private boolean connectionEstablished(Channel channel) {
@@ -196,7 +204,11 @@
         } else if (!connectionEstablished(channelRef.get())) {
             return Status.Connecting;
         } else {
-            return Status.Ready;
+            if (saslChannelReady.get()) {
+                return Status.Ready;
+            } else {
+                return Status.Connecting; // need to wait until sasl channel is also ready
+            }
         }
     }
 
@@ -206,11 +218,16 @@
      * @throws java.lang.UnsupportedOperationException whenever this method is being called.
      */
     @Override
-    public Iterator<TaskMessage> recv(int flags, int clientId) {
+    public void registerRecv(IConnectionCallback cb) {
         throw new UnsupportedOperationException("Client connection should not receive any messages");
     }
 
     @Override
+    public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
+        throw new RuntimeException("Client connection should not send load metrics");
+    }
+
+    @Override
     public void send(int taskId, byte[] payload) {
         TaskMessage msg = new TaskMessage(taskId, payload);
         List<TaskMessage> wrapper = new ArrayList<TaskMessage>(1);
@@ -289,11 +306,14 @@
         }
     }
 
+    public InetSocketAddress getDstAddress() {
+        return dstAddress;
+    }
+
     private boolean hasMessages(Iterator<TaskMessage> msgs) {
         return msgs != null && msgs.hasNext();
     }
 
-
     private void dropMessages(Iterator<TaskMessage> msgs) {
         // We consume the iterator by traversing and thus "emptying" it.
         int msgCount = iteratorSize(msgs);
@@ -389,15 +409,13 @@
                     break;
                 }
                 Thread.sleep(PENDING_MESSAGES_FLUSH_INTERVAL_MS);
-            }
-            catch (InterruptedException e) {
+            } catch (InterruptedException e) {
                 break;
             }
         }
 
     }
 
-
     private void closeChannel() {
         Channel channel = channelRef.get();
         if (channel != null) {
@@ -406,9 +424,29 @@
         }
     }
 
+    void setLoadMetrics(Map<Integer, Double> taskToLoad) {
+        this.serverLoad = taskToLoad;
+    }
+
+    @Override
+    public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
+        Map<Integer, Double> loadCache = serverLoad;
+        Map<Integer, Load> ret = new HashMap<Integer, Load>();
+        if (loadCache != null) {
+            double clientLoad = Math.min(pendingMessages.get(), 1024)/1024.0;
+            for (Integer task : tasks) {
+                Double found = loadCache.get(task);
+                if (found != null) {
+                    ret.put(task, new Load(true, found, clientLoad));
+                }
+            }
+        }
+        return ret;
+    }
+
     @Override
     public Object getState() {
-        LOG.info("Getting metrics for client connection to {}", dstAddressPrefixedName);
+        LOG.debug("Getting metrics for client connection to {}", dstAddressPrefixedName);
         HashMap<String, Object> ret = new HashMap<String, Object>();
         ret.put("reconnects", totalConnectionAttempts.getAndSet(0));
         ret.put("sent", messagesSent.getAndSet(0));
@@ -422,10 +460,28 @@
         return ret;
     }
 
-    public Map getStormConf() {
+    public Map getConfig() {
         return stormConf;
     }
 
+    /** ISaslClient interface **/
+    public void channelConnected(Channel channel) {
+//        setChannel(channel);
+    }
+
+    public void channelReady() {
+        saslChannelReady.set(true);
+    }
+
+    public String name() {
+        return (String)stormConf.get(Config.TOPOLOGY_NAME);
+    }
+
+    public String secretKey() {
+        return SaslUtils.getSecretKey(stormConf);
+    }
+    /** end **/
+
     private String srcAddressName() {
         String name = null;
         Channel channel = channelRef.get();
@@ -519,5 +575,4 @@
             }
         }
     }
-
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java
index 5d27a16..bc978b6 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java
@@ -19,12 +19,8 @@
 
 import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
 import org.jboss.netty.util.HashedWheelTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -34,8 +30,6 @@
 import backtype.storm.utils.Utils;
 
 public class Context implements IContext {
-    private static final Logger LOG = LoggerFactory.getLogger(Context.class);
-        
     @SuppressWarnings("rawtypes")
     private Map storm_conf;
     private Map<String, IConnection> connections;
@@ -49,7 +43,7 @@
     @SuppressWarnings("rawtypes")
     public void prepare(Map storm_conf) {
         this.storm_conf = storm_conf;
-        connections = new HashMap<String, IConnection>();
+        connections = new HashMap<>();
 
         //each context will have a single client channel factory
         int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS));
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java b/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java
index fb3efe6..7f60078 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/ControlMessage.java
@@ -23,7 +23,7 @@
 import org.jboss.netty.buffer.ChannelBufferOutputStream;
 import org.jboss.netty.buffer.ChannelBuffers;
 
-enum ControlMessage {
+public enum ControlMessage implements INettySerializable {
     CLOSE_MESSAGE((short)-100),
     EOB_MESSAGE((short)-201),
     OK_RESPONSE((short)-200),
@@ -39,33 +39,37 @@
     }
 
     /**
-     * Return a control message per an encoded status code
-     * @param encoded
-     * @return
+     * @param encoded status code
+     * @return a control message per an encoded status code
      */
-    static ControlMessage mkMessage(short encoded) {
+    public static ControlMessage mkMessage(short encoded) {
         for(ControlMessage cm: ControlMessage.values()) {
           if(encoded == cm.code) return cm;
         }
         return null;
     }
 
-    int encodeLength() {
+    public int encodeLength() {
         return 2; //short
     }
     
     /**
      * encode the current Control Message into a channel buffer
-     * @throws Exception
+     * @throws IOException
      */
-    ChannelBuffer buffer() throws IOException {
+    public ChannelBuffer buffer() throws IOException {
         ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encodeLength()));      
         write(bout);
         bout.close();
         return bout.buffer();
     }
 
-    void write(ChannelBufferOutputStream bout) throws IOException {
+    public static ControlMessage read(byte[] serial) {
+        ChannelBuffer cm_buffer = ChannelBuffers.copiedBuffer(serial);
+        return mkMessage(cm_buffer.getShort(0));
+    }
+    
+    public void write(ChannelBufferOutputStream bout) throws IOException {
         bout.writeShort(code);        
     } 
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/INettySerializable.java b/storm-core/src/jvm/backtype/storm/messaging/netty/INettySerializable.java
new file mode 100644
index 0000000..6cc6de2
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/INettySerializable.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import java.io.IOException;
+import org.jboss.netty.buffer.ChannelBuffer;
+
+public interface INettySerializable {
+    ChannelBuffer buffer() throws IOException;
+    int encodeLength();
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/ISaslClient.java b/storm-core/src/jvm/backtype/storm/messaging/netty/ISaslClient.java
new file mode 100644
index 0000000..57dcfe8
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/ISaslClient.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import backtype.storm.Config;
+
+public interface ISaslClient {
+    void channelConnected(Channel channel);
+    void channelReady();
+    String name();
+    String secretKey();
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/ISaslServer.java b/storm-core/src/jvm/backtype/storm/messaging/netty/ISaslServer.java
new file mode 100644
index 0000000..4203dcc
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/ISaslServer.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+
+public interface ISaslServer extends IServer {
+    String name();
+    String secretKey();
+    void authenticated(Channel c);
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/IServer.java b/storm-core/src/jvm/backtype/storm/messaging/netty/IServer.java
new file mode 100644
index 0000000..d046492
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/IServer.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+
+public interface IServer {
+    void channelConnected(Channel c);
+    void received(Object message, String remote, Channel channel) throws InterruptedException;
+    void closeChannel(Channel c);
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslClientHandler.java
new file mode 100644
index 0000000..698800d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslClientHandler.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import java.io.IOException;
+import java.util.Map;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KerberosSaslClientHandler extends SimpleChannelUpstreamHandler {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslClientHandler.class);
+    private ISaslClient client;
+    long start_time;
+    /** Used for client or server's token to send or receive from each other. */
+    private Map storm_conf;
+    private String jaas_section;
+
+    public KerberosSaslClientHandler(ISaslClient client, Map storm_conf, String jaas_section) throws IOException {
+        this.client = client;
+        this.storm_conf = storm_conf;
+        this.jaas_section = jaas_section;
+        start_time = System.currentTimeMillis();
+    }
+
+    @Override
+    public void channelConnected(ChannelHandlerContext ctx,
+                                 ChannelStateEvent event) {
+        // register the newly established channel
+        Channel channel = ctx.getChannel();
+        client.channelConnected(channel);
+
+        LOG.info("Connection established from {} to {}",
+                 channel.getLocalAddress(), channel.getRemoteAddress());
+
+        try {
+            KerberosSaslNettyClient saslNettyClient = KerberosSaslNettyClientState.getKerberosSaslNettyClient
+                .get(channel);
+
+            if (saslNettyClient == null) {
+                LOG.debug("Creating saslNettyClient now for channel: {}",
+                          channel);
+                saslNettyClient = new KerberosSaslNettyClient(storm_conf, jaas_section);
+                KerberosSaslNettyClientState.getKerberosSaslNettyClient.set(channel,
+                                                                            saslNettyClient);
+            }
+            LOG.debug("Going to initiate Kerberos negotiations.");
+            byte[] initialChallenge = saslNettyClient.saslResponse(new SaslMessageToken(new byte[0]));
+            LOG.debug("Sending initial challenge: {}", initialChallenge);
+            channel.write(new SaslMessageToken(initialChallenge));
+        } catch (Exception e) {
+            LOG.error("Failed to authenticate with server due to error: ",
+                      e);
+        }
+        return;
+
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent event)
+        throws Exception {
+        LOG.debug("send/recv time (ms): {}",
+                  (System.currentTimeMillis() - start_time));
+
+        Channel channel = ctx.getChannel();
+
+        // Generate SASL response to server using Channel-local SASL client.
+        KerberosSaslNettyClient saslNettyClient = KerberosSaslNettyClientState.getKerberosSaslNettyClient
+            .get(channel);
+        if (saslNettyClient == null) {
+            throw new Exception("saslNettyClient was unexpectedly null for channel:" + channel);
+        }
+
+        // examine the response message from server
+        if (event.getMessage() instanceof ControlMessage) {
+            ControlMessage msg = (ControlMessage) event.getMessage();
+            if (msg == ControlMessage.SASL_COMPLETE_REQUEST) {
+                LOG.debug("Server has sent us the SaslComplete message. Allowing normal work to proceed.");
+
+                if (!saslNettyClient.isComplete()) {
+                    String message = "Server returned a Sasl-complete message, but as far as we can tell, we are not authenticated yet.";
+                    LOG.error(message);
+                    throw new Exception(message);
+                }
+                ctx.getPipeline().remove(this);
+                this.client.channelReady();
+
+                // We call fireMessageReceived since the client is allowed to
+                // perform this request. The client's request will now proceed
+                // to the next pipeline component namely StormClientHandler.
+                Channels.fireMessageReceived(ctx, msg);
+            } else {
+                LOG.warn("Unexpected control message: {}", msg);
+            }
+            return;
+        }
+        else if (event.getMessage() instanceof SaslMessageToken) {
+            SaslMessageToken saslTokenMessage = (SaslMessageToken) event
+                .getMessage();
+            LOG.debug("Responding to server's token of length: {}",
+                      saslTokenMessage.getSaslToken().length);
+
+            // Generate SASL response (but we only actually send the response if
+            // it's non-null.
+            byte[] responseToServer = saslNettyClient
+                .saslResponse(saslTokenMessage);
+            if (responseToServer == null) {
+                // If we generate a null response, then authentication has completed
+                // (if not, warn), and return without sending a response back to the
+                // server.
+                LOG.debug("Response to server is null: authentication should now be complete.");
+                if (!saslNettyClient.isComplete()) {
+                    LOG.warn("Generated a null response, but authentication is not complete.");
+                    throw new Exception("Our reponse to the server is null, but as far as we can tell, we are not authenticated yet.");
+                }
+                this.client.channelReady();
+                return;
+            } else {
+                LOG.debug("Response to server token has length: {}",
+                          responseToServer.length);
+            }
+            // Construct a message containing the SASL response and send it to the
+            // server.
+            SaslMessageToken saslResponse = new SaslMessageToken(responseToServer);
+            channel.write(saslResponse);
+        } else {
+            LOG.error("Unexpected message from server: {}", event.getMessage());
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyClient.java b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyClient.java
new file mode 100644
index 0000000..90dc4b6
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyClient.java
@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import backtype.storm.Config;
+import backtype.storm.security.auth.AuthUtils;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Map;
+import java.util.TreeMap;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import org.apache.zookeeper.Login;
+import org.apache.zookeeper.server.auth.KerberosName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements SASL logic for storm worker client processes.
+ */
+public class KerberosSaslNettyClient {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslNettyClient.class);
+
+    /**
+     * Used to respond to server's counterpart, SaslServer with SASL tokens
+     * represented as byte arrays.
+     */
+    private SaslClient saslClient;
+    private Subject subject;
+    private String jaas_section;
+
+    /**
+     * Create a KerberosSaslNettyClient for authentication with servers.
+     */
+    public KerberosSaslNettyClient(Map storm_conf, String jaas_section) {
+        LOG.debug("KerberosSaslNettyClient: Creating SASL {} client to authenticate to server ",
+                  SaslUtils.KERBEROS);
+
+        LOG.info("Creating Kerberos Client.");
+
+        Configuration login_conf;
+        try {
+            login_conf = AuthUtils.GetConfiguration(storm_conf);
+        }
+        catch (Throwable t) {
+            LOG.error("Failed to get login_conf: ", t);
+            throw t;
+        }
+        LOG.debug("KerberosSaslNettyClient: authmethod {}", SaslUtils.KERBEROS);
+
+        SaslClientCallbackHandler ch = new SaslClientCallbackHandler();
+
+        subject = null;
+        try {
+            LOG.debug("Setting Configuration to login_config: {}", login_conf);
+            //specify a configuration object to be used
+            Configuration.setConfiguration(login_conf);
+            //now login
+            LOG.debug("Trying to login.");
+            Login login = new Login(jaas_section, ch);
+            subject = login.getSubject();
+            LOG.debug("Got Subject: {}", subject.toString());
+        } catch (LoginException ex) {
+            LOG.error("Client failed to login in principal:" + ex, ex);
+            throw new RuntimeException(ex);
+        }
+
+        //check the credential of our principal
+        if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) {
+            LOG.error("Failed to verify user principal.");
+            throw new RuntimeException("Fail to verify user principal with section \"" +
+                                       jaas_section +
+                                       "\" in login configuration file " +
+                                       login_conf);
+        }
+
+        String serviceName = null;
+        try {
+            serviceName = AuthUtils.get(login_conf, jaas_section, "serviceName");
+        }
+        catch (IOException e) {
+            LOG.error("Failed to get service name.", e);
+            throw new RuntimeException(e);
+        }
+
+        try {
+            Principal principal = (Principal)subject.getPrincipals().toArray()[0];
+            final String fPrincipalName = principal.getName();
+            final String fHost = (String)storm_conf.get(Config.PACEMAKER_HOST);
+            final String fServiceName = serviceName;
+            final CallbackHandler fch = ch;
+            LOG.debug("Kerberos Client with principal: {}, host: {}", fPrincipalName, fHost);
+            saslClient = Subject.doAs(subject, new PrivilegedExceptionAction<SaslClient>() {
+                    public SaslClient run() {
+                        try {
+                            Map<String, String> props = new TreeMap<String,String>();
+                            props.put(Sasl.QOP, "auth");
+                            props.put(Sasl.SERVER_AUTH, "false");
+                            return Sasl.createSaslClient(
+                                new String[] { SaslUtils.KERBEROS },
+                                fPrincipalName,
+                                fServiceName,
+                                fHost,
+                                props, fch);
+                        }
+                        catch (Exception e) {
+                            LOG.error("Subject failed to create sasl client.", e);
+                            return null;
+                        }
+                    }
+                });
+            LOG.info("Got Client: {}", saslClient);
+
+        } catch (PrivilegedActionException e) {
+            LOG.error("KerberosSaslNettyClient: Could not create Sasl Netty Client.");
+            throw new RuntimeException(e);
+        }
+    }
+
+    public boolean isComplete() {
+        return saslClient.isComplete();
+    }
+
+    /**
+     * Respond to server's SASL token.
+     *
+     * @param saslTokenMessage
+     *            contains server's SASL token
+     * @return client's response SASL token
+     */
+    public byte[] saslResponse(SaslMessageToken saslTokenMessage) {
+        try {
+            final SaslMessageToken fSaslTokenMessage = saslTokenMessage;
+            byte [] retval = Subject.doAs(subject, new PrivilegedExceptionAction<byte[]>() {
+                    public byte[] run() {
+                        try {
+                            byte[] retval = saslClient.evaluateChallenge(fSaslTokenMessage
+                                                                         .getSaslToken());
+                            return retval;
+                        } catch (SaslException e) {
+                            LOG.error("saslResponse: Failed to respond to SASL server's token:",
+                                      e);
+                            throw new RuntimeException(e);
+                        }
+                    }
+                });
+            return retval;
+        }
+        catch (PrivilegedActionException e) {
+            LOG.error("Failed to generate response for token: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Implementation of javax.security.auth.callback.CallbackHandler that works
+     * with Storm topology tokens.
+     */
+    private static class SaslClientCallbackHandler implements CallbackHandler {
+
+        /**
+         * Implementation used to respond to SASL tokens from server.
+         *
+         * @param callbacks
+         *            objects that indicate what credential information the
+         *            server's SaslServer requires from the client.
+         * @throws UnsupportedCallbackException
+         */
+        public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                LOG.info("Kerberos Client Callback Handler got callback: {}", callback.getClass());
+            }
+        }
+    }
+
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyClientState.java b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyClientState.java
new file mode 100644
index 0000000..2546aa5
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyClientState.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelLocal;
+
+final class KerberosSaslNettyClientState {
+
+    public static final ChannelLocal<KerberosSaslNettyClient> getKerberosSaslNettyClient = new ChannelLocal<KerberosSaslNettyClient>() {
+        protected KerberosSaslNettyClient initialValue(Channel channel) {
+            return null;
+        }
+    };
+
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyServer.java b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyServer.java
new file mode 100644
index 0000000..a16c01b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyServer.java
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import backtype.storm.security.auth.AuthUtils;
+import backtype.storm.security.auth.KerberosPrincipalToLocal;
+import java.io.IOException;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import org.apache.zookeeper.Login;
+import org.apache.zookeeper.server.auth.KerberosName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+class KerberosSaslNettyServer {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslNettyServer.class);
+
+    private SaslServer saslServer;
+    private Subject subject;
+    private List<String> authorizedUsers;
+
+    KerberosSaslNettyServer(Map storm_conf, String jaas_section, List<String> authorizedUsers) {
+        this.authorizedUsers = authorizedUsers;
+        LOG.debug("Getting Configuration.");
+        Configuration login_conf;
+        try {
+            login_conf = AuthUtils.GetConfiguration(storm_conf);
+        }
+        catch (Throwable t) {
+            LOG.error("Failed to get login_conf: ", t);
+            throw t;
+        }
+
+        LOG.debug("KerberosSaslNettyServer: authmethod {}", SaslUtils.KERBEROS);
+
+        KerberosSaslCallbackHandler ch = new KerberosSaslNettyServer.KerberosSaslCallbackHandler(authorizedUsers);
+
+        //login our principal
+        subject = null;
+        try {
+            LOG.debug("Setting Configuration to login_config: {}", login_conf);
+            //specify a configuration object to be used
+            Configuration.setConfiguration(login_conf);
+            //now login
+            LOG.debug("Trying to login.");
+            Login login = new Login(jaas_section, ch);
+            subject = login.getSubject();
+            LOG.debug("Got Subject: {}", subject.toString());
+        } catch (LoginException ex) {
+            LOG.error("Server failed to login in principal:", ex);
+            throw new RuntimeException(ex);
+        }
+
+        //check the credential of our principal
+        if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) {
+            LOG.error("Failed to verifyuser principal.");
+            throw new RuntimeException("Fail to verify user principal with section \""
+                                       + jaas_section
+                                       + "\" in login configuration file "
+                                       + login_conf);
+        }
+
+        try {
+            LOG.info("Creating Kerberos Server.");
+            final CallbackHandler fch = ch;
+            Principal p = (Principal)subject.getPrincipals().toArray()[0];
+            KerberosName kName = new KerberosName(p.getName());
+            final String fHost = kName.getHostName();
+            final String fServiceName = kName.getServiceName();
+            LOG.debug("Server with host: {}", fHost);
+            saslServer =
+                Subject.doAs(subject, new PrivilegedExceptionAction<SaslServer>() {
+                        public SaslServer run() {
+                            try {
+                                Map<String, String> props = new TreeMap<String,String>();
+                                props.put(Sasl.QOP, "auth");
+                                props.put(Sasl.SERVER_AUTH, "false");
+                                return Sasl.createSaslServer(SaslUtils.KERBEROS,
+                                                             fServiceName,
+                                                             fHost, props, fch);
+                            }
+                            catch (Exception e) {
+                                LOG.error("Subject failed to create sasl server.", e);
+                                return null;
+                            }
+                        }
+                    });
+            LOG.info("Got Server: {}", saslServer);
+
+        } catch (PrivilegedActionException e) {
+            LOG.error("KerberosSaslNettyServer: Could not create SaslServer: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public boolean isComplete() {
+        return saslServer.isComplete();
+    }
+
+    public String getUserName() {
+        return saslServer.getAuthorizationID();
+    }
+
+    /** CallbackHandler for SASL DIGEST-MD5 mechanism */
+    public static class KerberosSaslCallbackHandler implements CallbackHandler {
+
+        /** Used to authenticate the clients */
+        private List<String> authorizedUsers;
+
+        public KerberosSaslCallbackHandler(List<String> authorizedUsers) {
+            LOG.debug("KerberosSaslCallback: Creating KerberosSaslCallback handler.");
+            this.authorizedUsers = authorizedUsers;
+        }
+
+        @Override
+        public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+            for (Callback callback : callbacks) {
+                LOG.info("Kerberos Callback Handler got callback: {}", callback.getClass());
+                if(callback instanceof AuthorizeCallback) {
+                    AuthorizeCallback ac = (AuthorizeCallback)callback;
+                    if(!ac.getAuthenticationID().equals(ac.getAuthorizationID())) {
+                        LOG.debug("{} != {}", ac.getAuthenticationID(), ac.getAuthorizationID());
+                        continue;
+                    }
+
+                    LOG.debug("Authorized Users: {}", authorizedUsers);
+                    LOG.debug("Checking authorization for: {}", ac.getAuthorizationID());
+                    for(String user : authorizedUsers) {
+                        String requester = ac.getAuthorizationID();
+
+                        KerberosPrincipal principal = new KerberosPrincipal(requester);
+                        requester = new KerberosPrincipalToLocal().toLocal(principal);
+
+                        if(requester.equals(user) ) {
+                            ac.setAuthorized(true);
+                            break;
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Used by SaslTokenMessage::processToken() to respond to server SASL
+     * tokens.
+     *
+     * @param token
+     *            Server's SASL token
+     * @return token to send back to the server.
+     */
+    public byte[] response(final byte[] token) {
+        try {
+            byte [] retval = Subject.doAs(subject, new PrivilegedExceptionAction<byte[]>() {
+                    public byte[] run(){
+                        try {
+                            LOG.debug("response: Responding to input token of length: {}",
+                                      token.length);
+                            byte[] retval = saslServer.evaluateResponse(token);
+                            return retval;
+                        } catch (SaslException e) {
+                            LOG.error("response: Failed to evaluate client token of length: {} : {}",
+                                      token.length, e);
+                            throw new RuntimeException(e);
+                        }
+                    }
+                });
+            return retval;
+        }
+        catch (PrivilegedActionException e) {
+            LOG.error("Failed to generate response for token: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyServerState.java b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyServerState.java
new file mode 100644
index 0000000..e7a127e
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslNettyServerState.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelLocal;
+
+final class KerberosSaslNettyServerState {
+
+    public static final ChannelLocal<KerberosSaslNettyServer> getKerberosSaslNettyServer = new ChannelLocal<KerberosSaslNettyServer>() {
+            protected KerberosSaslNettyServer initialValue(Channel channel) {
+                return null;
+            }
+        };
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslServerHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslServerHandler.java
new file mode 100644
index 0000000..c31da17
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/KerberosSaslServerHandler.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
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KerberosSaslServerHandler extends SimpleChannelUpstreamHandler {
+
+    ISaslServer server;
+    /** Used for client or server's token to send or receive from each other. */
+    private Map storm_conf;
+    private String jaas_section;
+    private List<String> authorizedUsers;
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(KerberosSaslServerHandler.class);
+
+    public KerberosSaslServerHandler(ISaslServer server, Map storm_conf, String jaas_section, List<String> authorizedUsers) throws IOException {
+        this.server = server;
+        this.storm_conf = storm_conf;
+        this.jaas_section = jaas_section;
+        this.authorizedUsers = authorizedUsers;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+        throws Exception {
+        Object msg = e.getMessage();
+        if (msg == null) {
+            return;
+        }
+
+        Channel channel = ctx.getChannel();
+
+
+        if (msg instanceof SaslMessageToken) {
+            // initialize server-side SASL functionality, if we haven't yet
+            // (in which case we are looking at the first SASL message from the
+            // client).
+            try {
+                LOG.debug("Got SaslMessageToken!");
+
+                KerberosSaslNettyServer saslNettyServer = KerberosSaslNettyServerState.getKerberosSaslNettyServer
+                    .get(channel);
+                if (saslNettyServer == null) {
+                    LOG.debug("No saslNettyServer for {}  yet; creating now, with topology token: ", channel);
+                    try {
+                        saslNettyServer = new KerberosSaslNettyServer(storm_conf, jaas_section, authorizedUsers);
+                        KerberosSaslNettyServerState.getKerberosSaslNettyServer.set(channel,
+                                                                                    saslNettyServer);
+                    } catch (RuntimeException ioe) {
+                        LOG.error("Error occurred while creating saslNettyServer on server {} for client {}",
+                                  channel.getLocalAddress(), channel.getRemoteAddress());
+                        throw ioe;
+                    }
+                } else {
+                    LOG.debug("Found existing saslNettyServer on server: {} for client {}",
+                              channel.getLocalAddress(), channel.getRemoteAddress());
+                }
+
+                byte[] responseBytes = saslNettyServer.response(((SaslMessageToken) msg)
+                                                                .getSaslToken());
+
+                SaslMessageToken saslTokenMessageRequest = new SaslMessageToken(responseBytes);
+
+                if(saslTokenMessageRequest.getSaslToken() == null) {
+                    channel.write(ControlMessage.SASL_COMPLETE_REQUEST);
+                } else {
+                    // Send response to client.
+                    channel.write(saslTokenMessageRequest);
+                }
+
+                if (saslNettyServer.isComplete()) {
+                    // If authentication of client is complete, we will also send a
+                    // SASL-Complete message to the client.
+                    LOG.info("SASL authentication is complete for client with username: {}",
+                             saslNettyServer.getUserName());
+                    channel.write(ControlMessage.SASL_COMPLETE_REQUEST);
+                    LOG.debug("Removing SaslServerHandler from pipeline since SASL authentication is complete.");
+                    ctx.getPipeline().remove(this);
+                    server.authenticated(channel);
+                }
+                return;
+            }
+            catch (Exception ex) {
+                LOG.error("Failed to handle SaslMessageToken: ", ex);
+                throw ex;
+            }
+        } else {
+            // Client should not be sending other-than-SASL messages before
+            // SaslServerHandler has removed itself from the pipeline. Such
+            // non-SASL requests will be denied by the Authorize channel handler
+            // (the next handler upstream in the server pipeline) if SASL
+            // authentication has not completed.
+            LOG.warn("Sending upstream an unexpected non-SASL message : {}",
+                     msg);
+            Channels.fireMessageReceived(ctx, msg);
+        }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
+        if(server != null) {
+            server.closeChannel(e.getChannel());
+        }
+    }
+
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java
index ec0dc0f..fea33fd 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java
@@ -31,7 +31,7 @@
 
     MessageBatch(int buffer_size) {
         this.buffer_size = buffer_size;
-        msgs = new ArrayList<TaskMessage>();
+        msgs = new ArrayList<>();
         encoded_length = ControlMessage.EOB_MESSAGE.encodeLength();
     }
 
@@ -54,24 +54,21 @@
     }
 
     /**
-     * Has this batch used up allowed buffer size
-     * @return
+     * @return true if this batch used up allowed buffer size
      */
     boolean isFull() {
         return encoded_length >= buffer_size;
     }
 
     /**
-     * true if this batch doesn't have any messages 
-     * @return
+     * @return true if this batch doesn't have any messages
      */
     boolean isEmpty() {
         return msgs.isEmpty();
     }
 
     /**
-     * # of msgs in this batch
-     * @return
+     * @return number of msgs in this batch
      */
     int size() {
         return msgs.size();
@@ -83,8 +80,9 @@
     ChannelBuffer buffer() throws Exception {
         ChannelBufferOutputStream bout = new ChannelBufferOutputStream(ChannelBuffers.directBuffer(encoded_length));
         
-        for (TaskMessage msg : msgs)
+        for (TaskMessage msg : msgs) {
             writeTaskMessage(bout, msg);
+        }
 
         //add a END_OF_BATCH indicator
         ControlMessage.EOB_MESSAGE.write(bout);
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java
index 7d8bf54..397ec21 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageDecoder.java
@@ -43,7 +43,7 @@
             return null;
         }
 
-        List<Object> ret = new ArrayList<Object>();
+        List<Object> ret = new ArrayList<>();
 
         // Use while loop, try to decode as more messages as possible in single call
         while (available >= 2) {
@@ -70,7 +70,7 @@
             }
             
             //case 2: SaslTokenMessageRequest
-            if(code==-500) {
+            if(code == SaslMessageToken.IDENTIFIER) {
             	// Make sure that we have received at least an integer (length) 
                 if (buf.readableBytes() < 4) {
                     //need more data
@@ -100,7 +100,6 @@
             }
 
             // case 3: task Message
-            short task = code;
 
             // Make sure that we have received at least an integer (length)
             if (available < 4) {
@@ -115,7 +114,7 @@
             available -= 4;
 
             if (length <= 0) {
-                ret.add(new TaskMessage(task, null));
+                ret.add(new TaskMessage(code, null));
                 break;
             }
 
@@ -133,7 +132,7 @@
 
             // Successfully decoded a frame.
             // Return a TaskMessage object
-            ret.add(new TaskMessage(task, payload.array()));
+            ret.add(new TaskMessage(code, payload.array()));
         }
 
         if (ret.size() == 0) {
@@ -142,4 +141,4 @@
             return ret;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
index 3a91a58..2a1cdea 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyRenameThreadFactory.java
@@ -33,8 +33,9 @@
     final ThreadGroup group;
     final AtomicInteger index = new AtomicInteger(1);
     final String name;
+    static final NettyUncaughtExceptionHandler uncaughtExceptionHandler = new NettyUncaughtExceptionHandler();
 
-    NettyRenameThreadFactory(String name) {
+    public NettyRenameThreadFactory(String name) {
         SecurityManager s = System.getSecurityManager();
         group = (s != null)? s.getThreadGroup() :
                              Thread.currentThread().getThreadGroup();
@@ -43,10 +44,13 @@
 
     public Thread newThread(Runnable r) {
         Thread t = new Thread(group, r, name + "-" + index.getAndIncrement(), 0);
-        if (t.isDaemon())
+        if (t.isDaemon()) {
             t.setDaemon(false);
-        if (t.getPriority() != Thread.NORM_PRIORITY)
+        }
+        if (t.getPriority() != Thread.NORM_PRIORITY) {
             t.setPriority(Thread.NORM_PRIORITY);
+        }
+        t.setUncaughtExceptionHandler(uncaughtExceptionHandler);
         return t;
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/NettyUncaughtExceptionHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyUncaughtExceptionHandler.java
new file mode 100644
index 0000000..ad8b5d9
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/NettyUncaughtExceptionHandler.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.messaging.netty;
+
+import backtype.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class NettyUncaughtExceptionHandler implements Thread.UncaughtExceptionHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(NettyUncaughtExceptionHandler.class);
+    @Override
+    public void uncaughtException(Thread t, Throwable e) {
+        try {
+            Utils.handleUncaughtException(e);
+        } catch (Error error) {
+            LOG.info("Received error in netty thread.. terminating server...");
+            Runtime.getRuntime().exit(1);
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java
index d0d3ca1..8a133d1 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslMessageToken.java
@@ -17,6 +17,7 @@
  */
 package backtype.storm.messaging.netty;
 
+import java.io.IOException;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBufferOutputStream;
 import org.jboss.netty.buffer.ChannelBuffers;
@@ -26,7 +27,9 @@
 /**
  * Send and receive SASL tokens.
  */
-public class SaslMessageToken {
+public class SaslMessageToken implements INettySerializable {
+    public static final short IDENTIFIER = -500;
+
     /** Class logger */
     private static final Logger LOG = LoggerFactory
             .getLogger(SaslMessageToken.class);
@@ -69,31 +72,43 @@
         this.token = token;
     }
 
-    int encodeLength() {
+    public int encodeLength() {
         return 2 + 4 + token.length;
     }
 
     /**
      * encode the current SaslToken Message into a channel buffer
-     * SaslTokenMessageRequest is encoded as: identifier .... short(2) always it
-     * is -500 payload length .... int payload .... byte[]
+     * SaslTokenMessageRequest is encoded as: identifier .... short(2)
+     * payload length .... int payload .... byte[]
      * 
-     * @throws Exception
+     * @throws IOException
      */
-    ChannelBuffer buffer() throws Exception {
+    public ChannelBuffer buffer() throws IOException {
         ChannelBufferOutputStream bout = new ChannelBufferOutputStream(
                 ChannelBuffers.directBuffer(encodeLength()));
-        short identifier = -500;
         int payload_len = 0;
         if (token != null)
             payload_len = token.length;
 
-        bout.writeShort((short) identifier);
-        bout.writeInt((int) payload_len);
+        bout.writeShort(IDENTIFIER);
+        bout.writeInt(payload_len);
+
         if (payload_len > 0) {
             bout.write(token);
         }
         bout.close();
         return bout.buffer();
     }
+    
+    public static SaslMessageToken read(byte[] serial) {
+        ChannelBuffer sm_buffer = ChannelBuffers.copiedBuffer(serial);
+        short identifier = sm_buffer.readShort();
+        int payload_len = sm_buffer.readInt();
+        if(identifier != IDENTIFIER) {
+            return null;
+        }
+        byte token[] = new byte[payload_len];
+        sm_buffer.readBytes(token, 0, payload_len);
+        return new SaslMessageToken(token);
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java
index 023e950..4f579fb 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyClient.java
@@ -52,9 +52,8 @@
      */
     public SaslNettyClient(String topologyName, byte[] token) {
         try {
-            LOG.debug("SaslNettyClient: Creating SASL "
-                    + SaslUtils.AUTH_DIGEST_MD5
-                    + " client to authenticate to server ");
+            LOG.debug("SaslNettyClient: Creating SASL {} client to authenticate to server ",
+                      SaslUtils.AUTH_DIGEST_MD5);
 
             saslClient = Sasl.createSaslClient(
                     new String[] { SaslUtils.AUTH_DIGEST_MD5 }, null, null,
@@ -81,9 +80,7 @@
      */
     public byte[] saslResponse(SaslMessageToken saslTokenMessage) {
         try {
-            byte[] retval = saslClient.evaluateChallenge(saslTokenMessage
-                    .getSaslToken());
-            return retval;
+            return saslClient.evaluateChallenge(saslTokenMessage.getSaslToken());
         } catch (SaslException e) {
             LOG.error(
                     "saslResponse: Failed to respond to SASL server's token:",
@@ -104,8 +101,6 @@
 
         /**
          * Set private members using topology token.
-         * 
-         * @param topologyToken
          */
         public SaslClientCallbackHandler(String topologyToken, byte[] token) {
             this.userName = SaslUtils
@@ -141,26 +136,19 @@
                 }
             }
             if (nc != null) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("handle: SASL client callback: setting username: "
-                            + userName);
-                }
+                LOG.debug("handle: SASL client callback: setting username: {}",
+                          userName);
                 nc.setName(userName);
             }
             if (pc != null) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("handle: SASL client callback: setting userPassword");
-                }
+                LOG.debug("handle: SASL client callback: setting userPassword");
                 pc.setPassword(userPassword);
             }
             if (rc != null) {
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("handle: SASL client callback: setting realm: "
-                            + rc.getDefaultText());
-                }
+                LOG.debug("handle: SASL client callback: setting realm: {}",
+                        rc.getDefaultText());
                 rc.setText(rc.getDefaultText());
             }
         }
     }
-
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java
index 2cb47d9..887a6c6 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServer.java
@@ -18,9 +18,6 @@
 package backtype.storm.messaging.netty;
 
 import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.HashMap;
-import java.util.Map;
 
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
@@ -33,133 +30,128 @@
 import javax.security.sasl.SaslException;
 import javax.security.sasl.SaslServer;
 
-import org.apache.commons.codec.binary.Base64;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 class SaslNettyServer {
 
-	private static final Logger LOG = LoggerFactory
-			.getLogger(SaslNettyServer.class);
+    private static final Logger LOG = LoggerFactory
+        .getLogger(SaslNettyServer.class);
 
-	private SaslServer saslServer;
+        private SaslServer saslServer;
 
-	SaslNettyServer(String topologyName, byte[] token) throws IOException {
-		LOG.debug("SaslNettyServer: Topology token is: " + topologyName
-				+ " with authmethod " + SaslUtils.AUTH_DIGEST_MD5);
+    SaslNettyServer(String topologyName, byte[] token) throws IOException {
+        LOG.debug("SaslNettyServer: Topology token is: {} with authmethod {}",
+                  topologyName, SaslUtils.AUTH_DIGEST_MD5);
 
-		try {
+        try {
+            SaslDigestCallbackHandler ch = new SaslNettyServer.SaslDigestCallbackHandler(
+                topologyName, token);
 
-			SaslDigestCallbackHandler ch = new SaslNettyServer.SaslDigestCallbackHandler(
-					topologyName, token);
+            saslServer = Sasl.createSaslServer(SaslUtils.AUTH_DIGEST_MD5, null,
+                                               SaslUtils.DEFAULT_REALM,
+                                               SaslUtils.getSaslProps(), ch);
+        } catch (SaslException e) {
+            LOG.error("SaslNettyServer: Could not create SaslServer: ", e);
+        }
+    }
 
-			saslServer = Sasl.createSaslServer(SaslUtils.AUTH_DIGEST_MD5, null,
-					SaslUtils.DEFAULT_REALM, SaslUtils.getSaslProps(), ch);
+    public boolean isComplete() {
+        return saslServer.isComplete();
+    }
 
-		} catch (SaslException e) {
-			LOG.error("SaslNettyServer: Could not create SaslServer: " + e);
-		}
+    public String getUserName() {
+        return saslServer.getAuthorizationID();
+    }
 
-	}
+    /** CallbackHandler for SASL DIGEST-MD5 mechanism */
+    public static class SaslDigestCallbackHandler implements CallbackHandler {
 
-	public boolean isComplete() {
-		return saslServer.isComplete();
-	}
+        /** Used to authenticate the clients */
+        private byte[] userPassword;
+        private String userName;
 
-	public String getUserName() {
-		return saslServer.getAuthorizationID();
-	}
+        public SaslDigestCallbackHandler(String topologyName, byte[] token) {
+            LOG.debug("SaslDigestCallback: Creating SaslDigestCallback handler with topology token: {}", topologyName);
+            this.userName = topologyName;
+            this.userPassword = token;
+        }
 
-	/** CallbackHandler for SASL DIGEST-MD5 mechanism */
-	public static class SaslDigestCallbackHandler implements CallbackHandler {
+        @Override
+        public void handle(Callback[] callbacks) throws IOException,
+            UnsupportedCallbackException {
+            NameCallback nc = null;
+            PasswordCallback pc = null;
+            AuthorizeCallback ac = null;
 
-		/** Used to authenticate the clients */
-		private byte[] userPassword;
-		private String userName;
+            for (Callback callback : callbacks) {
+                if (callback instanceof AuthorizeCallback) {
+                    ac = (AuthorizeCallback) callback;
+                } else if (callback instanceof NameCallback) {
+                    nc = (NameCallback) callback;
+                } else if (callback instanceof PasswordCallback) {
+                    pc = (PasswordCallback) callback;
+                } else if (callback instanceof RealmCallback) {
+                    continue; // realm is ignored
+                } else {
+                    throw new UnsupportedCallbackException(callback,
+                                                           "handle: Unrecognized SASL DIGEST-MD5 Callback");
+                }
+            }
 
-		public SaslDigestCallbackHandler(String topologyName, byte[] token) {
-			LOG.debug("SaslDigestCallback: Creating SaslDigestCallback handler "
-					+ "with topology token: " + topologyName);
-			this.userName = topologyName;
-			this.userPassword = token;
-		}
+            if (nc != null) {
+                LOG.debug("handle: SASL server DIGEST-MD5 callback: setting username for client: {}",
+                          userName);
+                nc.setName(userName);
+            }
 
-		@Override
-		public void handle(Callback[] callbacks) throws IOException,
-				UnsupportedCallbackException {
-			NameCallback nc = null;
-			PasswordCallback pc = null;
-			AuthorizeCallback ac = null;
+            if (pc != null) {
+                char[] password = SaslUtils.encodePassword(userPassword);
 
-			for (Callback callback : callbacks) {
-				if (callback instanceof AuthorizeCallback) {
-					ac = (AuthorizeCallback) callback;
-				} else if (callback instanceof NameCallback) {
-					nc = (NameCallback) callback;
-				} else if (callback instanceof PasswordCallback) {
-					pc = (PasswordCallback) callback;
-				} else if (callback instanceof RealmCallback) {
-					continue; // realm is ignored
-				} else {
-					throw new UnsupportedCallbackException(callback,
-							"handle: Unrecognized SASL DIGEST-MD5 Callback");
-				}
-			}
+                LOG.debug("handle: SASL server DIGEST-MD5 callback: setting password for client: ",
+                          userPassword);
 
-			if (nc != null) {
-				LOG.debug("handle: SASL server DIGEST-MD5 callback: setting "
-						+ "username for client: " + userName);
+                pc.setPassword(password);
+            }
+            if (ac != null) {
 
-				nc.setName(userName);
-			}
+                String authid = ac.getAuthenticationID();
+                String authzid = ac.getAuthorizationID();
 
-			if (pc != null) {
-				char[] password = SaslUtils.encodePassword(userPassword);
+                if (authid.equals(authzid)) {
+                    ac.setAuthorized(true);
+                } else {
+                    ac.setAuthorized(false);
+                }
 
-				LOG.debug("handle: SASL server DIGEST-MD5 callback: setting "
-						+ "password for client: " + userPassword);
+                if (ac.isAuthorized()) {
+                    LOG.debug("handle: SASL server DIGEST-MD5 callback: setting canonicalized client ID: ",
+                              userName);
+                    ac.setAuthorizedID(authzid);
+                }
+            }
+        }
+    }
 
-				pc.setPassword(password);
-			}
-			if (ac != null) {
-
-				String authid = ac.getAuthenticationID();
-				String authzid = ac.getAuthorizationID();
-
-				if (authid.equals(authzid)) {
-					ac.setAuthorized(true);
-				} else {
-					ac.setAuthorized(false);
-				}
-
-				if (ac.isAuthorized()) {
-					LOG.debug("handle: SASL server DIGEST-MD5 callback: setting "
-							+ "canonicalized client ID: " + userName);
-					ac.setAuthorizedID(authzid);
-				}
-			}
-		}
-	}
-
-	/**
-	 * Used by SaslTokenMessage::processToken() to respond to server SASL
-	 * tokens.
-	 * 
-	 * @param token
-	 *            Server's SASL token
-	 * @return token to send back to the server.
-	 */
-	public byte[] response(byte[] token) {
-		try {
-			LOG.debug("response: Responding to input token of length: "
-					+ token.length);
-			byte[] retval = saslServer.evaluateResponse(token);
-			LOG.debug("response: Response token length: " + retval.length);
-			return retval;
-		} catch (SaslException e) {
-			LOG.error("response: Failed to evaluate client token of length: "
-					+ token.length + " : " + e);
-			return null;
-		}
-	}
-}
\ No newline at end of file
+    /**
+     * Used by SaslTokenMessage::processToken() to respond to server SASL
+     * tokens.
+     *
+     * @param token
+     *            Server's SASL token
+     * @return token to send back to the server.
+     */
+    public byte[] response(byte[] token) {
+        try {
+            LOG.debug("response: Responding to input token of length: {}",
+                      token.length);
+            byte[] retval = saslServer.evaluateResponse(token);
+            LOG.debug("response: Response token length: {}", retval.length);
+            return retval;
+        } catch (SaslException e) {
+            LOG.error("response: Failed to evaluate client token of length: {} : {}",
+                      token.length, e);
+            return null;
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java
index 9800959..4264e51 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslNettyServerState.java
@@ -22,10 +22,9 @@
 
 final class SaslNettyServerState {
 
-	public static final ChannelLocal<SaslNettyServer> getSaslNettyServer = new ChannelLocal<SaslNettyServer>() {
-		protected SaslNettyServer initialValue(Channel channel) {
-			return null;
-		}
-	};
-
-}
\ No newline at end of file
+    public static final ChannelLocal<SaslNettyServer> getSaslNettyServer = new ChannelLocal<SaslNettyServer>() {
+            protected SaslNettyServer initialValue(Channel channel) {
+                return null;
+            }
+        };
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java
index 12b466c..980c0f6 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormClientHandler.java
@@ -28,19 +28,18 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import backtype.storm.Config;
-
 public class SaslStormClientHandler extends SimpleChannelUpstreamHandler {
 
     private static final Logger LOG = LoggerFactory
             .getLogger(SaslStormClientHandler.class);
-    private Client client;
+
+    private ISaslClient client;
     long start_time;
     /** Used for client or server's token to send or receive from each other. */
     private byte[] token;
-    private String topologyName;
+    private String name;
 
-    public SaslStormClientHandler(Client client) throws IOException {
+    public SaslStormClientHandler(ISaslClient client) throws IOException {
         this.client = client;
         start_time = System.currentTimeMillis();
         getSASLCredentials();
@@ -51,9 +50,7 @@
             ChannelStateEvent event) {
         // register the newly established channel
         Channel channel = ctx.getChannel();
-
-        LOG.info("Connection established from " + channel.getLocalAddress()
-                + " to " + channel.getRemoteAddress());
+        client.channelConnected(channel);
 
         try {
             SaslNettyClient saslNettyClient = SaslNettyClientState.getSaslNettyClient
@@ -62,17 +59,16 @@
             if (saslNettyClient == null) {
                 LOG.debug("Creating saslNettyClient now " + "for channel: "
                         + channel);
-                saslNettyClient = new SaslNettyClient(topologyName, token);
+                saslNettyClient = new SaslNettyClient(name, token);
                 SaslNettyClientState.getSaslNettyClient.set(channel,
                         saslNettyClient);
             }
+            LOG.debug("Sending SASL_TOKEN_MESSAGE_REQUEST");
             channel.write(ControlMessage.SASL_TOKEN_MESSAGE_REQUEST);
         } catch (Exception e) {
             LOG.error("Failed to authenticate with server " + "due to error: ",
                     e);
         }
-        return;
-
     }
 
     @Override
@@ -96,7 +92,7 @@
             ControlMessage msg = (ControlMessage) event.getMessage();
             if (msg == ControlMessage.SASL_COMPLETE_REQUEST) {
                 LOG.debug("Server has sent us the SaslComplete "
-                        + "message. Allowing normal work to proceed.");
+                          + "message. Allowing normal work to proceed.");
 
                 if (!saslNettyClient.isComplete()) {
                     LOG.error("Server returned a Sasl-complete message, "
@@ -106,6 +102,8 @@
                             + "we can tell, we are not authenticated yet.");
                 }
                 ctx.getPipeline().remove(this);
+                this.client.channelReady();
+
                 // We call fireMessageReceived since the client is allowed to
                 // perform this request. The client's request will now proceed
                 // to the next pipeline component namely StormClientHandler.
@@ -116,7 +114,7 @@
         SaslMessageToken saslTokenMessage = (SaslMessageToken) event
                 .getMessage();
         LOG.debug("Responding to server's token of length: "
-                + saslTokenMessage.getSaslToken().length);
+                  + saslTokenMessage.getSaslToken().length);
 
         // Generate SASL response (but we only actually send the response if
         // it's non-null.
@@ -127,17 +125,18 @@
             // (if not, warn), and return without sending a response back to the
             // server.
             LOG.debug("Response to server is null: "
-                    + "authentication should now be complete.");
+                      + "authentication should now be complete.");
             if (!saslNettyClient.isComplete()) {
                 LOG.warn("Generated a null response, "
                         + "but authentication is not complete.");
-                throw new Exception("Server reponse is null, but as far as "
+                throw new Exception("Server response is null, but as far as "
                         + "we can tell, we are not authenticated yet.");
             }
+            this.client.channelReady();
             return;
         } else {
             LOG.debug("Response to server token has length:"
-                    + responseToServer.length);
+                      + responseToServer.length);
         }
         // Construct a message containing the SASL response and send it to the
         // server.
@@ -146,12 +145,14 @@
     }
 
     private void getSASLCredentials() throws IOException {
-        topologyName = (String) this.client.getStormConf().get(Config.TOPOLOGY_NAME);
-        String secretKey = SaslUtils.getSecretKey(this.client.getStormConf());
+        String secretKey;
+        name = client.name();
+        secretKey = client.secretKey();
+
         if (secretKey != null) {
             token = secretKey.getBytes();
         }
-        LOG.debug("SASL credentials for storm topology " + topologyName
+        LOG.debug("SASL credentials for storm topology " + name
                 + " is " + secretKey);
     }
-}
\ No newline at end of file
+}
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java
index 02448e2..a8f9978 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslStormServerHandler.java
@@ -28,11 +28,9 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import backtype.storm.Config;
-
 public class SaslStormServerHandler extends SimpleChannelUpstreamHandler {
 
-    Server server;
+    ISaslServer server;
     /** Used for client or server's token to send or receive from each other. */
     private byte[] token;
     private String topologyName;
@@ -40,7 +38,7 @@
     private static final Logger LOG = LoggerFactory
             .getLogger(SaslStormServerHandler.class);
 
-    public SaslStormServerHandler(Server server) throws IOException {
+    public SaslStormServerHandler(ISaslServer server) throws IOException {
         this.server = server;
         getSASLCredentials();
     }
@@ -55,7 +53,7 @@
         Channel channel = ctx.getChannel();
 
         if (msg instanceof ControlMessage
-                && ((ControlMessage) e.getMessage()) == ControlMessage.SASL_TOKEN_MESSAGE_REQUEST) {
+                && e.getMessage() == ControlMessage.SASL_TOKEN_MESSAGE_REQUEST) {
             // initialize server-side SASL functionality, if we haven't yet
             // (in which case we are looking at the first SASL message from the
             // client).
@@ -85,7 +83,7 @@
             LOG.debug("processToken:  With nettyServer: " + saslNettyServer
                     + " and token length: " + token.length);
 
-            SaslMessageToken saslTokenMessageRequest = null;
+            SaslMessageToken saslTokenMessageRequest;
             saslTokenMessageRequest = new SaslMessageToken(
                     saslNettyServer.response(new byte[0]));
             // Send response to client.
@@ -102,10 +100,8 @@
             SaslNettyServer saslNettyServer = SaslNettyServerState.getSaslNettyServer
                     .get(channel);
             if (saslNettyServer == null) {
-                if (saslNettyServer == null) {
-                    throw new Exception("saslNettyServer was unexpectedly "
-                            + "null for channel: " + channel);
-                }
+                throw new Exception("saslNettyServer was unexpectedly "
+                        + "null for channel: " + channel);
             }
             SaslMessageToken saslTokenMessageRequest = new SaslMessageToken(
                     saslNettyServer.response(((SaslMessageToken) msg)
@@ -123,8 +119,8 @@
                 LOG.debug("Removing SaslServerHandler from pipeline since SASL "
                         + "authentication is complete.");
                 ctx.getPipeline().remove(this);
+                server.authenticated(channel);
             }
-            return;
         } else {
             // Client should not be sending other-than-SASL messages before
             // SaslServerHandler has removed itself from the pipeline. Such
@@ -139,17 +135,19 @@
 
     @Override
     public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
-        server.closeChannel(e.getChannel());
+        if(server != null) server.closeChannel(e.getChannel());
     }
 
     private void getSASLCredentials() throws IOException {
-        topologyName = (String) this.server.storm_conf
-                .get(Config.TOPOLOGY_NAME);
-        String secretKey = SaslUtils.getSecretKey(this.server.storm_conf);
+        String secretKey;
+        topologyName = server.name();
+        secretKey = server.secretKey();
+
         if (secretKey != null) {
             token = secretKey.getBytes();
         }
-        LOG.debug("SASL credentials for storm topology " + topologyName
-                + " is " + secretKey);
+
+        LOG.debug("SASL credentials for storm topology {} is {}",
+                  topologyName, secretKey);
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java
index a2d0b26..4a8106b 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/SaslUtils.java
@@ -17,7 +17,6 @@
  */
 package backtype.storm.messaging.netty;
 
-import java.nio.charset.Charset;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -29,11 +28,12 @@
 import backtype.storm.Config;
 
 class SaslUtils {
+    public static final String KERBEROS = "GSSAPI";
     public static final String AUTH_DIGEST_MD5 = "DIGEST-MD5";
     public static final String DEFAULT_REALM = "default";
 
     static Map<String, String> getSaslProps() {
-        Map<String, String> props = new HashMap<String, String>();
+        Map<String, String> props = new HashMap<>();
         props.put(Sasl.POLICY_NOPLAINTEXT, "true");
         return props;
     }
@@ -62,13 +62,7 @@
     }
 
     static String getSecretKey(Map conf) {
-        if (conf == null || conf.isEmpty())
-            return null;
-
-        String secretPayLoad = (String) conf
-                .get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
-
-        return secretPayLoad;
+        return conf == null || conf.isEmpty() ? null : (String) conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
     }
 
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java
index e984144..4990f63 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java
@@ -17,19 +17,26 @@
  */
 package backtype.storm.messaging.netty;
 
+import backtype.storm.Config;
+import backtype.storm.grouping.Load;
+import backtype.storm.messaging.ConnectionWithStatus;
+import backtype.storm.messaging.IConnectionCallback;
+import backtype.storm.messaging.TaskMessage;
+import backtype.storm.metric.api.IStatefulObject;
+import backtype.storm.serialization.KryoValuesSerializer;
+import backtype.storm.utils.Utils;
+import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
-
+import java.util.concurrent.atomic.AtomicInteger;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.channel.Channel;
 import org.jboss.netty.channel.ChannelFactory;
@@ -39,74 +46,48 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import backtype.storm.Config;
-import backtype.storm.messaging.ConnectionWithStatus;
-import backtype.storm.messaging.IConnection;
-import backtype.storm.messaging.TaskMessage;
-import backtype.storm.metric.api.IStatefulObject;
-import backtype.storm.utils.Utils;
-
-class Server extends ConnectionWithStatus implements IStatefulObject {
+class Server extends ConnectionWithStatus implements IStatefulObject, ISaslServer {
 
     private static final Logger LOG = LoggerFactory.getLogger(Server.class);
     @SuppressWarnings("rawtypes")
     Map storm_conf;
     int port;
-    private final ConcurrentHashMap<String, AtomicInteger> messagesEnqueued = new ConcurrentHashMap<String, AtomicInteger>();
+    private final ConcurrentHashMap<String, AtomicInteger> messagesEnqueued = new ConcurrentHashMap<>();
     private final AtomicInteger messagesDequeued = new AtomicInteger(0);
-    private final AtomicInteger[] pendingMessages;
-    
-    
-    // Create multiple queues for incoming messages. The size equals the number of receiver threads.
-    // For message which is sent to same task, it will be stored in the same queue to preserve the message order.
-    private LinkedBlockingQueue<ArrayList<TaskMessage>>[] message_queue;
     
     volatile ChannelGroup allChannels = new DefaultChannelGroup("storm-server");
     final ChannelFactory factory;
     final ServerBootstrap bootstrap;
-    
-    private int queueCount;
-    private volatile HashMap<Integer, Integer> taskToQueueId = null;
-    int roundRobinQueueId;
-	
+ 
     private volatile boolean closing = false;
     List<TaskMessage> closeMessage = Arrays.asList(new TaskMessage(-1, null));
-    
+    private KryoValuesSerializer _ser;
+    private IConnectionCallback _cb = null; 
     
     @SuppressWarnings("rawtypes")
     Server(Map storm_conf, int port) {
         this.storm_conf = storm_conf;
         this.port = port;
-        
-        queueCount = Utils.getInt(storm_conf.get(Config.WORKER_RECEIVER_THREAD_COUNT), 1);
-        roundRobinQueueId = 0;
-        taskToQueueId = new HashMap<Integer, Integer>();
-    
-        message_queue = new LinkedBlockingQueue[queueCount];
-        pendingMessages = new AtomicInteger[queueCount];
-        for (int i = 0; i < queueCount; i++) {
-            message_queue[i] = new LinkedBlockingQueue<ArrayList<TaskMessage>>();
-            pendingMessages[i] = new AtomicInteger(0);
-        }
-        
+        _ser = new KryoValuesSerializer(storm_conf);
+
         // Configure the server.
         int buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
         int backlog = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_SOCKET_BACKLOG), 500);
         int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS));
 
-        ThreadFactory bossFactory = new NettyRenameThreadFactory(name() + "-boss");
-        ThreadFactory workerFactory = new NettyRenameThreadFactory(name() + "-worker");
-        
+        ThreadFactory bossFactory = new NettyRenameThreadFactory(netty_name() + "-boss");
+        ThreadFactory workerFactory = new NettyRenameThreadFactory(netty_name() + "-worker");
+
         if (maxWorkers > 0) {
-            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), 
+            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
                 Executors.newCachedThreadPool(workerFactory), maxWorkers);
         } else {
-            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), 
+            factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
                 Executors.newCachedThreadPool(workerFactory));
         }
-        
-        LOG.info("Create Netty Server " + name() + ", buffer_size: " + buffer_size + ", maxWorkers: " + maxWorkers);
-        
+
+        LOG.info("Create Netty Server " + netty_name() + ", buffer_size: " + buffer_size + ", maxWorkers: " + maxWorkers);
+
         bootstrap = new ServerBootstrap(factory);
         bootstrap.setOption("child.tcpNoDelay", true);
         bootstrap.setOption("child.receiveBufferSize", buffer_size);
@@ -121,48 +102,6 @@
         allChannels.add(channel);
     }
     
-    private ArrayList<TaskMessage>[] groupMessages(List<TaskMessage> msgs) {
-        ArrayList<TaskMessage> messageGroups[] = new ArrayList[queueCount];
-
-        for (int i = 0; i < msgs.size(); i++) {
-            TaskMessage message = msgs.get(i);
-            int task = message.task();
-
-            if (task == -1) {
-                closing = true;
-                return null;
-            }
-
-            Integer queueId = getMessageQueueId(task);
-
-            if (null == messageGroups[queueId]) {
-                messageGroups[queueId] = new ArrayList<TaskMessage>();
-            }
-            messageGroups[queueId].add(message);
-        }
-        return messageGroups;
-    }
-    
-    private Integer getMessageQueueId(int task) {
-        // try to construct the map from taskId -> queueId in round robin manner.
-        Integer queueId = taskToQueueId.get(task);
-        if (null == queueId) {
-            synchronized (this) {
-                queueId = taskToQueueId.get(task);
-                if (queueId == null) {
-                    queueId = roundRobinQueueId++;
-                    if (roundRobinQueueId == queueCount) {
-                        roundRobinQueueId = 0;
-                    }
-                    HashMap<Integer, Integer> newRef = new HashMap<Integer, Integer>(taskToQueueId);
-                    newRef.put(task, queueId);
-                    taskToQueueId = newRef;
-                }
-            }
-        }
-        return queueId;
-    }
-
     private void addReceiveCount(String from, int amount) {
         //This is possibly lossy in the case where a value is deleted
         // because it has received no messages over the metrics collection
@@ -181,9 +120,8 @@
         }
     }
 
-
     /**
-     * enqueue a received message 
+     * enqueue a received message
      * @throws InterruptedException
      */
     protected void enqueue(List<TaskMessage> msgs, String from) throws InterruptedException {
@@ -191,65 +129,28 @@
             return;
         }
         addReceiveCount(from, msgs.size());
-        ArrayList<TaskMessage> messageGroups[] = groupMessages(msgs);
-
-        if (null == messageGroups || closing) {
-            return;
-        }
-
-        for (int receiverId = 0; receiverId < messageGroups.length; receiverId++) {
-            ArrayList<TaskMessage> msgGroup = messageGroups[receiverId];
-            if (null != msgGroup) {
-                message_queue[receiverId].put(msgGroup);
-                pendingMessages[receiverId].addAndGet(msgGroup.size());
-            }
+        if (_cb != null) {
+            _cb.recv(msgs);
         }
     }
 
-    public Iterator<TaskMessage> recv(int flags, int receiverId) {
-        if (closing) {
-            return closeMessage.iterator();
-        }
-
-        ArrayList<TaskMessage> ret = null;
-        int queueId = receiverId % queueCount;
-        if ((flags & 0x01) == 0x01) {
-            //non-blocking
-            ret = message_queue[queueId].poll();
-        }
-        else {
-            try {
-                ArrayList<TaskMessage> request = message_queue[queueId].take();
-                LOG.debug("request to be processed: {}", request);
-                ret = request;
-            }
-            catch (InterruptedException e) {
-                LOG.info("exception within msg receiving", e);
-                ret = null;
-            }
-        }
-
-        if (null != ret) {
-            messagesDequeued.addAndGet(ret.size());
-            pendingMessages[queueId].addAndGet(0 - ret.size());
-            return ret.iterator();
-        }
-        return null;
+    @Override
+    public void registerRecv(IConnectionCallback cb) {
+        _cb = cb;
     }
-   
+
     /**
      * register a newly created channel
-     * @param channel
+     * @param channel newly created channel
      */
     protected void addChannel(Channel channel) {
         allChannels.add(channel);
     }
-    
+
     /**
-     * close a channel
-     * @param channel
+     * @param channel channel to close
      */
-    protected void closeChannel(Channel channel) {
+    public void closeChannel(Channel channel) {
         channel.close().awaitUninterruptibly();
         allChannels.remove(channel);
     }
@@ -265,15 +166,33 @@
         }
     }
 
+    @Override
+    public void sendLoadMetrics(Map<Integer, Double> taskToLoad) {
+        try {
+            MessageBatch mb = new MessageBatch(1);
+            mb.add(new TaskMessage(-1, _ser.serialize(Arrays.asList((Object)taskToLoad))));
+            allChannels.write(mb);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public Map<Integer, Load> getLoad(Collection<Integer> tasks) {
+        throw new RuntimeException("Server connection cannot get load");
+    }
+
+    @Override
     public void send(int task, byte[] message) {
         throw new UnsupportedOperationException("Server connection should not send any messages");
     }
-    
+
+    @Override
     public void send(Iterator<TaskMessage> msgs) {
       throw new UnsupportedOperationException("Server connection should not send any messages");
     }
-	
-    public String name() {
+
+    public String netty_name() {
       return "Netty-server-localhost-" + port;
     }
 
@@ -306,14 +225,9 @@
     }
 
     public Object getState() {
-        LOG.info("Getting metrics for server on port {}", port);
-        HashMap<String, Object> ret = new HashMap<String, Object>();
+        LOG.debug("Getting metrics for server on port {}", port);
+        HashMap<String, Object> ret = new HashMap<>();
         ret.put("dequeuedMessages", messagesDequeued.getAndSet(0));
-        ArrayList<Integer> pending = new ArrayList<Integer>(pendingMessages.length);
-        for (AtomicInteger p: pendingMessages) {
-            pending.add(p.get());
-        }
-        ret.put("pending", pending);
         HashMap<String, Integer> enqueued = new HashMap<String, Integer>();
         Iterator<Map.Entry<String, AtomicInteger>> it = messagesEnqueued.entrySet().iterator();
         while (it.hasNext()) {
@@ -330,8 +244,30 @@
         return ret;
     }
 
-    @Override public String toString() {
-       return String.format("Netty server listening on port %s", port);
+    /** Implementing IServer. **/
+    public void channelConnected(Channel c) {
+        addChannel(c);
     }
 
+    public void received(Object message, String remote, Channel channel)  throws InterruptedException {
+        List<TaskMessage>msgs = (List<TaskMessage>)message;
+        enqueue(msgs, remote);
+    }
+
+    public String name() {
+        return (String)storm_conf.get(Config.TOPOLOGY_NAME);
+    }
+
+    public String secretKey() {
+        return SaslUtils.getSecretKey(storm_conf);
+    }
+
+    public void authenticated(Channel c) {
+        return;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("Netty server listening on port %s", port);
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java
index 2d25001..877b6d8 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java
@@ -17,7 +17,20 @@
  */
 package backtype.storm.messaging.netty;
 
+import backtype.storm.messaging.TaskMessage;
+import backtype.storm.serialization.KryoValuesDeserializer;
+
 import java.net.ConnectException;
+import java.util.Map;
+import java.util.List;
+import java.io.IOException;
+
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
 
 import org.jboss.netty.channel.*;
 import org.slf4j.Logger;
@@ -26,9 +39,41 @@
 public class StormClientHandler extends SimpleChannelUpstreamHandler  {
     private static final Logger LOG = LoggerFactory.getLogger(StormClientHandler.class);
     private Client client;
-    
-    StormClientHandler(Client client) {
+    private KryoValuesDeserializer _des;
+
+    StormClientHandler(Client client, Map conf) {
         this.client = client;
+        _des = new KryoValuesDeserializer(conf);
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) {
+        //examine the response message from server
+        Object message = event.getMessage();
+        if (message instanceof ControlMessage) {
+            ControlMessage msg = (ControlMessage)message;
+            if (msg==ControlMessage.FAILURE_RESPONSE) {
+                LOG.info("failure response:{}", msg);
+            }
+        } else if (message instanceof List) {
+            try {
+                //This should be the metrics, and there should only be one of them
+                List<TaskMessage> list = (List<TaskMessage>)message;
+                if (list.size() < 1) throw new RuntimeException("Didn't see enough load metrics ("+client.getDstAddress()+") "+list);
+                if (list.size() != 1) LOG.warn("Messages are not being delivered fast enough, got "+list.size()+" metrics messages at once("+client.getDstAddress()+")");
+                TaskMessage tm = ((List<TaskMessage>)message).get(list.size() - 1);
+                if (tm.task() != -1) throw new RuntimeException("Metrics messages are sent to the system task ("+client.getDstAddress()+") "+tm);
+                List metrics = _des.deserialize(tm.message());
+                if (metrics.size() < 1) throw new RuntimeException("No metrics data in the metrics message ("+client.getDstAddress()+") "+metrics);
+                if (!(metrics.get(0) instanceof Map)) throw new RuntimeException("The metrics did not have a map in the first slot ("+client.getDstAddress()+") "+metrics);
+                client.setLoadMetrics((Map<Integer, Double>)metrics.get(0));
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        } else {
+            throw new RuntimeException("Don't know how to handle a message of type "
+                                       + message + " (" + client.getDstAddress() + ")");
+        }
     }
 
     @Override
@@ -40,7 +85,7 @@
     public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) {
         Throwable cause = event.getCause();
         if (!(cause instanceof ConnectException)) {
-            LOG.info("Connection failed " + client.dstAddressPrefixedName, cause);
+            LOG.info("Connection to "+client.getDstAddress()+" failed:", cause);
         }
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java
index 4be06cd..6158eef 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java
@@ -22,12 +22,15 @@
 import org.jboss.netty.channel.Channels;
 
 import backtype.storm.Config;
+import java.util.Map;
 
 class StormClientPipelineFactory implements ChannelPipelineFactory {
     private Client client;
+    private Map conf;
 
-    StormClientPipelineFactory(Client client) {
+    StormClientPipelineFactory(Client client, Map conf) {
         this.client = client;
+        this.conf = conf;
     }
 
     public ChannelPipeline getPipeline() throws Exception {
@@ -39,15 +42,15 @@
         // Encoder
         pipeline.addLast("encoder", new MessageEncoder());
 
-        boolean isNettyAuth = (Boolean) this.client.getStormConf().get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION);
+        boolean isNettyAuth = (Boolean) conf
+                .get(Config.STORM_MESSAGING_NETTY_AUTHENTICATION);
         if (isNettyAuth) {
             // Authenticate: Removed after authentication completes
             pipeline.addLast("saslClientHandler", new SaslStormClientHandler(
                     client));
         }
         // business logic.
-        pipeline.addLast("handler", new StormClientHandler(client));
-
+        pipeline.addLast("handler", new StormClientHandler(client, conf));
         return pipeline;
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java
index 6b71171..8140df6 100644
--- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java
+++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormServerHandler.java
@@ -17,7 +17,8 @@
  */
 package backtype.storm.messaging.netty;
 
-import backtype.storm.messaging.TaskMessage;
+import backtype.storm.utils.Utils;
+import org.jboss.netty.channel.Channel;
 import org.jboss.netty.channel.ChannelHandlerContext;
 import org.jboss.netty.channel.ChannelStateEvent;
 import org.jboss.netty.channel.ExceptionEvent;
@@ -25,33 +26,39 @@
 import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
-class StormServerHandler extends SimpleChannelUpstreamHandler  {
+public class StormServerHandler extends SimpleChannelUpstreamHandler  {
     private static final Logger LOG = LoggerFactory.getLogger(StormServerHandler.class);
-    Server server;
+    IServer server;
     private AtomicInteger failure_count; 
+    private Channel channel;
     
-    StormServerHandler(Server server) {
+    public StormServerHandler(IServer server) {
         this.server = server;
         failure_count = new AtomicInteger(0);
     }
     
     @Override
     public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) {
-        server.addChannel(e.getChannel());
+        server.channelConnected(e.getChannel());
+        if(channel != null) {
+            LOG.debug("Replacing channel with new channel: {} -> ",
+                      channel, e.getChannel());
+        }
+        channel = e.getChannel();
+        server.channelConnected(channel);
     }
     
     @Override
     public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
-      List<TaskMessage> msgs = (List<TaskMessage>) e.getMessage();
+      Object msgs = e.getMessage();
       if (msgs == null) {
         return;
       }
       
       try {
-        server.enqueue(msgs, e.getRemoteAddress().toString());
+        server.received(msgs, e.getRemoteAddress().toString(), channel);
       } catch (InterruptedException e1) {
         LOG.info("failed to enqueue a request message", e);
         failure_count.incrementAndGet();
@@ -61,6 +68,7 @@
     @Override
     public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) {
         LOG.error("server errors in handling the request", e.getCause());
+        Utils.handleUncaughtException(e.getCause());
         server.closeChannel(e.getChannel());
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java b/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
index 17ebbd1..6d43e65 100644
--- a/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
+++ b/storm-core/src/jvm/backtype/storm/metric/EventLoggerBolt.java
@@ -1,20 +1,29 @@
+/**
+ * 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.metric;
 
 import backtype.storm.task.IBolt;
 import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
 import java.util.Map;
 import static backtype.storm.metric.IEventLogger.EventInfo;
 
diff --git a/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java b/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java
index 3834c55..3003427 100644
--- a/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java
+++ b/storm-core/src/jvm/backtype/storm/metric/FileBasedEventLogger.java
@@ -1,3 +1,20 @@
+/**
+ * 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.metric;
 
 import backtype.storm.task.TopologyContext;
@@ -5,6 +22,7 @@
 import org.slf4j.LoggerFactory;
 
 import java.io.BufferedWriter;
+import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
@@ -17,7 +35,7 @@
 import java.util.concurrent.TimeUnit;
 
 public class FileBasedEventLogger implements IEventLogger {
-    private static Logger LOG = LoggerFactory.getLogger(FileBasedEventLogger.class);
+    private static final Logger LOG = LoggerFactory.getLogger(FileBasedEventLogger.class);
 
     private static final int FLUSH_INTERVAL_MILLIS = 1000;
 
@@ -61,11 +79,11 @@
 
     @Override
     public void prepare(Map stormConf, TopologyContext context) {
-        String logDir;
+        String logDir; // storm local directory
         String stormId = context.getStormId();
         int port = context.getThisWorkerPort();
-        if((logDir = System.getProperty("storm.log.dir")) == null
-                && (logDir = System.getProperty("java.io.tmpdir")) == null) {
+        if ((logDir = System.getProperty("storm.local.dir")) == null &&
+                (logDir = (String)stormConf.get("storm.local.dir")) == null) {
             String msg = "Could not determine the directory to log events.";
             LOG.error(msg);
             throw new RuntimeException(msg);
@@ -77,7 +95,16 @@
          * Include the topology name & worker port in the file name so that
          * multiple event loggers can log independently.
          */
-        initLogWriter(Paths.get(logDir, String.format("%s-events-%d.log", stormId, port)));
+        Path path = Paths.get(logDir, "workers-artifacts", stormId, Integer.toString(port), "events.log");
+        if (!path.isAbsolute()) {
+            path = Paths.get(System.getProperty("storm.home"), logDir, "workers-artifacts",
+                    stormId, Integer.toString(port), "events.log");
+        }
+        File dir = path.toFile().getParentFile();
+        if (!dir.exists()) {
+             dir.mkdirs();
+        }
+        initLogWriter(path);
         setUpFlushTask();
     }
 
diff --git a/storm-core/src/jvm/backtype/storm/metric/HttpForwardingMetricsConsumer.java b/storm-core/src/jvm/backtype/storm/metric/HttpForwardingMetricsConsumer.java
new file mode 100644
index 0000000..4f4242a
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/HttpForwardingMetricsConsumer.java
@@ -0,0 +1,80 @@
+/**
+ * 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.metric;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.net.URL;
+import java.net.HttpURLConnection;
+
+import com.esotericsoftware.kryo.io.Output;
+import backtype.storm.serialization.KryoValuesSerializer;
+
+import backtype.storm.metric.api.IMetricsConsumer;
+import backtype.storm.task.IErrorReporter;
+import backtype.storm.task.TopologyContext;
+
+/**
+ * Listens for all metrics and POSTs them serialized to a configured URL
+ *
+ * To use, add this to your topology's configuration:
+ *   conf.registerMetricsConsumer(backtype.storm.metrics.HttpForwardingMetricsConsumer.class, "http://example.com:8080/metrics/my-topology/", 1);
+ *
+ * The body of the post is data serialized using backtype.storm.serialization.KryoValuesSerializer, with the data passed in
+ * as a list of [TaskInfo, Collection<DataPoint>].  More things may be appended to the end of the list in the future.
+ *
+ * The values can be deserialized using the backtype.storm.serialization.KryoValuesDeserializer, and a 
+ * correct config + classpath. 
+ */
+public class HttpForwardingMetricsConsumer implements IMetricsConsumer {
+    private transient URL _url; 
+    private transient IErrorReporter _errorReporter;
+    private transient KryoValuesSerializer _serializer;
+
+    @Override
+    public void prepare(Map stormConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { 
+        try {
+            _url = new URL((String)registrationArgument);
+            _errorReporter = errorReporter;
+            _serializer = new KryoValuesSerializer(stormConf);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void handleDataPoints(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
+        try {
+            HttpURLConnection con = (HttpURLConnection)_url.openConnection();
+            con.setRequestMethod("POST");
+            con.setDoOutput(true);
+            Output out = new Output(con.getOutputStream());
+            _serializer.serializeInto(Arrays.asList(taskInfo, dataPoints), out);
+            out.flush();
+            out.close();
+            //The connection is not sent unless a response is requested
+            int response = con.getResponseCode();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void cleanup() { }
+}
diff --git a/storm-core/src/jvm/backtype/storm/metric/HttpForwardingMetricsServer.java b/storm-core/src/jvm/backtype/storm/metric/HttpForwardingMetricsServer.java
new file mode 100644
index 0000000..7a8f676
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/HttpForwardingMetricsServer.java
@@ -0,0 +1,118 @@
+/**
+ * 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.metric;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.List;
+import java.net.ServerSocket;
+import java.net.InetAddress;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.ServletException;
+
+import backtype.storm.metric.api.IMetricsConsumer.TaskInfo;
+import backtype.storm.metric.api.IMetricsConsumer.DataPoint;
+
+import com.esotericsoftware.kryo.io.Input;
+import backtype.storm.serialization.KryoValuesDeserializer;
+import backtype.storm.utils.Utils;
+
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+/**
+ * A server that can listen for metrics from the HttpForwardingMetricsConsumer.
+ */
+public abstract class HttpForwardingMetricsServer {
+    private Map _conf;
+    private Server _server = null;
+    private int _port = -1;
+    private String _url = null;
+
+    ThreadLocal<KryoValuesDeserializer> _des = new ThreadLocal<KryoValuesDeserializer>() {
+        @Override
+        protected KryoValuesDeserializer initialValue() {
+            return new KryoValuesDeserializer(_conf);
+        }
+    };
+
+    private class MetricsCollectionServlet extends HttpServlet
+    {
+        protected void doPost(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException
+        {
+            Input in = new Input(request.getInputStream());
+            List<Object> metrics = _des.get().deserializeFrom(in);
+            handle((TaskInfo)metrics.get(0), (Collection<DataPoint>)metrics.get(1));
+            response.setStatus(HttpServletResponse.SC_OK);
+        }
+    }
+
+    public HttpForwardingMetricsServer(Map conf) {
+        _conf = Utils.readStormConfig();
+        if (conf != null) {
+            _conf.putAll(conf);
+        }
+    }
+
+    //This needs to be thread safe
+    public abstract void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints);
+
+    public void serve(Integer port) {
+        try {
+            if (_server != null) throw new RuntimeException("The server is already running");
+    
+            if (port == null || port <= 0) {
+                ServerSocket s = new ServerSocket(0);
+                port = s.getLocalPort();
+                s.close();
+            }
+            _server = new Server(port);
+            _port = port;
+            _url = "http://"+InetAddress.getLocalHost().getHostName()+":"+_port+"/";
+ 
+            ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
+            context.setContextPath("/");
+            _server.setHandler(context);
+ 
+            context.addServlet(new ServletHolder(new MetricsCollectionServlet()),"/*");
+
+            _server.start();
+         } catch (RuntimeException e) {
+             throw e;
+         } catch (Exception e) {
+             throw new RuntimeException(e);
+         }
+    }
+
+    public void serve() {
+        serve(null);
+    }
+
+    public int getPort() {
+        return _port;
+    }
+
+    public String getUrl() {
+        return _url;
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java b/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java
index 9d0e7e9..5fd5a32 100644
--- a/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java
+++ b/storm-core/src/jvm/backtype/storm/metric/IEventLogger.java
@@ -1,8 +1,23 @@
+/**
+ * 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.metric;
 
-import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Tuple;
 
 import java.util.Date;
 import java.util.Map;
@@ -37,11 +52,7 @@
          */
         @Override
         public String toString() {
-            return new StringBuilder(new Date(Long.parseLong(ts)).toString()).append(",")
-                    .append(component).append(",")
-                    .append(task).append(",")
-                    .append(messageId).append(",")
-                    .append(values).toString();
+            return new Date(Long.parseLong(ts)).toString() + "," + component + "," + task + "," + messageId + "," + values;
         }
     }
 
diff --git a/storm-core/src/jvm/backtype/storm/metric/LoggingMetricsConsumer.java b/storm-core/src/jvm/backtype/storm/metric/LoggingMetricsConsumer.java
index c1c7c0a..98fb527 100644
--- a/storm-core/src/jvm/backtype/storm/metric/LoggingMetricsConsumer.java
+++ b/storm-core/src/jvm/backtype/storm/metric/LoggingMetricsConsumer.java
@@ -26,7 +26,6 @@
 import backtype.storm.metric.api.IMetricsConsumer;
 import backtype.storm.task.IErrorReporter;
 import backtype.storm.task.TopologyContext;
-import backtype.storm.utils.Utils;
 
 /*
  * Listens for all metrics, dumps them to log
diff --git a/storm-core/src/jvm/backtype/storm/metric/MetricsConsumerBolt.java b/storm-core/src/jvm/backtype/storm/metric/MetricsConsumerBolt.java
index 5a40c27..e3ec069 100644
--- a/storm-core/src/jvm/backtype/storm/metric/MetricsConsumerBolt.java
+++ b/storm-core/src/jvm/backtype/storm/metric/MetricsConsumerBolt.java
@@ -20,7 +20,6 @@
 import backtype.storm.Config;
 import backtype.storm.metric.api.IMetricsConsumer;
 import backtype.storm.task.IBolt;
-import backtype.storm.task.IErrorReporter;
 import backtype.storm.task.OutputCollector;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.tuple.Tuple;
diff --git a/storm-core/src/jvm/backtype/storm/metric/SystemBolt.java b/storm-core/src/jvm/backtype/storm/metric/SystemBolt.java
index d0bbe44..5cba7d3 100644
--- a/storm-core/src/jvm/backtype/storm/metric/SystemBolt.java
+++ b/storm-core/src/jvm/backtype/storm/metric/SystemBolt.java
@@ -18,7 +18,6 @@
 package backtype.storm.metric;
 
 import backtype.storm.Config;
-import backtype.storm.metric.api.AssignableMetric;
 import backtype.storm.metric.api.IMetric;
 import backtype.storm.task.IBolt;
 import backtype.storm.task.OutputCollector;
@@ -28,12 +27,9 @@
 import clojure.lang.AFn;
 import clojure.lang.IFn;
 import clojure.lang.RT;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.lang.management.*;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 
@@ -41,7 +37,6 @@
 // TaskID is always -1, therefore you can only send-unanchored tuples to co-located SystemBolt.
 // This bolt was conceived to export worker stats via metrics api.
 public class SystemBolt implements IBolt {
-    private static Logger LOG = LoggerFactory.getLogger(SystemBolt.class);
     private static boolean _prepareWasCalled = false;
 
     private static class MemoryUsageMetric implements IMetric {
diff --git a/storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java b/storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java
index dd048b8..6b317da 100644
--- a/storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java
+++ b/storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java
@@ -17,8 +17,6 @@
  */
 package backtype.storm.metric.api;
 
-import backtype.storm.metric.api.IMetric;
-
 public class CountMetric implements IMetric {
     long _value = 0;
 
diff --git a/storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java b/storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java
index e25e26d..fde7053 100644
--- a/storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java
+++ b/storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java
@@ -17,8 +17,6 @@
  */
 package backtype.storm.metric.api;
 
-import backtype.storm.metric.api.IReducer;
-
 class MeanReducerState {
     public int count = 0;
     public double sum = 0.0;
@@ -47,7 +45,7 @@
 
     public Object extractResult(MeanReducerState acc) {
         if(acc.count > 0) {
-            return new Double(acc.sum / (double)acc.count);
+            return acc.sum / (double) acc.count;
         } else {
             return null;
         }
diff --git a/storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java b/storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java
index c420a16..a6077e6 100644
--- a/storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java
+++ b/storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java
@@ -22,7 +22,7 @@
 import java.util.Map;
 
 public class MultiCountMetric implements IMetric {
-    Map<String, CountMetric> _value = new HashMap();
+    Map<String, CountMetric> _value = new HashMap<>();
 
     public MultiCountMetric() {
     }
diff --git a/storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java b/storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java
index 530b168..d9d3a02 100644
--- a/storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java
+++ b/storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java
@@ -22,7 +22,7 @@
 import java.util.Map;
 
 public class MultiReducedMetric implements IMetric {
-    Map<String, ReducedMetric> _value = new HashMap();
+    Map<String, ReducedMetric> _value = new HashMap<>();
     IReducer _reducer;
 
     public MultiReducedMetric(IReducer reducer) {
diff --git a/storm-core/src/jvm/backtype/storm/metric/api/rpc/CountShellMetric.java b/storm-core/src/jvm/backtype/storm/metric/api/rpc/CountShellMetric.java
index def74c2..eae982b 100644
--- a/storm-core/src/jvm/backtype/storm/metric/api/rpc/CountShellMetric.java
+++ b/storm-core/src/jvm/backtype/storm/metric/api/rpc/CountShellMetric.java
@@ -21,8 +21,7 @@
 
 public class CountShellMetric extends CountMetric implements IShellMetric {
     /***
-     * @param
-     *  params should be null or long
+     * @param value should be null or long
      *  if value is null, it will call incr()
      *  if value is long, it will call incrBy((long)params)
      * */
diff --git a/storm-core/src/jvm/backtype/storm/metric/internal/CountStatAndMetric.java b/storm-core/src/jvm/backtype/storm/metric/internal/CountStatAndMetric.java
new file mode 100644
index 0000000..194a9e2
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/internal/CountStatAndMetric.java
@@ -0,0 +1,211 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicLong;
+
+import backtype.storm.metric.api.IMetric;
+
+/**
+ * Acts as a Count Metric, but also keeps track of approximate counts
+ * for the last 10 mins, 3 hours, 1 day, and all time.
+ */
+public class CountStatAndMetric implements IMetric{
+    private final AtomicLong _currentBucket;
+    // All internal state except for the count of the current bucket are
+    // protected using a lock on this counter
+    private long _bucketStart;
+
+    //exact variable time, that is added to the current bucket
+    private long _exactExtra;
+ 
+    //10 min values
+    private final int _tmSize;
+    private final long[] _tmBuckets;
+    private final long[] _tmTime;
+    
+    //3 hour values
+    private final int _thSize;
+    private final long[] _thBuckets;
+    private final long[] _thTime;
+
+    //1 day values
+    private final int _odSize;
+    private final long[] _odBuckets;
+    private final long[] _odTime;
+ 
+    //all time
+    private long _allTime;
+
+    private final TimerTask _task;
+
+    /**
+     * @param numBuckets the number of buckets to divide the time periods into.
+     */
+    public CountStatAndMetric(int numBuckets) {
+        this(numBuckets, -1);
+    }
+
+    /**
+     * Constructor
+     * @param numBuckets the number of buckets to divide the time periods into.
+     * @param startTime if positive the simulated time to start the from.
+     */
+    CountStatAndMetric(int numBuckets, long startTime){
+        numBuckets = Math.max(numBuckets, 2);
+        //We want to capture the full time range, so the target size is as
+        // if we had one bucket less, then we do
+        _tmSize = 10 * 60 * 1000 / (numBuckets - 1);
+        _thSize = 3 * 60 * 60 * 1000 / (numBuckets - 1);
+        _odSize = 24 * 60 * 60 * 1000 / (numBuckets - 1);
+        if (_tmSize < 1 || _thSize < 1 || _odSize < 1) {
+            throw new IllegalArgumentException("number of buckets is too large to be supported");
+        }
+        _tmBuckets = new long[numBuckets];
+        _tmTime = new long[numBuckets];
+        _thBuckets = new long[numBuckets];
+        _thTime = new long[numBuckets];
+        _odBuckets = new long[numBuckets];
+        _odTime = new long[numBuckets];
+        _allTime = 0;
+        _exactExtra = 0;
+
+        _bucketStart = startTime >= 0 ? startTime : System.currentTimeMillis();
+        _currentBucket = new AtomicLong(0);
+        if (startTime < 0) {
+            _task = new Fresher();
+            MetricStatTimer._timer.scheduleAtFixedRate(_task, _tmSize, _tmSize);
+        } else {
+            _task = null;
+        }
+    }
+
+    /**
+     * Increase the count by the given value.
+     *
+     * @param count number to count
+     */
+    public void incBy(long count) {
+        _currentBucket.addAndGet(count);
+    }
+
+   
+
+    @Override
+    public synchronized Object getValueAndReset() {
+        return getValueAndReset(System.currentTimeMillis());
+    }
+
+    synchronized Object getValueAndReset(long now) {
+        long value = _currentBucket.getAndSet(0);
+        long timeSpent = now - _bucketStart;
+        long ret = value + _exactExtra;
+        _bucketStart = now;
+        _exactExtra = 0;
+        rotateBuckets(value, timeSpent);
+        return ret;
+    }
+
+    synchronized void rotateSched(long now) {
+        long value = _currentBucket.getAndSet(0);
+        long timeSpent = now - _bucketStart;
+        _exactExtra += value;
+        _bucketStart = now;
+        rotateBuckets(value, timeSpent);
+    }
+
+    synchronized void rotateBuckets(long value, long timeSpent) {
+        rotate(value, timeSpent, _tmSize, _tmTime, _tmBuckets);
+        rotate(value, timeSpent, _thSize, _thTime, _thBuckets);
+        rotate(value, timeSpent, _odSize, _odTime, _odBuckets);
+        _allTime += value;
+    }
+
+    private synchronized void rotate(long value, long timeSpent, long targetSize, long [] times, long [] buckets) {
+        times[0] += timeSpent;
+        buckets[0] += value;
+
+        long currentTime = 0;
+        long currentVal = 0;
+        if (times[0] >= targetSize) {
+            for (int i = 0; i < buckets.length; i++) {
+                long tmpTime = times[i];
+                times[i] = currentTime;
+                currentTime = tmpTime;
+
+                long cnt = buckets[i];
+                buckets[i] = currentVal;
+                currentVal = cnt;
+            }
+        }
+    }
+
+    /**
+     * @return a map of time window to count.
+     * Keys are "600" for last 10 mins
+     * "10800" for the last 3 hours
+     * "86400" for the last day
+     * ":all-time" for all time
+     */
+    public synchronized Map<String, Long> getTimeCounts() {
+        return getTimeCounts(System.currentTimeMillis());
+    }
+
+    synchronized Map<String, Long> getTimeCounts(long now) {
+        Map<String, Long> ret = new HashMap<>();
+        long value = _currentBucket.get();
+        long timeSpent = now - _bucketStart;
+        ret.put("600", readApproximateTime(value, timeSpent, _tmTime, _tmBuckets, 600 * 1000));
+        ret.put("10800", readApproximateTime(value, timeSpent, _thTime, _thBuckets, 10800 * 1000));
+        ret.put("86400", readApproximateTime(value, timeSpent, _odTime, _odBuckets, 86400 * 1000));
+        ret.put(":all-time", value + _allTime);
+        return ret;
+    }
+
+    long readApproximateTime(long value, long timeSpent, long[] bucketTime, long[] buckets, long desiredTime) {
+        long timeNeeded = desiredTime - timeSpent;
+        long total = value;
+        for (int i = 0; i < bucketTime.length; i++) {
+            if (timeNeeded < bucketTime[i]) {
+                double pct = timeNeeded/((double)bucketTime[i]);
+                total += (long)(pct * buckets[i]);
+                timeNeeded = 0;
+                break;
+            }
+            total += buckets[i];
+            timeNeeded -= bucketTime[i];
+        }
+        return total;
+    }
+
+    public void close() {
+        if (_task != null) {
+            _task.cancel();
+        }
+    }
+
+    private class Fresher extends TimerTask {
+        public void run () {
+            rotateSched(System.currentTimeMillis());
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/metric/internal/LatencyStatAndMetric.java b/storm-core/src/jvm/backtype/storm/metric/internal/LatencyStatAndMetric.java
new file mode 100644
index 0000000..614f95e
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/internal/LatencyStatAndMetric.java
@@ -0,0 +1,262 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.TimerTask;
+
+import backtype.storm.metric.api.IMetric;
+import backtype.storm.utils.Utils;
+
+/**
+ * Acts as a Latency Metric, but also keeps track of approximate latency
+ * for the last 10 mins, 3 hours, 1 day, and all time.
+ */
+public class LatencyStatAndMetric implements IMetric {
+    //The current lat and count buckets are protected by a different lock
+    // from the other buckets.  This is to reduce the lock contention
+    // When doing complex calculations.  Never grab the instance object lock
+    // while holding _currentLock to avoid deadlocks
+    private final Object _currentLock = new byte[0];
+    private long _currentLatBucket;
+    private long _currentCountBucket;
+
+    // All internal state except for the current buckets are
+    // protected using the Object Lock
+    private long _bucketStart;
+
+    //exact variable time, that is added to the current bucket
+    private long _exactExtraLat;
+    private long _exactExtraCount;
+ 
+    //10 min values
+    private final int _tmSize;
+    private final long[] _tmLatBuckets;
+    private final long[] _tmCountBuckets;
+    private final long[] _tmTime;
+    
+    //3 hour values
+    private final int _thSize;
+    private final long[] _thLatBuckets;
+    private final long[] _thCountBuckets;
+    private final long[] _thTime;
+
+    //1 day values
+    private final int _odSize;
+    private final long[] _odLatBuckets;
+    private final long[] _odCountBuckets;
+    private final long[] _odTime;
+ 
+    //all time
+    private long _allTimeLat;
+    private long _allTimeCount;
+
+    private final TimerTask _task;
+
+    /**
+     * @param numBuckets the number of buckets to divide the time periods into.
+     */
+    public LatencyStatAndMetric(int numBuckets) {
+        this(numBuckets, -1);
+    }
+
+    /**
+     * Constructor
+     * @param numBuckets the number of buckets to divide the time periods into.
+     * @param startTime if positive the simulated time to start the from.
+     */
+    LatencyStatAndMetric(int numBuckets, long startTime){
+        numBuckets = Math.max(numBuckets, 2);
+        //We want to capture the full time range, so the target size is as
+        // if we had one bucket less, then we do
+        _tmSize = 10 * 60 * 1000 / (numBuckets - 1);
+        _thSize = 3 * 60 * 60 * 1000 / (numBuckets - 1);
+        _odSize = 24 * 60 * 60 * 1000 / (numBuckets - 1);
+        if (_tmSize < 1 || _thSize < 1 || _odSize < 1) {
+            throw new IllegalArgumentException("number of buckets is too large to be supported");
+        }
+        _tmLatBuckets = new long[numBuckets];
+        _tmCountBuckets = new long[numBuckets];
+        _tmTime = new long[numBuckets];
+        _thLatBuckets = new long[numBuckets];
+        _thCountBuckets = new long[numBuckets];
+        _thTime = new long[numBuckets];
+        _odLatBuckets = new long[numBuckets];
+        _odCountBuckets = new long[numBuckets];
+        _odTime = new long[numBuckets];
+        _allTimeLat = 0;
+        _allTimeCount = 0;
+        _exactExtraLat = 0;
+        _exactExtraCount = 0;
+
+        _bucketStart = startTime >= 0 ? startTime : System.currentTimeMillis();
+        _currentLatBucket = 0;
+        _currentCountBucket = 0;
+        if (startTime < 0) {
+            _task = new Fresher();
+            MetricStatTimer._timer.scheduleAtFixedRate(_task, _tmSize, _tmSize);
+        } else {
+            _task = null;
+        }
+    }
+
+    /**
+     * Record a specific latency
+     *
+     * @param latency what we are recording
+     */
+    public void record(long latency) {
+        synchronized(_currentLock) {
+            _currentLatBucket += latency;
+            _currentCountBucket++;
+        }
+    }
+
+    @Override
+    public synchronized Object getValueAndReset() {
+        return getValueAndReset(System.currentTimeMillis());
+    }
+
+    synchronized Object getValueAndReset(long now) {
+        long lat;
+        long count;
+        synchronized(_currentLock) {
+            lat = _currentLatBucket;
+            count = _currentCountBucket;
+            _currentLatBucket = 0;
+            _currentCountBucket = 0;
+        }
+
+        long timeSpent = now - _bucketStart;
+        long exactExtraCountSum = count + _exactExtraCount;
+        double ret = Utils.zeroIfNaNOrInf(
+                ((double) (lat + _exactExtraLat)) / exactExtraCountSum);
+        _bucketStart = now;
+        _exactExtraLat = 0;
+        _exactExtraCount = 0;
+        rotateBuckets(lat, count, timeSpent);
+        return ret;
+    }
+
+    synchronized void rotateSched(long now) {
+        long lat;
+        long count;
+        synchronized(_currentLock) {
+            lat = _currentLatBucket;
+            count = _currentCountBucket;
+            _currentLatBucket = 0;
+            _currentCountBucket = 0;
+        }
+
+        long timeSpent = now - _bucketStart;
+        _exactExtraLat += lat;
+        _exactExtraCount += count;
+        _bucketStart = now;
+        rotateBuckets(lat, count, timeSpent);
+    }
+
+    synchronized void rotateBuckets(long lat, long count, long timeSpent) {
+        rotate(lat, count, timeSpent, _tmSize, _tmTime, _tmLatBuckets, _tmCountBuckets);
+        rotate(lat, count, timeSpent, _thSize, _thTime, _thLatBuckets, _thCountBuckets);
+        rotate(lat, count, timeSpent, _odSize, _odTime, _odLatBuckets, _odCountBuckets);
+        _allTimeLat += lat;
+        _allTimeCount += count;
+    }
+
+    private synchronized void rotate(long lat, long count, long timeSpent, long targetSize,
+            long [] times, long [] latBuckets, long [] countBuckets) {
+        times[0] += timeSpent;
+        latBuckets[0] += lat;
+        countBuckets[0] += count;
+
+        long currentTime = 0;
+        long currentLat = 0;
+        long currentCount = 0;
+        if (times[0] >= targetSize) {
+            for (int i = 0; i < latBuckets.length; i++) {
+                long tmpTime = times[i];
+                times[i] = currentTime;
+                currentTime = tmpTime;
+
+                long lt = latBuckets[i];
+                latBuckets[i] = currentLat;
+                currentLat = lt;
+
+                long cnt = countBuckets[i];
+                countBuckets[i] = currentCount;
+                currentCount = cnt;
+            }
+        }
+    }
+
+    /**
+     * @return a map of time window to average latency.
+     * Keys are "600" for last 10 mins
+     * "10800" for the last 3 hours
+     * "86400" for the last day
+     * ":all-time" for all time
+     */
+    public synchronized Map<String, Double> getTimeLatAvg() {
+        return getTimeLatAvg(System.currentTimeMillis());
+    }
+
+    synchronized Map<String, Double> getTimeLatAvg(long now) {
+        Map<String, Double> ret = new HashMap<>();
+        long lat;
+        long count;
+        synchronized(_currentLock) {
+            lat = _currentLatBucket;
+            count = _currentCountBucket;
+        }
+        long timeSpent = now - _bucketStart;
+        ret.put("600", readApproximateLatAvg(lat, count, timeSpent, _tmTime, _tmLatBuckets, _tmCountBuckets, 600 * 1000));
+        ret.put("10800", readApproximateLatAvg(lat, count, timeSpent, _thTime, _thLatBuckets, _thCountBuckets, 10800 * 1000));
+        ret.put("86400", readApproximateLatAvg(lat, count, timeSpent, _odTime, _odLatBuckets, _odCountBuckets, 86400 * 1000));
+        long allTimeCountSum = count + _allTimeCount;
+        ret.put(":all-time", Utils.zeroIfNaNOrInf(
+                (double) lat + _allTimeLat)/allTimeCountSum);
+        return ret;
+    }
+
+    double readApproximateLatAvg(long lat, long count, long timeSpent, long[] bucketTime,
+              long[] latBuckets, long[] countBuckets, long desiredTime) {
+        long timeNeeded = desiredTime - timeSpent;
+        long totalLat = lat;
+        long totalCount = count;
+        for (int i = 0; i < bucketTime.length && timeNeeded > 0; i++) {
+            //Don't pro-rate anything, it is all approximate so an extra bucket is not that bad.
+            totalLat += latBuckets[i];
+            totalCount += countBuckets[i];
+            timeNeeded -= bucketTime[i];
+        }
+        return Utils.zeroIfNaNOrInf(((double) totalLat) / totalCount);
+    }
+
+    public void close() {
+        if (_task != null) {
+            _task.cancel();
+        }
+    }
+
+    private class Fresher extends TimerTask {
+        public void run () {
+            rotateSched(System.currentTimeMillis());
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/metric/internal/MetricStatTimer.java b/storm-core/src/jvm/backtype/storm/metric/internal/MetricStatTimer.java
new file mode 100644
index 0000000..5f48793
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/internal/MetricStatTimer.java
@@ -0,0 +1,27 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Timer;
+
+/**
+ * Just holds a singleton metric/stat timer for use by metric/stat calculations
+ */
+class MetricStatTimer {
+    static Timer _timer = new Timer("metric/stat timer", true);
+}
diff --git a/storm-core/src/jvm/backtype/storm/metric/internal/MultiCountStatAndMetric.java b/storm-core/src/jvm/backtype/storm/metric/internal/MultiCountStatAndMetric.java
new file mode 100644
index 0000000..5fae4bf
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/internal/MultiCountStatAndMetric.java
@@ -0,0 +1,112 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Map;
+import java.util.List;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+import backtype.storm.metric.api.IMetric;
+
+/**
+ * Acts as a MultiCount Metric, but keeps track of approximate counts
+ * for the last 10 mins, 3 hours, 1 day, and all time. for the same keys
+ */
+public class MultiCountStatAndMetric<T> implements IMetric {
+    private ConcurrentHashMap<T, CountStatAndMetric> _counts = new ConcurrentHashMap<>();
+    private final int _numBuckets;
+
+    /**
+     * @param numBuckets the number of buckets to divide the time periods into.
+     */
+    public MultiCountStatAndMetric(int numBuckets) {
+        _numBuckets = numBuckets;
+    }
+
+    CountStatAndMetric get(T key) {
+        CountStatAndMetric c = _counts.get(key);
+        if (c == null) {
+            synchronized(this) {
+                c = _counts.get(key);
+                if (c == null) {
+                    c = new CountStatAndMetric(_numBuckets);
+                    _counts.put(key, c);
+                }
+            }
+        }
+        return c;
+    }
+
+    /**
+     * Increase the count by the given value.
+     *
+     * @param count number to count
+     */
+    public void incBy(T key, long count) {
+        get(key).incBy(count);
+    }
+
+    protected String keyToString(T key) {
+        if (key instanceof List) {
+            //This is a bit of a hack.  If it is a list, then it is [component, stream]
+            //we want to format this as component:stream
+            List<String> lk = (List<String>)key;
+            return lk.get(0) + ":" + lk.get(1);
+        }
+        return key.toString();
+    }
+
+    @Override
+    public Object getValueAndReset() {
+        Map<String, Long> ret = new HashMap<String, Long>();
+        for (Map.Entry<T, CountStatAndMetric> entry: _counts.entrySet()) {
+            String key = keyToString(entry.getKey());
+            //There could be collisions if keyToString returns only part of a result.
+            Long val = (Long)entry.getValue().getValueAndReset();
+            Long other = ret.get(key);
+            val += other == null ? 0l : other;
+            ret.put(key, val);
+        }
+        return ret;
+    }
+
+    public Map<String, Map<T, Long>> getTimeCounts() {
+        Map<String, Map<T, Long>> ret = new HashMap<>();
+        for (Map.Entry<T, CountStatAndMetric> entry: _counts.entrySet()) {
+            T key = entry.getKey();
+            Map<String, Long> toFlip = entry.getValue().getTimeCounts();
+            for (Map.Entry<String, Long> subEntry: toFlip.entrySet()) {
+                String time = subEntry.getKey();
+                Map<T, Long> tmp = ret.get(time);
+                if (tmp == null) {
+                    tmp = new HashMap<>();
+                    ret.put(time, tmp);
+                }
+                tmp.put(key, subEntry.getValue());
+            }
+        }
+        return ret;
+    }
+
+    public void close() {
+        for (CountStatAndMetric cc: _counts.values()) {
+            cc.close();
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/metric/internal/MultiLatencyStatAndMetric.java b/storm-core/src/jvm/backtype/storm/metric/internal/MultiLatencyStatAndMetric.java
new file mode 100644
index 0000000..032eef1
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/internal/MultiLatencyStatAndMetric.java
@@ -0,0 +1,109 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Map;
+import java.util.List;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+import backtype.storm.metric.api.IMetric;
+
+/**
+ * Acts as a Latnecy Metric for multiple keys, but keeps track of approximate counts
+ * for the last 10 mins, 3 hours, 1 day, and all time. for the same keys
+ */
+public class MultiLatencyStatAndMetric<T> implements IMetric {
+    private ConcurrentHashMap<T, LatencyStatAndMetric> _lat = new ConcurrentHashMap<>();
+    private final int _numBuckets;
+
+    /**
+     * @param numBuckets the number of buckets to divide the time periods into.
+     */
+    public MultiLatencyStatAndMetric(int numBuckets) {
+        _numBuckets = numBuckets;
+    }
+
+    LatencyStatAndMetric get(T key) {
+        LatencyStatAndMetric c = _lat.get(key);
+        if (c == null) {
+            synchronized(this) {
+                c = _lat.get(key);
+                if (c == null) {
+                    c = new LatencyStatAndMetric(_numBuckets);
+                    _lat.put(key, c);
+                }
+            }
+        }
+        return c;
+    }
+
+    /**
+     * Record a latency value
+     *
+     * @param latency the measurement to record
+     */
+    public void record(T key, long latency) {
+        get(key).record(latency);
+    }
+
+    protected String keyToString(T key) {
+        if (key instanceof List) {
+            //This is a bit of a hack.  If it is a list, then it is [component, stream]
+            //we want to format this as component:stream
+            List<String> lk = (List<String>)key;
+            return lk.get(0) + ":" + lk.get(1);
+        }
+        return key.toString();
+    }
+
+    @Override
+    public Object getValueAndReset() {
+        Map<String, Double> ret = new HashMap<String, Double>();
+        for (Map.Entry<T, LatencyStatAndMetric> entry: _lat.entrySet()) {
+            String key = keyToString(entry.getKey());
+            Double val = (Double)entry.getValue().getValueAndReset();
+            ret.put(key, val);
+        }
+        return ret;
+    }
+
+    public Map<String, Map<T, Double>> getTimeLatAvg() {
+        Map<String, Map<T, Double>> ret = new HashMap<>();
+        for (Map.Entry<T, LatencyStatAndMetric> entry: _lat.entrySet()) {
+            T key = entry.getKey();
+            Map<String, Double> toFlip = entry.getValue().getTimeLatAvg();
+            for (Map.Entry<String, Double> subEntry: toFlip.entrySet()) {
+                String time = subEntry.getKey();
+                Map<T, Double> tmp = ret.get(time);
+                if (tmp == null) {
+                    tmp = new HashMap<>();
+                    ret.put(time, tmp);
+                }
+                tmp.put(key, subEntry.getValue());
+            }
+        }
+        return ret;
+    }
+
+    public void close() {
+        for (LatencyStatAndMetric l: _lat.values()) {
+            l.close();
+        }
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/metric/internal/RateTracker.java b/storm-core/src/jvm/backtype/storm/metric/internal/RateTracker.java
new file mode 100644
index 0000000..f3eb6ae
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/metric/internal/RateTracker.java
@@ -0,0 +1,165 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class is a utility to track the rate of something.
+ */
+public class RateTracker{
+    private final int _bucketSizeMillis;
+    //Old Buckets and their length are only touched when rotating or gathering the metrics, which should not be that frequent
+    // As such all access to them should be protected by synchronizing with the RateTracker instance
+    private final long[] _bucketTime;
+    private final long[] _oldBuckets;
+    
+    private final AtomicLong _bucketStart;
+    private final AtomicLong _currentBucket;
+    
+    private final TimerTask _task;
+
+    /**
+     * @param validTimeWindowInMils events that happened before validTimeWindowInMils are not considered
+     *                        when reporting the rate.
+     * @param numBuckets the number of time sildes to divide validTimeWindows. The more buckets,
+     *                    the smother the reported results will be.
+     */
+    public RateTracker(int validTimeWindowInMils, int numBuckets) {
+        this(validTimeWindowInMils, numBuckets, -1);
+    }
+
+    /**
+     * Constructor
+     * @param validTimeWindowInMils events that happened before validTimeWindow are not considered
+     *                        when reporting the rate.
+     * @param numBuckets the number of time sildes to divide validTimeWindows. The more buckets,
+     *                    the smother the reported results will be.
+     * @param startTime if positive the simulated time to start the first bucket at.
+     */
+    RateTracker(int validTimeWindowInMils, int numBuckets, long startTime){
+        numBuckets = Math.max(numBuckets, 1);
+        _bucketSizeMillis = validTimeWindowInMils / numBuckets;
+        if (_bucketSizeMillis < 1 ) {
+            throw new IllegalArgumentException("validTimeWindowInMilis and numOfSildes cause each slide to have a window that is too small");
+        }
+        _bucketTime = new long[numBuckets - 1];
+        _oldBuckets = new long[numBuckets - 1];
+
+        _bucketStart = new AtomicLong(startTime >= 0 ? startTime : System.currentTimeMillis());
+        _currentBucket = new AtomicLong(0);
+        if (startTime < 0) {
+            _task = new Fresher();
+            MetricStatTimer._timer.scheduleAtFixedRate(_task, _bucketSizeMillis, _bucketSizeMillis);
+        } else {
+            _task = null;
+        }
+    }
+
+    /**
+     * Notify the tracker upon new arrivals
+     *
+     * @param count number of arrivals
+     */
+    public void notify(long count) {
+        _currentBucket.addAndGet(count);
+    }
+
+    /**
+     * @return the approximate average rate per second.
+     */
+    public synchronized double reportRate() {
+        return reportRate(System.currentTimeMillis());
+    }
+
+    synchronized double reportRate(long currentTime) {
+        long duration = Math.max(1l, currentTime - _bucketStart.get());
+        long events = _currentBucket.get();
+        for (int i = 0; i < _oldBuckets.length; i++) {
+            events += _oldBuckets[i];
+            duration += _bucketTime[i];
+        }
+
+        return events * 1000.0 / duration;
+    }
+
+    public void close() {
+        if (_task != null) {
+            _task.cancel();
+        }
+    }
+
+    /**
+     * Rotate the buckets a set number of times for testing purposes.
+     * @param numToEclipse the number of rotations to perform.
+     */
+    final void forceRotate(int numToEclipse, long interval) {
+        long time = _bucketStart.get();
+        for (int i = 0; i < numToEclipse; i++) {
+            time += interval;
+            rotateBuckets(time);
+        }
+    }
+
+    private synchronized void rotateBuckets(long time) {
+        long timeSpent = time - _bucketStart.getAndSet(time); 
+        long currentVal = _currentBucket.getAndSet(0);
+        for (int i = 0; i < _oldBuckets.length; i++) {
+            long tmpTime = _bucketTime[i];
+            _bucketTime[i] = timeSpent;
+            timeSpent = tmpTime;
+
+            long cnt = _oldBuckets[i];
+            _oldBuckets[i] = currentVal;
+            currentVal = cnt;
+        }
+    }
+
+    private class Fresher extends TimerTask {
+        public void run () {
+            rotateBuckets(System.currentTimeMillis());
+        }
+    }
+
+    public static void main (String args[]) throws Exception {
+        final int number = (args.length >= 1) ? Integer.parseInt(args[0]) : 100000000;
+        for (int i = 0; i < 10; i++) {
+            testRate(number);
+        }
+    }
+
+    private static void testRate(int number) {
+        RateTracker rt = new RateTracker(10000, 10);
+        long start = System.currentTimeMillis();
+        for (int i = 0; i < number; i++) {
+            rt.notify(1);
+            if ((i % 1000000) == 0) {
+                //There is an issue with some JVM versions where an integer for loop that takes a long time
+                // can starve other threads resulting in  the timer thread not getting called.
+                // This is a work around for that, and we still get the same results.
+                Thread.yield();
+            }
+        }
+        long end = System.currentTimeMillis();
+        double rate = rt.reportRate();
+        rt.close();
+        System.out.printf("time %,8d count %,8d rate %,15.2f reported rate %,15.2f\n", end-start,number, ((number * 1000.0)/(end-start)), rate);
+    }
+}
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..b17ca40
--- /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<>();
+            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();
+    }
+
+}
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);
+}
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..18eac60
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/networktopography/DefaultRackDNSToSwitchMapping.java
@@ -0,0 +1,52 @@
+/**
+ * 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.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<>();
+
+    @Override
+    public Map<String,String> resolve(List<String> names) {
+
+        Map<String, String> m = new HashMap<>();
+        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();
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/nimbus/ILeaderElector.java b/storm-core/src/jvm/backtype/storm/nimbus/ILeaderElector.java
index a54ae7e..5b54d46 100644
--- a/storm-core/src/jvm/backtype/storm/nimbus/ILeaderElector.java
+++ b/storm-core/src/jvm/backtype/storm/nimbus/ILeaderElector.java
@@ -1,10 +1,23 @@
+/**
+ * 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.nimbus;
 
-import org.apache.curator.framework.CuratorFramework;
-
 import java.io.Closeable;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
 import java.util.List;
 import java.util.Map;
 
@@ -15,7 +28,7 @@
 
     /**
      * Method guaranteed to be called as part of initialization of leader elector instance.
-     * @param conf
+     * @param conf configuration
      */
     void prepare(Map conf);
 
diff --git a/storm-core/src/jvm/backtype/storm/nimbus/ITopologyActionNotifierPlugin.java b/storm-core/src/jvm/backtype/storm/nimbus/ITopologyActionNotifierPlugin.java
new file mode 100644
index 0000000..5539c05
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/nimbus/ITopologyActionNotifierPlugin.java
@@ -0,0 +1,43 @@
+/**
+ * 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.nimbus;
+
+import java.util.Map;
+
+/**
+ * A plugin interface that gets invoked any time there is an action for a topology.
+ */
+public interface ITopologyActionNotifierPlugin {
+    /**
+     * Called once during nimbus initialization.
+     * @param StormConf
+     */
+    void prepare(Map StormConf);
+
+    /**
+     * When a new actions is executed for a topology, this method will be called.
+     * @param topologyName
+     * @param action
+     */
+    void notify(String topologyName, String action);
+
+    /**
+     * called during shutdown.
+     */
+    void cleanup();
+}
diff --git a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
index bf56e2d..539df62 100644
--- a/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
+++ b/storm-core/src/jvm/backtype/storm/nimbus/NimbusInfo.java
@@ -1,6 +1,25 @@
+/**
+ * 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.nimbus;
 
 import backtype.storm.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.Serializable;
 import java.net.InetAddress;
@@ -8,6 +27,7 @@
 import java.util.Map;
 
 public class NimbusInfo implements Serializable {
+    private static final Logger LOG = LoggerFactory.getLogger(NimbusInfo.class);
     private static final String DELIM = ":";
 
     private String host;
@@ -32,6 +52,11 @@
     public static NimbusInfo fromConf(Map conf) {
         try {
             String host = InetAddress.getLocalHost().getCanonicalHostName();
+            if (conf.containsKey(Config.STORM_LOCAL_HOSTNAME)) {
+                host = conf.get(Config.STORM_LOCAL_HOSTNAME).toString();
+                LOG.info("Overriding nimbus host to storm.local.hostname -> {}", host);
+            }
+
             int port = Integer.parseInt(conf.get(Config.NIMBUS_THRIFT_PORT).toString());
             return new NimbusInfo(host, port, false);
 
@@ -69,9 +94,7 @@
 
         if (isLeader != that.isLeader) return false;
         if (port != that.port) return false;
-        if (!host.equals(that.host)) return false;
-
-        return true;
+        return host.equals(that.host);
     }
 
     @Override
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
index e0c7cc7..ff2b233 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Cluster.java
@@ -25,12 +25,26 @@
 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;
+    private Map<String, SupervisorDetails> supervisors;
+    /**
+     * key: supervisor id, value: supervisor's total and used resources
+     */
+    private Map<String, Double[]> supervisorsResources;
+
+    /**
+     * key: rack, value: nodes in that rack
+     */
+    private Map<String, List<String>> networkTopography;
+
     /**
      * key: topologyId, value: topology's current assignments.
      */
@@ -41,29 +55,40 @@
     private Map<String, String> status;
 
     /**
+     * key topologyId, Value: requested and assigned resources (e.g., on-heap/off-heap mem, cpu) for each topology.
+     */
+    private Map<String, Double[]> resources;
+
+    /**
      * a map from hostname to supervisor id.
      */
-    private Map<String, List<String>>        hostToId;
-    
+    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);
         this.assignments = new HashMap<String, SchedulerAssignmentImpl>(assignments.size());
         this.assignments.putAll(assignments);
         this.status = new HashMap<String, String>();
+        this.resources = new HashMap<String, Double[]>();
+        this.supervisorsResources = new HashMap<String, Double[]>();
         this.hostToId = new HashMap<String, List<String>>();
-        for (String nodeId : supervisors.keySet()) {
-            SupervisorDetails supervisor = supervisors.get(nodeId);
+        for (Map.Entry<String, SupervisorDetails> entry : supervisors.entrySet()) {
+            String nodeId = entry.getKey();
+            SupervisorDetails supervisor = entry.getValue();
             String host = supervisor.getHost();
             if (!this.hostToId.containsKey(host)) {
                 this.hostToId.put(host, new ArrayList<String>());
             }
             this.hostToId.get(host).add(nodeId);
         }
+        this.conf = storm_conf;
     }
     
     public void setBlacklistedHosts(Set<String> hosts) {
@@ -95,10 +120,7 @@
     }
     
     /**
-     * Gets all the topologies which needs scheduling.
-     * 
-     * @param topologies
-     * @return
+     * @return all the topologies which needs scheduling.
      */
     public List<TopologyDetails> needsSchedulingTopologies(Topologies topologies) {
         List<TopologyDetails> ret = new ArrayList<TopologyDetails>();
@@ -123,19 +145,12 @@
     public boolean needsScheduling(TopologyDetails topology) {
         int desiredNumWorkers = topology.getNumWorkers();
         int assignedNumWorkers = this.getAssignedNumWorkers(topology);
-
-        if (desiredNumWorkers > assignedNumWorkers) {
-            return true;
-        }
-
-        return this.getUnassignedExecutors(topology).size() > 0;
+        return desiredNumWorkers > assignedNumWorkers || this.getUnassignedExecutors(topology).size() > 0;
     }
 
     /**
-     * Gets a executor -> component-id map which needs scheduling in this topology.
-     * 
      * @param topology
-     * @return
+     * @return a executor -> component-id map which needs scheduling in this topology.
      */
     public Map<ExecutorDetails, String> getNeedsSchedulingExecutorToComponents(TopologyDetails topology) {
         Collection<ExecutorDetails> allExecutors = new HashSet(topology.getExecutors());
@@ -150,16 +165,15 @@
     }
     
     /**
-     * Gets a component-id -> executors map which needs scheduling in this topology.
-     * 
      * @param topology
-     * @return
+     * @return a component-id -> executors map which needs scheduling in this topology.
      */
     public Map<String, List<ExecutorDetails>> getNeedsSchedulingComponentToExecutors(TopologyDetails topology) {
         Map<ExecutorDetails, String> executorToComponents = this.getNeedsSchedulingExecutorToComponents(topology);
         Map<String, List<ExecutorDetails>> componentToExecutors = new HashMap<String, List<ExecutorDetails>>();
-        for (ExecutorDetails executor : executorToComponents.keySet()) {
-            String component = executorToComponents.get(executor);
+        for (Map.Entry<ExecutorDetails, String> entry : executorToComponents.entrySet()) {
+            ExecutorDetails executor = entry.getKey();
+            String component = entry.getValue();
             if (!componentToExecutors.containsKey(component)) {
                 componentToExecutors.put(component, new ArrayList<ExecutorDetails>());
             }
@@ -173,9 +187,6 @@
 
     /**
      * Get all the used ports of this supervisor.
-     * 
-     * @param cluster
-     * @return
      */
     public Set<Integer> getUsedPorts(SupervisorDetails supervisor) {
         Map<String, SchedulerAssignment> assignments = this.getAssignments();
@@ -194,9 +205,6 @@
 
     /**
      * Return the available ports of this supervisor.
-     * 
-     * @param cluster
-     * @return
      */
     public Set<Integer> getAvailablePorts(SupervisorDetails supervisor) {
         Set<Integer> usedPorts = this.getUsedPorts(supervisor);
@@ -215,9 +223,6 @@
 
     /**
      * Return all the available slots on this supervisor.
-     * 
-     * @param cluster
-     * @return
      */
     public List<WorkerSlot> getAvailableSlots(SupervisorDetails supervisor) {
         Set<Integer> ports = this.getAvailablePorts(supervisor);
@@ -261,20 +266,17 @@
     }
 
     /**
-     * Gets the number of workers assigned to this topology.
-     * 
      * @param topology
-     * @return
+     * @return the number of workers assigned to this topology.
      */
     public int getAssignedNumWorkers(TopologyDetails topology) {
-        SchedulerAssignment assignment = this.getAssignmentById(topology.getId());
-        if (topology == null || assignment == null) {
+        SchedulerAssignment assignment = topology != null ? this.getAssignmentById(topology.getId()) : null;
+        if (assignment == null) {
             return 0;
         }
 
         Set<WorkerSlot> slots = new HashSet<WorkerSlot>();
         slots.addAll(assignment.getExecutorToSlot().values());
-
         return slots.size();
     }
 
@@ -304,9 +306,7 @@
     }
 
     /**
-     * Gets all the available slots in the cluster.
-     * 
-     * @return
+     * @return all the available worker slots in the cluster.
      */
     public List<WorkerSlot> getAvailableSlots() {
         List<WorkerSlot> slots = new ArrayList<WorkerSlot>();
@@ -354,10 +354,8 @@
     }
 
     /**
-     * Checks the specified slot is occupied.
-     * 
      * @param slot the slot be to checked.
-     * @return
+     * @return true if the specified slot is occupied.
      */
     public boolean isSlotOccupied(WorkerSlot slot) {
         for (SchedulerAssignment assignment : this.assignments.values()) {
@@ -427,7 +425,7 @@
         for (String topologyId : this.assignments.keySet()) {
             ret.put(topologyId, this.assignments.get(topologyId));
         }
-        
+
         return ret;
     }
 
@@ -438,6 +436,130 @@
         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 (Map.Entry<String, String> entry : resolvedSuperVisors.entrySet()) {
+                String hostName = entry.getKey();
+                String rack = entry.getValue();
+                List<String> nodesForRack = networkTopography.get(rack);
+                if (nodesForRack == null) {
+                    nodesForRack = new ArrayList<String>();
+                    networkTopography.put(rack, nodesForRack);
+                }
+                nodesForRack.add(hostName);
+            }
+        }
+        return networkTopography;
+    }
+
+    private String getStringFromStringList(Object o) {
+        StringBuilder sb = new StringBuilder();
+        for (String s : (List<String>) o) {
+            sb.append(s);
+            sb.append(" ");
+        }
+        return sb.toString();
+    }
+
+    /*
+    * Get heap memory usage for a worker's main process and logwriter process
+    * */
+    private Double getAssignedMemoryForSlot(Map topConf) {
+        Double totalWorkerMemory = 0.0;
+
+        String topologyWorkerChildopts = null;
+        if (topConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS) instanceof List) {
+            topologyWorkerChildopts = getStringFromStringList(topConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS));
+        } else {
+            topologyWorkerChildopts = Utils.getString(topConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS), null);
+        }
+        Double memTopologyWorkerChildopts = Utils.parseJvmHeapMemByChildOpts(topologyWorkerChildopts, null);
+
+        String workerChildopts = null;
+        if (topConf.get(Config.WORKER_CHILDOPTS) instanceof List) {
+            workerChildopts = getStringFromStringList(topConf.get(Config.WORKER_CHILDOPTS));
+        } else {
+            workerChildopts = Utils.getString(topConf.get(Config.WORKER_CHILDOPTS), null);
+        }
+        Double memWorkerChildopts = Utils.parseJvmHeapMemByChildOpts(workerChildopts, null);
+
+        if (memTopologyWorkerChildopts != null) {
+            totalWorkerMemory += memTopologyWorkerChildopts;
+        } else if (memWorkerChildopts != null) {
+            totalWorkerMemory += memWorkerChildopts;
+        } else {
+            totalWorkerMemory += Utils.getInt(topConf.get(Config.WORKER_HEAP_MEMORY_MB));
+        }
+
+        String topoWorkerLwChildopts = null;
+        if (topConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS) instanceof List) {
+            topoWorkerLwChildopts = getStringFromStringList(topConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS));
+        } else {
+            topoWorkerLwChildopts = Utils.getString(topConf.get(Config.TOPOLOGY_WORKER_LOGWRITER_CHILDOPTS), null);
+        }
+        if (topoWorkerLwChildopts != null) {
+            totalWorkerMemory += Utils.parseJvmHeapMemByChildOpts(topoWorkerLwChildopts, 0.0);
+        }
+        return totalWorkerMemory;
+    }
+
+    /*
+    * Update memory usage for each topology and each supervisor node after every round of scheduling
+    * */
+    public void updateAssignedMemoryForTopologyAndSupervisor(Topologies topologies) {
+        Map<String, Double> supervisorToAssignedMem = new HashMap<String, Double>();
+
+        for (Map.Entry<String, SchedulerAssignment> entry : this.getAssignments().entrySet()) {
+            String topId = entry.getValue().getTopologyId();
+            Map topConf = topologies.getById(topId).getConf();
+            Double assignedMemForTopology = 0.0;
+            Double assignedMemPerSlot = getAssignedMemoryForSlot(topConf);
+            for (WorkerSlot ws: entry.getValue().getSlots()) {
+                assignedMemForTopology += assignedMemPerSlot;
+                String nodeId = ws.getNodeId();
+                if (supervisorToAssignedMem.containsKey(nodeId)) {
+                    supervisorToAssignedMem.put(nodeId, supervisorToAssignedMem.get(nodeId) + assignedMemPerSlot);
+                } else {
+                    supervisorToAssignedMem.put(nodeId, assignedMemPerSlot);
+                }
+            }
+            if (this.getResourcesMap().containsKey(topId)) {
+                Double[] topo_resources = getResourcesMap().get(topId);
+                topo_resources[3] = assignedMemForTopology;
+            } else {
+                Double[] topo_resources = {0.0, 0.0, 0.0, 0.0, 0.0, 0.0};
+                topo_resources[3] = assignedMemForTopology;
+                this.setResources(topId, topo_resources);
+            }
+        }
+
+        for (Map.Entry<String, Double> entry : supervisorToAssignedMem.entrySet()) {
+            String nodeId = entry.getKey();
+            if (this.supervisorsResources.containsKey(nodeId)) {
+                Double[] supervisor_resources = supervisorsResources.get(nodeId);
+                supervisor_resources[2] = entry.getValue();
+            } else {
+                Double[] supervisor_resources = {0.0, 0.0, 0.0, 0.0};
+                supervisor_resources[2] = entry.getValue();
+                this.supervisorsResources.put(nodeId, supervisor_resources);
+            }
+        }
+    }
+
     public void setStatus(String topologyId, String status) {
         this.status.put(topologyId, status);
     }
@@ -445,4 +567,24 @@
     public Map<String, String> getStatusMap() {
         return this.status;
     }
+
+    public void setResources(String topologyId, Double[] resources) {
+        this.resources.put(topologyId, resources);
+    }
+
+    public void setResourcesMap(Map<String, Double[]> topologies_resources) {
+        this.resources.putAll(topologies_resources);
+    }
+
+    public Map<String, Double[]> getResourcesMap() {
+        return this.resources;
+    }
+
+    public void setSupervisorsResourcesMap(Map<String, Double[]> supervisors_resources) {
+        this.supervisorsResources.putAll(supervisors_resources);
+    }
+
+    public Map<String, Double[]> getSupervisorsResourcesMap() {
+        return this.supervisorsResources;
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignmentImpl.java b/storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignmentImpl.java
index 08af4b7..e06abf8 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignmentImpl.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/SchedulerAssignmentImpl.java
@@ -38,7 +38,7 @@
     
     public SchedulerAssignmentImpl(String topologyId, Map<ExecutorDetails, WorkerSlot> executorToSlots) {
         this.topologyId = topologyId;
-        this.executorToSlot = new HashMap<ExecutorDetails, WorkerSlot>(0);
+        this.executorToSlot = new HashMap<>(0);
         if (executorToSlots != null) {
             this.executorToSlot.putAll(executorToSlots);
         }
@@ -46,13 +46,11 @@
 
     @Override
     public Set<WorkerSlot> getSlots() {
-        return new HashSet(executorToSlot.values());
+        return new HashSet<>(executorToSlot.values());
     }    
     
     /**
      * Assign the slot to executors.
-     * @param slot
-     * @param executors
      */
     public void assign(WorkerSlot slot, Collection<ExecutorDetails> executors) {
         for (ExecutorDetails executor : executors) {
@@ -62,10 +60,9 @@
     
     /**
      * Release the slot occupied by this assignment.
-     * @param slot
      */
     public void unassignBySlot(WorkerSlot slot) {
-        List<ExecutorDetails> executors = new ArrayList<ExecutorDetails>();
+        List<ExecutorDetails> executors = new ArrayList<>();
         for (ExecutorDetails executor : this.executorToSlot.keySet()) {
             WorkerSlot ws = this.executorToSlot.get(executor);
             if (ws.equals(slot)) {
@@ -80,9 +77,8 @@
     }
 
     /**
-     * Does this slot occupied by this assignment?
      * @param slot
-     * @return
+     * @return true if slot is occupied by this assignment
      */
     public boolean isSlotOccupied(WorkerSlot slot) {
         return this.executorToSlot.containsValue(slot);
@@ -101,8 +97,7 @@
     }
 
     /**
-     * Return the executors covered by this assignments
-     * @return
+     * @return the executors covered by this assignments
      */
     public Set<ExecutorDetails> getExecutors() {
         return this.executorToSlot.keySet();
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/SupervisorDetails.java
index 7497f26..a748e11 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 @@
 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,29 +43,50 @@
      * all the ports of the supervisor
      */
     Set<Integer> allPorts;
+    /**
+     * Map containing a manifest of resources for the node the supervisor resides
+     */
+    private Map<String, Double> _total_resources;
 
-    public SupervisorDetails(String id, Object meta){
-        this.id = id;
-        this.meta = meta;
-        allPorts = new HashSet();
-    }
-    
-    public SupervisorDetails(String id, Object meta, Collection<Number> allPorts){
-        this.id = id;
-        this.meta = meta;
-        setAllPorts(allPorts);
-    }
+    public SupervisorDetails(String id, String host, Object meta, Object schedulerMeta,
+                             Collection<Number> allPorts, Map<String, Double> total_resources){
 
-    public SupervisorDetails(String id, String host, Object schedulerMeta, Collection<Number> allPorts){
         this.id = id;
         this.host = host;
+        this.meta = meta;
         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);
+    }
 
-        setAllPorts(allPorts);
+    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, null, meta, null, allPorts, null);
+    }
+
+    public SupervisorDetails(String id, String host, Object schedulerMeta, Collection<Number> allPorts) {
+        this(id, host, null, schedulerMeta, allPorts, null);
+    }
+
+    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) {
-        this.allPorts = new HashSet<Integer>();
+        this.allPorts = new HashSet<>();
         if(allPorts!=null) {
             for(Number n: allPorts) {
                 this.allPorts.add(n.intValue());
@@ -86,4 +113,16 @@
     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);
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
index 70af1b4..0828a73 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/Topologies.java
@@ -21,19 +21,22 @@
 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());
+        if(topologies==null) topologies = new HashMap<>();
+        this.topologies = new HashMap<>(topologies.size());
         this.topologies.putAll(topologies);
-        this.nameToId = new HashMap<String, String>(topologies.size());
+        this.nameToId = new HashMap<>(topologies.size());
         
-        for (String topologyId : topologies.keySet()) {
-            TopologyDetails topology = topologies.get(topologyId);
-            this.nameToId.put(topology.getName(), topologyId);
+        for (Map.Entry<String, TopologyDetails> entry : topologies.entrySet()) {
+            TopologyDetails topology = entry.getValue();
+            this.nameToId.put(topology.getName(), entry.getKey());
         }
     }
     
@@ -54,4 +57,14 @@
     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;
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java b/storm-core/src/jvm/backtype/storm/scheduler/TopologyDetails.java
index 6daf4ed..95aa5c8 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,59 +43,394 @@
     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);
+        this.executorToComponent = new HashMap<>(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;
     }
 
     public Map<ExecutorDetails, String> selectExecutorToComponent(Collection<ExecutorDetails> executors) {
-        Map<ExecutorDetails, String> ret = new HashMap<ExecutorDetails, String>(executors.size());
+        Map<ExecutorDetails, String> ret = new HashMap<>(executors.size());
         for (ExecutorDetails executor : executors) {
             String compId = this.executorToComponent.get(executor);
             if (compId != null) {
                 ret.put(executor, compId);
             }
         }
-        
+
         return ret;
     }
-    
+
     public Collection<ExecutorDetails> getExecutors() {
         return this.executorToComponent.keySet();
     }
+
+    private void initResourceList() {
+        _resourceList = new HashMap<>();
+        // 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)) {
+                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<>();
+
+        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;
+                    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;
+                    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<>();
+        for (ExecutorDetails exec : _resourceList.keySet()) {
+            ret.put(exec, getTotalMemReqTask(exec));
+        }
+        return ret;
+    }
+
+    /**
+     * Get the total CPU requirement for executor
+     * @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;
+    }
+
+    /**
+     * Note: The public API relevant to resource aware scheduling is unstable as of May 2015.
+     *       We reserve the right to change them.
+     *
+     * @return the total on-heap memory requested for this topology
+     */
+    public Double getTotalRequestedMemOnHeap() {
+        Double total_memonheap = 0.0;
+        for (ExecutorDetails exec : this.getExecutors()) {
+            Double exec_mem = getOnHeapMemoryRequirement(exec);
+            if (exec_mem != null) {
+                total_memonheap += exec_mem;
+            }
+        }
+        return total_memonheap;
+    }
+
+    /**
+     * Note: The public API relevant to resource aware scheduling is unstable as of May 2015.
+     *       We reserve the right to change them.
+     *
+     * @return the total off-heap memory requested for this topology
+     */
+    public Double getTotalRequestedMemOffHeap() {
+        Double total_memoffheap = 0.0;
+        for (ExecutorDetails exec : this.getExecutors()) {
+            Double exec_mem = getOffHeapMemoryRequirement(exec);
+            if (exec_mem != null) {
+                total_memoffheap += exec_mem;
+            }
+        }
+        return total_memoffheap;
+    }
+
+    /**
+     * Note: The public API relevant to resource aware scheduling is unstable as of May 2015.
+     *       We reserve the right to change them.
+     *
+     * @return the total cpu requested for this topology
+     */
+    public Double getTotalRequestedCpu() {
+        Double total_cpu = 0.0;
+        for (ExecutorDetails exec : this.getExecutors()) {
+            Double exec_cpu = getTotalCpuReqTask(exec);
+            if (exec_cpu != null) {
+                total_cpu += exec_cpu;
+            }
+        }
+        return total_cpu;
+    }
+
+    /**
+     * 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) {
+        return _resourceList != null && _resourceList.containsKey(exec);
+    }
+
+    /**
+     * add resource requirements for a executor
+     */
+    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
+     */
+    public void addDefaultResforExec(ExecutorDetails exec) {
+        Map<String, Double> defaultResourceList = new HashMap<>();
+        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);
+    }
+
+    /**
+     * initializes 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;
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java b/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
index c89b3bc..25892df 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/WorkerSlot.java
@@ -20,6 +20,12 @@
 public class WorkerSlot {
     String nodeId;
     int port;
+    // amount of on-heap memory allocated to it
+    double memOnHeap = 0.0;
+    // amount of off-heap memory allocated to it
+    double memOffHeap = 0.0;
+    // amount of cpu allocated to it
+    double cpu = 0.0;
     
     public WorkerSlot(String nodeId, Number port) {
         this.nodeId = nodeId;
@@ -34,6 +40,25 @@
         return port;
     }
 
+    public WorkerSlot allocateResource(double memOnHeap, double memOffHeap, double cpu) {
+        this.memOnHeap += memOnHeap;
+        this.memOffHeap += memOffHeap;
+        this.cpu += cpu;
+        return this;
+    }
+
+    public double getAllocatedMemOnHeap() {
+        return memOnHeap;
+    }
+
+    public double getAllocatedMemOffHeap() {
+        return memOffHeap;
+    }
+
+    public double getAllocatedCpu() {
+        return cpu;
+    }
+
     @Override
     public int hashCode() {
         return nodeId.hashCode() + 13 * ((Integer) port).hashCode();
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java
index 3053b5b..2e418e9 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/DefaultPool.java
@@ -37,8 +37,8 @@
  */
 public class DefaultPool extends NodePool {
   private static final Logger LOG = LoggerFactory.getLogger(DefaultPool.class);
-  private Set<Node> _nodes = new HashSet<Node>();
-  private HashMap<String, TopologyDetails> _tds = new HashMap<String, TopologyDetails>();
+  private Set<Node> _nodes = new HashSet<>();
+  private HashMap<String, TopologyDetails> _tds = new HashMap<>();
   
   @Override
   public void addTopology(TopologyDetails td) {
@@ -61,8 +61,8 @@
 
   @Override
   public Collection<Node> takeNodes(int nodesNeeded) {
-    HashSet<Node> ret = new HashSet<Node>();
-    LinkedList<Node> sortedNodes = new LinkedList<Node>(_nodes);
+    HashSet<Node> ret = new HashSet<>();
+    LinkedList<Node> sortedNodes = new LinkedList<>(_nodes);
     Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
     for (Node n: sortedNodes) {
       if (nodesNeeded <= ret.size()) {
@@ -95,7 +95,7 @@
   public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) {
     int nodesFound = 0;
     int slotsFound = 0;
-    LinkedList<Node> sortedNodes = new LinkedList<Node>(_nodes);
+    LinkedList<Node> sortedNodes = new LinkedList<>(_nodes);
     Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
     for (Node n: sortedNodes) {
       if (slotsNeeded <= 0) {
@@ -113,8 +113,8 @@
   
   @Override
   public Collection<Node> takeNodesBySlots(int slotsNeeded) {
-    HashSet<Node> ret = new HashSet<Node>();
-    LinkedList<Node> sortedNodes = new LinkedList<Node>(_nodes);
+    HashSet<Node> ret = new HashSet<>();
+    LinkedList<Node> sortedNodes = new LinkedList<>(_nodes);
     Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
     for (Node n: sortedNodes) {
       if (slotsNeeded <= 0) {
@@ -148,8 +148,8 @@
         }
         int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree + slotsAvailable);
         int executorsNotRunning = _cluster.getUnassignedExecutors(td).size();
-        LOG.debug("Slots... requested {} used {} free {} available {} to be used {}, executors not running {}", 
-            new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse, executorsNotRunning}); 
+        LOG.debug("Slots... requested {} used {} free {} available {} to be used {}, executors not running {}",
+                slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse, executorsNotRunning);
         if (slotsToUse <= 0) {
           if (executorsNotRunning > 0) {
             _cluster.setStatus(topId,"Not fully scheduled (No free slots in default pool) "+executorsNotRunning+" executors not scheduled");
@@ -180,9 +180,9 @@
         RoundRobinSlotScheduler slotSched = 
           new RoundRobinSlotScheduler(td, slotsToUse, _cluster);
         
-        LinkedList<Node> nodes = new LinkedList<Node>(_nodes);
+        LinkedList<Node> nodes = new LinkedList<>(_nodes);
         while (true) {
-          Node n = null;
+          Node n;
           do {
             if (nodes.isEmpty()) {
               throw new IllegalStateException("This should not happen, we" +
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java
index c625895..456900b 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/FreePool.java
@@ -35,7 +35,7 @@
  */
 public class FreePool extends NodePool {
   private static final Logger LOG = LoggerFactory.getLogger(FreePool.class);
-  private Set<Node> _nodes = new HashSet<Node>();
+  private Set<Node> _nodes = new HashSet<>();
   private int _totalSlots = 0;
 
   @Override
@@ -63,7 +63,7 @@
   
   @Override
   public Collection<Node> takeNodes(int nodesNeeded) {
-    HashSet<Node> ret = new HashSet<Node>();
+    HashSet<Node> ret = new HashSet<>();
     Iterator<Node> it = _nodes.iterator();
     while (it.hasNext() && nodesNeeded > ret.size()) {
       Node n = it.next();
@@ -86,7 +86,7 @@
 
   @Override
   public Collection<Node> takeNodesBySlots(int slotsNeeded) {
-    HashSet<Node> ret = new HashSet<Node>();
+    HashSet<Node> ret = new HashSet<>();
     Iterator<Node> it = _nodes.iterator();
     while (it.hasNext() && slotsNeeded > 0) {
       Node n = it.next();
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java
index dc7eded..25a6f25 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/IsolatedPool.java
@@ -41,9 +41,9 @@
  */
 public class IsolatedPool extends NodePool {
   private static final Logger LOG = LoggerFactory.getLogger(IsolatedPool.class);
-  private Map<String, Set<Node>> _topologyIdToNodes = new HashMap<String, Set<Node>>();
-  private HashMap<String, TopologyDetails> _tds = new HashMap<String, TopologyDetails>();
-  private HashSet<String> _isolated = new HashSet<String>();
+  private Map<String, Set<Node>> _topologyIdToNodes = new HashMap<>();
+  private HashMap<String, TopologyDetails> _tds = new HashMap<>();
+  private HashSet<String> _isolated = new HashSet<>();
   private int _maxNodes;
   private int _usedNodes;
 
@@ -57,7 +57,7 @@
     String topId = td.getId();
     LOG.debug("Adding in Topology {}", topId);
     SchedulerAssignment assignment = _cluster.getAssignmentById(topId);
-    Set<Node> assignedNodes = new HashSet<Node>();
+    Set<Node> assignedNodes = new HashSet<>();
     if (assignment != null) {
       for (WorkerSlot ws: assignment.getSlots()) {
         Node n = _nodeIdToNode.get(ws.getNodeId());
@@ -96,7 +96,7 @@
         LOG.debug("Scheduling topology {}",topId);
         Set<Node> allNodes = _topologyIdToNodes.get(topId);
         Number nodesRequested = (Number) td.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES);
-        int slotsToUse = 0;
+        int slotsToUse;
         if (nodesRequested == null) {
           slotsToUse = getNodesForNotIsolatedTop(td, allNodes, lesserPools);
         } else {
@@ -111,7 +111,7 @@
         RoundRobinSlotScheduler slotSched = 
           new RoundRobinSlotScheduler(td, slotsToUse, _cluster);
         
-        LinkedList<Node> sortedNodes = new LinkedList<Node>(allNodes);
+        LinkedList<Node> sortedNodes = new LinkedList<>(allNodes);
         Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC);
 
         LOG.debug("Nodes sorted by free space {}", sortedNodes);
@@ -157,9 +157,9 @@
     int nodesUsed = _topologyIdToNodes.get(topId).size();
     int nodesNeeded = nodesRequested - nodesUsed;
     LOG.debug("Nodes... requested {} used {} available from us {} " +
-        "avail from other {} needed {}", new Object[] {nodesRequested, 
-        nodesUsed, nodesFromUsAvailable, nodesFromOthersAvailable,
-        nodesNeeded});
+        "avail from other {} needed {}", nodesRequested,
+            nodesUsed, nodesFromUsAvailable, nodesFromOthersAvailable,
+            nodesNeeded);
     if ((nodesNeeded - nodesFromUsAvailable) > (_maxNodes - _usedNodes)) {
       _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. "
         + ((nodesNeeded - nodesFromUsAvailable) - (_maxNodes - _usedNodes)) 
@@ -224,8 +224,8 @@
       slotsAvailable = NodePool.slotsAvailable(lesserPools);
     }
     int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree + slotsAvailable);
-    LOG.debug("Slots... requested {} used {} free {} available {} to be used {}", 
-        new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse});
+    LOG.debug("Slots... requested {} used {} free {} available {} to be used {}",
+            slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse);
     if (slotsToUse <= 0) {
       _cluster.setStatus(topId, "Not Enough Slots Available to Schedule Topology");
       return 0;
@@ -233,7 +233,7 @@
     int slotsNeeded = slotsToUse - slotsFree;
     int numNewNodes = NodePool.getNodeCountIfSlotsWereTaken(slotsNeeded, lesserPools);
     LOG.debug("Nodes... new {} used {} max {}",
-        new Object[]{numNewNodes, _usedNodes, _maxNodes});
+            numNewNodes, _usedNodes, _maxNodes);
     if ((numNewNodes + _usedNodes) > _maxNodes) {
       _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. " +
       (numNewNodes - (_maxNodes - _usedNodes)) + " more nodes needed to run topology.");
@@ -249,7 +249,7 @@
   @Override
   public Collection<Node> takeNodes(int nodesNeeded) {
     LOG.debug("Taking {} from {}", nodesNeeded, this);
-    HashSet<Node> ret = new HashSet<Node>();
+    HashSet<Node> ret = new HashSet<>();
     for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
       if (!_isolated.contains(entry.getKey())) {
         Iterator<Node> it = entry.getValue().iterator();
@@ -293,7 +293,7 @@
 
   @Override
   public Collection<Node> takeNodesBySlots(int slotsNeeded) {
-    HashSet<Node> ret = new HashSet<Node>();
+    HashSet<Node> ret = new HashSet<>();
     for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
       if (!_isolated.contains(entry.getKey())) {
         Iterator<Node> it = entry.getValue().iterator();
@@ -321,9 +321,7 @@
     int slotsFound = 0;
     for (Entry<String, Set<Node>> entry: _topologyIdToNodes.entrySet()) {
       if (!_isolated.contains(entry.getKey())) {
-        Iterator<Node> it = entry.getValue().iterator();
-        while (it.hasNext()) {
-          Node n = it.next();
+        for (Node n : entry.getValue()) {
           if (n.isAlive()) {
             nodesFound++;
             int totalSlotsFree = n.totalSlots();
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java
index 320b388..6b77c63 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/MultitenantScheduler.java
@@ -44,9 +44,9 @@
   private Map<String, Number> getUserConf() {
     Map<String, Number> ret = (Map<String, Number>)_conf.get(Config.MULTITENANT_SCHEDULER_USER_POOLS);
     if (ret == null) {
-      ret = new HashMap<String, Number>();
+      ret = new HashMap<>();
     } else {
-      ret = new HashMap<String, Number>(ret); 
+      ret = new HashMap<>(ret);
     }
 
     Map fromFile = Utils.findAndReadConfigFile("multitenant-scheduler.yaml", false);
@@ -65,7 +65,7 @@
     
     Map<String, Number> userConf = getUserConf();
     
-    Map<String, IsolatedPool> userPools = new HashMap<String, IsolatedPool>();
+    Map<String, IsolatedPool> userPools = new HashMap<>();
     for (Map.Entry<String, Number> entry : userConf.entrySet()) {
       userPools.put(entry.getKey(), new IsolatedPool(entry.getValue().intValue()));
     }
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java
index 883c65f..6c2f06b 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/Node.java
@@ -40,8 +40,8 @@
  */
 public class Node {
   private static final Logger LOG = LoggerFactory.getLogger(Node.class);
-  private Map<String, Set<WorkerSlot>> _topIdToUsedSlots = new HashMap<String,Set<WorkerSlot>>();
-  private Set<WorkerSlot> _freeSlots = new HashSet<WorkerSlot>();
+  private Map<String, Set<WorkerSlot>> _topIdToUsedSlots = new HashMap<>();
+  private Set<WorkerSlot> _freeSlots = new HashSet<>();
   private final String _nodeId;
   private boolean _isAlive;
   
@@ -143,7 +143,7 @@
     }
     Set<WorkerSlot> usedSlots = _topIdToUsedSlots.get(topId);
     if (usedSlots == null) {
-      usedSlots = new HashSet<WorkerSlot>();
+      usedSlots = new HashSet<>();
       _topIdToUsedSlots.put(topId, usedSlots);
     }
     usedSlots.add(ws);
@@ -164,7 +164,7 @@
         _freeSlots.addAll(entry.getValue());
       }
     }
-    _topIdToUsedSlots = new HashMap<String,Set<WorkerSlot>>();
+    _topIdToUsedSlots = new HashMap<>();
   }
   
   /**
@@ -242,10 +242,7 @@
   
   @Override
   public boolean equals(Object other) {
-    if (other instanceof Node) {
-      return _nodeId.equals(((Node)other)._nodeId);
-    }
-    return false;
+      return other instanceof Node && _nodeId.equals(((Node) other)._nodeId);
   }
   
   @Override
@@ -295,13 +292,13 @@
   }
   
   public static Map<String, Node> getAllNodesFrom(Cluster cluster) {
-    Map<String, Node> nodeIdToNode = new HashMap<String, Node>();
+    Map<String, Node> nodeIdToNode = new HashMap<>();
     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 {} {}",
-          new Object[] {isAlive? "living":"dead", id, sup.getAllPorts()});
+              isAlive? "living":"dead", id, sup.getAllPorts());
       nodeIdToNode.put(id, new Node(id, sup.getAllPorts(), isAlive));
     }
     
diff --git a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java
index 21d1577..5a46df5 100644
--- a/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java
+++ b/storm-core/src/jvm/backtype/storm/scheduler/multitenant/NodePool.java
@@ -81,7 +81,7 @@
       
       Map<ExecutorDetails, String> execToComp = td.getExecutorToComponent();
       SchedulerAssignment assignment = _cluster.getAssignmentById(_topId);
-      _nodeToComps = new HashMap<String, Set<String>>();
+      _nodeToComps = new HashMap<>();
 
       if (assignment != null) {
         Map<ExecutorDetails, WorkerSlot> execToSlot = assignment.getExecutorToSlot();
@@ -90,14 +90,14 @@
           String nodeId = entry.getValue().getNodeId();
           Set<String> comps = _nodeToComps.get(nodeId);
           if (comps == null) {
-            comps = new HashSet<String>();
+            comps = new HashSet<>();
             _nodeToComps.put(nodeId, comps);
           }
           comps.add(execToComp.get(entry.getKey()));
         }
       }
       
-      _spreadToSchedule = new HashMap<String, List<ExecutorDetails>>();
+      _spreadToSchedule = new HashMap<>();
       List<String> spreadComps = (List<String>)td.getConf().get(Config.TOPOLOGY_SPREAD_COMPONENTS);
       if (spreadComps != null) {
         for (String comp: spreadComps) {
@@ -105,7 +105,7 @@
         }
       }
       
-      _slots = new LinkedList<Set<ExecutorDetails>>();
+      _slots = new LinkedList<>();
       for (int i = 0; i < slotsToUse; i++) {
         _slots.add(new HashSet<ExecutorDetails>());
       }
@@ -118,7 +118,7 @@
           _spreadToSchedule.get(entry.getKey()).addAll(entry.getValue());
         } else {
           for (ExecutorDetails ed: entry.getValue()) {
-            LOG.debug("Assigning {} {} to slot {}", new Object[]{entry.getKey(), ed, at});
+            LOG.debug("Assigning {} {} to slot {}", entry.getKey(), ed, at);
             _slots.get(at).add(ed);
             at++;
             if (at >= _slots.size()) {
@@ -151,7 +151,7 @@
         String nodeId = n.getId();
         Set<String> nodeComps = _nodeToComps.get(nodeId);
         if (nodeComps == null) {
-          nodeComps = new HashSet<String>();
+          nodeComps = new HashSet<>();
           _nodeToComps.put(nodeId, nodeComps);
         }
         for (Entry<String, List<ExecutorDetails>> entry: _spreadToSchedule.entrySet()) {
@@ -251,7 +251,7 @@
   
   public static Collection<Node> takeNodesBySlot(int slotsNeeded,NodePool[] pools) {
     LOG.debug("Trying to grab {} free slots from {}",slotsNeeded, pools);
-    HashSet<Node> ret = new HashSet<Node>();
+    HashSet<Node> ret = new HashSet<>();
     for (NodePool pool: pools) {
       Collection<Node> got = pool.takeNodesBySlots(slotsNeeded);
       ret.addAll(got);
@@ -266,7 +266,7 @@
   
   public static Collection<Node> takeNodes(int nodesNeeded,NodePool[] pools) {
     LOG.debug("Trying to grab {} free nodes from {}",nodesNeeded, pools);
-    HashSet<Node> ret = new HashSet<Node>();
+    HashSet<Node> ret = new HashSet<>();
     for (NodePool pool: pools) {
       Collection<Node> got = pool.takeNodes(nodesNeeded);
       ret.addAll(got);
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;
+    }
+}
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..1f2e795
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/RAS_Node.java
@@ -0,0 +1,575 @@
+/**
+ * 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);
+    }
+
+    /**
+     * Allocate Mem and CPU resources to the assigned slot for the topology's executors.
+     * @param td the TopologyDetails that the slot is assigned to.
+     * @param executors the executors to run in that slot.
+     * @param slot the slot to allocate resource to
+     */
+    public void allocateResourceToSlot (TopologyDetails td, Collection<ExecutorDetails> executors, WorkerSlot slot) {
+        double onHeapMem = 0.0;
+        double offHeapMem = 0.0;
+        double cpu = 0.0;
+        for (ExecutorDetails exec : executors) {
+            Double onHeapMemForExec = td.getOnHeapMemoryRequirement(exec);
+            if (onHeapMemForExec != null) {
+                onHeapMem += onHeapMemForExec;
+            }
+            Double offHeapMemForExec = td.getOffHeapMemoryRequirement(exec);
+            if (offHeapMemForExec != null) {
+                offHeapMem += offHeapMemForExec;
+            }
+            Double cpuForExec = td.getTotalCpuReqTask(exec);
+            if (cpuForExec != null) {
+                cpu += cpuForExec;
+            }
+        }
+        slot.allocateResource(onHeapMem, offHeapMem, cpu);
+    }
+
+    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 {
+            allocateResourceToSlot(td, executors, target);
+            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);
+    }
+}
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..c7c7867
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/ResourceAwareScheduler.java
@@ -0,0 +1,183 @@
+/**
+ * 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.getUnassignedExecutors(td).size() > 0) {
+                LOG.debug("/********Scheduling topology {} ************/", topId);
+
+                schedulerAssignmentMap = RAStrategy.schedule(td);
+
+                double requestedMemOnHeap = td.getTotalRequestedMemOnHeap();
+                double requestedMemOffHeap = td.getTotalRequestedMemOffHeap();
+                double requestedCpu = td.getTotalRequestedCpu();
+                double assignedMemOnHeap = 0.0;
+                double assignedMemOffHeap = 0.0;
+                double assignedCpu = 0.0;
+
+                if (schedulerAssignmentMap != null) {
+                    try {
+                        Set<String> nodesUsed = new HashSet<String>();
+                        int assignedWorkers = schedulerAssignmentMap.keySet().size();
+                        for (Map.Entry<WorkerSlot, Collection<ExecutorDetails>> workerToTasksEntry : schedulerAssignmentMap.entrySet()) {
+                            WorkerSlot targetSlot = workerToTasksEntry.getKey();
+                            Collection<ExecutorDetails> execsNeedScheduling = workerToTasksEntry.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());
+                            }
+                            assignedMemOnHeap += targetSlot.getAllocatedMemOnHeap();
+                            assignedMemOffHeap += targetSlot.getAllocatedMemOffHeap();
+                            assignedCpu += targetSlot.getAllocatedCpu();
+                        }
+                        LOG.debug("Topology: {} assigned to {} nodes on {} workers", td.getId(), nodesUsed.size(), assignedWorkers);
+                        cluster.setStatus(td.getId(), "Fully Scheduled");
+                    } catch (IllegalStateException ex) {
+                        LOG.error(ex.toString());
+                        LOG.error("Unsuccessful in scheduling", td.getId());
+                        cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                    }
+                } else {
+                    LOG.error("Unsuccessful in scheduling", td.getId());
+                    cluster.setStatus(td.getId(), "Unsuccessful in scheduling");
+                }
+                Double[] resources = {requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+                        assignedMemOnHeap, assignedMemOffHeap, assignedCpu};
+                LOG.debug("setResources for {}: requested on-heap mem, off-heap mem, cpu: {} {} {} " +
+                        "assigned on-heap mem, off-heap mem, cpu: {} {} {}",
+                        td.getId(), requestedMemOnHeap, requestedMemOffHeap, requestedCpu,
+                        assignedMemOnHeap, assignedMemOffHeap, assignedCpu);
+                cluster.setResources(td.getId(), resources);
+            } else {
+                cluster.setStatus(td.getId(), "Fully Scheduled");
+            }
+        }
+        updateSupervisorsResources(cluster, topologies);
+    }
+
+    private void updateSupervisorsResources(Cluster cluster, Topologies topologies) {
+        Map<String, Double[]> supervisors_resources = new HashMap<String, Double[]>();
+        Map<String, RAS_Node> nodes = RAS_Node.getAllNodesFrom(cluster, topologies);
+        for (Map.Entry<String, RAS_Node> entry : nodes.entrySet()) {
+            RAS_Node node = entry.getValue();
+            Double totalMem = node.getTotalMemoryResources();
+            Double totalCpu = node.getTotalCpuResources();
+            Double usedMem = totalMem - node.getAvailableMemoryResources();
+            Double usedCpu = totalCpu - node.getAvailableCpuResources();
+            Double[] resources = {totalMem, totalCpu, usedMem, usedCpu};
+            supervisors_resources.put(entry.getKey(), resources);
+        }
+        cluster.setSupervisorsResourcesMap(supervisors_resources);
+    }
+
+    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();
+    }
+}
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));
+        }
+    }
+}
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);
+}
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..3fe37dd
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/scheduler/resource/strategies/ResourceAwareStrategy.java
@@ -0,0 +1,479 @@
+/**
+ * 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 static final Logger LOG = LoggerFactory.getLogger(ResourceAwareStrategy.class);
+    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();
+        _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 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<>();
+        LOG.debug("ExecutorsNeedScheduling: {}", unassignedExecutors);
+        Collection<ExecutorDetails> scheduledTasks = new ArrayList<>();
+        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<>(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)) {
+                    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;
+      // 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<>();
+        for (RAS_Node n : nodes) {
+            if(n.getFreeSlots().size()>0) {
+                if (n.getAvailableMemoryResources() >= taskMem
+                        && n.getAvailableCpuResources() >= taskCPU) {
+                    double a = Math.pow(((taskCPU - n.getAvailableCpuResources())/(n.getAvailableCpuResources() + 1))
+                            * this.CPU_WEIGHT, 2);
+                    double b = Math.pow(((taskMem - n.getAvailableMemoryResources())/(n.getAvailableMemoryResources() + 1))
+                            * 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())) {
+            return 0.0;
+        } else if (this.NodeToCluster(src).equals(this.NodeToCluster(dest))) {
+            return 0.5;
+        } else {
+            return 1.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<>();
+        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<>();
+        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<>();
+        for(RAS_Node node : nodes) {
+            workers.addAll(node.getFreeSlots());
+        }
+        return workers;
+    }
+
+    private List<WorkerSlot> getAvailableWorker() {
+        List<WorkerSlot> workers = new LinkedList<>();
+        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<>();
+
+        /* 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<>();
+                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<>();
+                    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<>();
+        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 currently.  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);
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
index ac3fb53..6f6969e 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/AuthUtils.java
@@ -21,26 +21,30 @@
 import javax.security.auth.login.Configuration;
 import javax.security.auth.login.AppConfigurationEntry;
 import javax.security.auth.Subject;
-import java.security.NoSuchAlgorithmException;
 import java.security.URIParameter;
+import java.security.MessageDigest;
 
 import backtype.storm.security.INimbusCredentialPlugin;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Collection;
 import java.util.Set;
 import java.util.HashSet;
 import java.util.Map;
-import java.util.concurrent.ExecutorService;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.lang.StringBuilder;
 
 public class AuthUtils {
     private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class);
     public static final String LOGIN_CONTEXT_SERVER = "StormServer";
     public static final String LOGIN_CONTEXT_CLIENT = "StormClient";
+    public static final String LOGIN_CONTEXT_PACEMAKER_DIGEST = "PacemakerDigest";
+    public static final String LOGIN_CONTEXT_PACEMAKER_SERVER = "PacemakerServer";
+    public static final String LOGIN_CONTEXT_PACEMAKER_CLIENT = "PacemakerClient";
     public static final String SERVICE = "storm_thrift_server";
 
     /**
@@ -71,12 +75,41 @@
     }
 
     /**
+     * Pull a set of keys out of a Configuration.
+     * @param conf The config to pull the key/value pairs out of.
+     * @param conf_entry The app configuration entry name to get stuff from.
+     * @return Return a map of the configs in conf.
+     */
+    public static SortedMap<String, ?> PullConfig(Configuration conf,
+                                            String conf_entry) throws IOException {
+        if(conf == null) {
+            return null;
+        }
+        AppConfigurationEntry configurationEntries[] = conf.getAppConfigurationEntry(conf_entry);
+        if(configurationEntries == null) {
+            String errorMessage = "Could not find a '" + conf_entry
+                + "' entry in this configuration: Client cannot start.";
+            throw new IOException(errorMessage);
+        }
+
+        TreeMap<String, Object> results = new TreeMap<>();
+
+        for(AppConfigurationEntry entry: configurationEntries) {
+            Map<String, ?> options = entry.getOptions();
+            for(String key : options.keySet()) {
+                results.put(key, options.get(key));
+            }
+        }
+        return results;
+    }
+
+    /**
      * Construct a principal to local plugin
-     * @param conf storm configuration
+     * @param storm_conf storm configuration
      * @return the plugin
      */
     public static IPrincipalToLocal GetPrincipalToLocalPlugin(Map storm_conf) {
-        IPrincipalToLocal ptol = null;
+        IPrincipalToLocal ptol;
         try {
           String ptol_klassName = (String) storm_conf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN);
           Class klass = Class.forName(ptol_klassName);
@@ -90,11 +123,11 @@
 
     /**
      * Construct a group mapping service provider plugin
-     * @param conf storm configuration
+     * @param storm_conf storm configuration
      * @return the plugin
      */
     public static IGroupMappingServiceProvider GetGroupMappingServiceProviderPlugin(Map storm_conf) {
-        IGroupMappingServiceProvider gmsp = null;
+        IGroupMappingServiceProvider gmsp;
         try {
             String gmsp_klassName = (String) storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN);
             Class klass = Class.forName(gmsp_klassName);
@@ -107,13 +140,13 @@
     }
 
     /**
-     * Get all of the configured Credential Renwer Plugins.
-     * @param storm_conf the storm configuration to use.
+     * Get all of the configured Credential Renewer Plugins.
+     * @param conf the storm configuration to use.
      * @return the configured credential renewers.
      */
     public static Collection<ICredentialsRenewer> GetCredentialRenewers(Map conf) {
         try {
-            Set<ICredentialsRenewer> ret = new HashSet<ICredentialsRenewer>();
+            Set<ICredentialsRenewer> ret = new HashSet<>();
             Collection<String> clazzes = (Collection<String>)conf.get(Config.NIMBUS_CREDENTIAL_RENEWERS);
             if (clazzes != null) {
                 for (String clazz : clazzes) {
@@ -135,7 +168,7 @@
      */
     public static Collection<INimbusCredentialPlugin> getNimbusAutoCredPlugins(Map conf) {
         try {
-            Set<INimbusCredentialPlugin> ret = new HashSet<INimbusCredentialPlugin>();
+            Set<INimbusCredentialPlugin> ret = new HashSet<>();
             Collection<String> clazzes = (Collection<String>)conf.get(Config.NIMBUS_AUTO_CRED_PLUGINS);
             if (clazzes != null) {
                 for (String clazz : clazzes) {
@@ -157,7 +190,7 @@
      */
     public static Collection<IAutoCredentials> GetAutoCredentials(Map storm_conf) {
         try {
-            Set<IAutoCredentials> autos = new HashSet<IAutoCredentials>();
+            Set<IAutoCredentials> autos = new HashSet<>();
             Collection<String> clazzes = (Collection<String>)storm_conf.get(Config.TOPOLOGY_AUTO_CREDENTIALS);
             if (clazzes != null) {
                 for (String clazz : clazzes) {
@@ -216,11 +249,9 @@
 
     /**
      * Construct a transport plugin per storm configuration
-     * @param conf storm configuration
-     * @return
      */
     public static ITransportPlugin GetTransportPlugin(ThriftConnectionType type, Map storm_conf, Configuration login_conf) {
-        ITransportPlugin  transportPlugin = null;
+        ITransportPlugin  transportPlugin;
         try {
             String transport_plugin_klassName = type.getTransportPlugin(storm_conf);
             Class klass = Class.forName(transport_plugin_klassName);
@@ -234,7 +265,7 @@
 
     private static IHttpCredentialsPlugin GetHttpCredentialsPlugin(Map conf,
             String klassName) {
-        IHttpCredentialsPlugin plugin = null;
+        IHttpCredentialsPlugin plugin;
         try {
             Class klass = Class.forName(klassName);
             plugin = (IHttpCredentialsPlugin)klass.newInstance();
@@ -281,4 +312,37 @@
         }
         return null;
     }
+
+    private static final String USERNAME = "username";
+    private static final String PASSWORD = "password";
+
+    public static String makeDigestPayload(Configuration login_config, String config_section) {
+        String username = null;
+        String password = null;
+        try {
+            Map<String, ?> results = AuthUtils.PullConfig(login_config, config_section);
+            username = (String)results.get(USERNAME);
+            password = (String)results.get(PASSWORD);
+        } catch (Exception e) {
+            LOG.error("Failed to pull username/password out of jaas conf", e);
+        }
+
+        if(username == null || password == null) {
+            return null;
+        }
+
+        try {
+            MessageDigest digest = MessageDigest.getInstance("SHA-512");
+            byte[] output = digest.digest((username + ":" + password).getBytes());
+
+            StringBuilder builder = new StringBuilder();
+            for(byte b : output) {
+                builder.append(String.format("%02x", b));
+            }
+            return builder.toString();
+        } catch (java.security.NoSuchAlgorithmException e) {
+            LOG.error("Cant run SHA-512 digest. Algorithm not available.", e);
+            throw new RuntimeException(e);
+        }
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java
index e2469e5..9c81cdf 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java
@@ -28,8 +28,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import backtype.storm.security.auth.ReqContext;
-
 public class DefaultHttpCredentialsPlugin implements IHttpCredentialsPlugin {
     private static final Logger LOG =
             LoggerFactory.getLogger(DefaultHttpCredentialsPlugin.class);
@@ -50,7 +48,7 @@
      */
     @Override
     public String getUserName(HttpServletRequest req) {
-        Principal princ = null;
+        Principal princ;
         if (req != null && (princ = req.getUserPrincipal()) != null) {
             String userName = princ.getName();
             if (userName != null && !userName.isEmpty()) {
@@ -83,7 +81,7 @@
             userName = doAsUser;
         }
 
-        Set<Principal> principals = new HashSet<Principal>();
+        Set<Principal> principals = new HashSet<>();
         if(userName != null) {
             Principal p = new SingleUserPrincipal(userName);
             principals.add(p);
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java
index 729d744..9f95101 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/DefaultPrincipalToLocal.java
@@ -28,7 +28,6 @@
 public class DefaultPrincipalToLocal implements IPrincipalToLocal {
     /**
      * Invoked once immediately after construction
-     * @param conf Storm configuration 
      */
     public void prepare(Map storm_conf) {}
     
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/IAuthorizer.java
index d592bb7..ff1e2ba 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/IAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/IAuthorizer.java
@@ -32,7 +32,7 @@
 public interface IAuthorizer {
     /**
      * Invoked once immediately after construction
-     * @param conf Storm configuration 
+     * @param storm_conf Storm configuration
      */
     void prepare(Map storm_conf);
     
@@ -40,7 +40,7 @@
      * permit() method is invoked for each incoming Thrift request.
      * @param context request context includes info about 
      * @param operation operation name
-     * @param topology_storm configuration of targeted topology 
+     * @param topology_conf configuration of targeted topology
      * @return true if the request is authorized, false if reject
      */
     public boolean permit(ReqContext context, String operation, Map topology_conf);
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java
index 3eaf6c4..9a6f02e 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ICredentialsRenewer.java
@@ -18,11 +18,10 @@
 
 package backtype.storm.security.auth;
 
-import java.util.Collection;
 import java.util.Map;
 
 /**
- * Provides a way to renew credentials on behelf of a user.
+ * Provides a way to renew credentials on behalf of a user.
  */
 public interface ICredentialsRenewer {
 
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java
index a012ce4..0b57eca 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/IHttpCredentialsPlugin.java
@@ -21,8 +21,6 @@
 import java.util.Map;
 import javax.servlet.http.HttpServletRequest;
 
-import backtype.storm.security.auth.ReqContext;
-
 /**
  * Interface for handling credentials in an HttpServletRequest
  */
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java
index fca3d37..e938d39 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/IPrincipalToLocal.java
@@ -28,7 +28,7 @@
 public interface IPrincipalToLocal {
     /**
      * Invoked once immediately after construction
-     * @param conf Storm configuration 
+     * @param storm_conf Storm configuration
      */
     void prepare(Map storm_conf);
     
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java
index 5ba2557..ba09fad 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ITransportPlugin.java
@@ -18,9 +18,7 @@
 package backtype.storm.security.auth;
 
 import java.io.IOException;
-import java.security.Principal;
 import java.util.Map;
-import java.util.concurrent.ExecutorService;
 
 import javax.security.auth.login.Configuration;
 
@@ -29,8 +27,6 @@
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 
-import backtype.storm.security.auth.ThriftConnectionType;
-
 /**
  * Interface for Thrift Transport plugin
  */
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java b/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java
index 35c7788..1f67c14 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/KerberosPrincipalToLocal.java
@@ -28,7 +28,7 @@
 
     /**
      * Invoked once immediately after construction
-     * @param conf Storm configuration 
+     * @param storm_conf Storm configuration
      */
     public void prepare(Map storm_conf) {}
     
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java b/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java
index a252f85..31aeef9 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ReqContext.java
@@ -22,8 +22,6 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.net.InetAddress;
 import com.google.common.annotations.VisibleForTesting;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.security.AccessControlContext;
 import java.security.AccessController;
@@ -44,17 +42,20 @@
     private Map _storm_conf;
     private Principal realPrincipal;
 
-    private static final Logger LOG = LoggerFactory.getLogger(ReqContext.class);
-
-
     /**
-     * Get a request context associated with current thread
-     * @return
+     * @return a request context associated with current thread
      */
     public static ReqContext context() {
         return ctxt.get();
     }
 
+    /**
+     * Reset the context back to a default.  used for testing.
+     */
+    public static void reset() {
+        ctxt.remove();
+    }
+
     //each thread will have its own request context
     private static final ThreadLocal < ReqContext > ctxt = 
             new ThreadLocal < ReqContext > () {
@@ -125,8 +126,7 @@
     }
 
     /**
-     * Returns true if this request is an impersonation request.
-     * @return
+     * @return true if this request is an impersonation request.
      */
     public boolean isImpersonating() {
         return this.realPrincipal != null;
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java
index 7013cd4..92004fa 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/SaslTransportPlugin.java
@@ -45,10 +45,6 @@
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.transport.TTransportFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import backtype.storm.security.auth.ThriftConnectionType;
 
 /**
  * Base class for SASL authentication plugin.
@@ -57,7 +53,6 @@
     protected ThriftConnectionType type;
     protected Map storm_conf;
     protected Configuration login_conf;
-    private static final Logger LOG = LoggerFactory.getLogger(SaslTransportPlugin.class);
 
     @Override
     public void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf) {
@@ -95,7 +90,7 @@
 
     /**
      * All subclass must implement this method
-     * @return
+     * @return server transport factory
      * @throws IOException
      */
     protected abstract TTransportFactory getServerTransportFactory() throws IOException;
@@ -162,11 +157,8 @@
         public boolean equals(Object o) {
             if (this == o) {
                 return true;
-            } else if (o == null || getClass() != o.getClass()) {
-                return false;
-            } else {
-                return (name.equals(((User) o).name));
             }
+            return !(o == null || getClass() != o.getClass()) && (name.equals(((User) o).name));
         }
 
         @Override
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java
index d83cf1c..5f53c17 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ShellBasedGroupsMapping.java
@@ -35,7 +35,7 @@
 public class ShellBasedGroupsMapping implements
                                              IGroupMappingServiceProvider {
 
-    public static Logger LOG = LoggerFactory.getLogger(ShellBasedGroupsMapping.class);
+    public static final Logger LOG = LoggerFactory.getLogger(ShellBasedGroupsMapping.class);
     public TimeCacheMap<String, Set<String>> cachedGroups;
 
     /**
@@ -45,7 +45,7 @@
     @Override
     public void prepare(Map storm_conf) {
         int timeout = Utils.getInt(storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS));
-        cachedGroups = new TimeCacheMap<String, Set<String>>(timeout);
+        cachedGroups = new TimeCacheMap<>(timeout);
     }
 
     /**
@@ -73,18 +73,18 @@
      * @throws IOException if encounter any error when running the command
      */
     private static Set<String> getUnixGroups(final String user) throws IOException {
-        String result = "";
+        String result;
         try {
             result = ShellUtils.execCommand(ShellUtils.getGroupsForUserCommand(user));
         } catch (ExitCodeException e) {
             // if we didn't get the group - just return empty list;
             LOG.debug("unable to get groups for user " + user + ".ShellUtils command failed with exit code "+ e.getExitCode());
-            return new HashSet<String>();
+            return new HashSet<>();
         }
 
         StringTokenizer tokenizer =
             new StringTokenizer(result, ShellUtils.TOKEN_SEPARATOR_REGEX);
-        Set<String> groups = new HashSet<String>();
+        Set<String> groups = new HashSet<>();
         while (tokenizer.hasMoreTokens()) {
             groups.add(tokenizer.nextToken());
         }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java
index 2abcdae..b73c585 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/SimpleTransportPlugin.java
@@ -45,8 +45,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import backtype.storm.security.auth.ThriftConnectionType;
-
 /**
  * Simple transport for Thrift plugin.
  * 
@@ -75,7 +73,7 @@
 
         THsHaServer.Args server_args = new THsHaServer.Args(serverTransport).
                 processor(new SimpleWrapProcessor(processor)).
-                workerThreads(numWorkerThreads).
+                maxWorkerThreads(numWorkerThreads).
                 protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize, -1));
 
         if (queueSize != null) {
@@ -146,12 +144,12 @@
             if (s == null) {
               final String user = (String)storm_conf.get("debug.simple.transport.user");
               if (user != null) {
-                HashSet<Principal> principals = new HashSet<Principal>();
+                HashSet<Principal> principals = new HashSet<>();
                 principals.add(new Principal() {
                   public String getName() { return user; }
                   public String toString() { return user; }
                 });
-                s = new Subject(true, principals, new HashSet<Object>(), new HashSet<Object>());
+                s = new Subject(true, principals, new HashSet<>(), new HashSet<>());
               }
             }
             req_context.setSubject(s);
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java b/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java
index 6af17fa..0cadba6 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/SingleUserPrincipal.java
@@ -33,10 +33,7 @@
 
     @Override
     public boolean equals(Object another) {
-        if (another instanceof SingleUserPrincipal) {
-            return _userName.equals(((SingleUserPrincipal)another)._userName);
-        }
-        return false;
+        return another instanceof SingleUserPrincipal && _userName.equals(((SingleUserPrincipal) another)._userName);
     }
 
     @Override
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java b/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java
index f547868..9729671 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/TBackoffConnect.java
@@ -19,7 +19,6 @@
 package backtype.storm.security.auth;
 
 import java.io.IOException;
-import java.util.Random;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java
index 9f77ab9..8b3d4c5 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftClient.java
@@ -24,14 +24,10 @@
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransport;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import backtype.storm.utils.Utils;
 import backtype.storm.Config;
-import backtype.storm.security.auth.TBackoffConnect;
 
-public class ThriftClient {	
-    private static final Logger LOG = LoggerFactory.getLogger(ThriftClient.class);
+public class ThriftClient {
     private TTransport _transport;
     protected TProtocol _protocol;
     private String _host;
@@ -90,8 +86,6 @@
             //construct a transport plugin
             ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(_type, _conf, login_conf);
 
-            final TTransport underlyingTransport = socket;
-
             //TODO get this from type instead of hardcoding to Nimbus.
             //establish client-server transport via plugin
             //do retries if the connect fails
@@ -100,7 +94,7 @@
                                       Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_TIMES)),
                                       Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL)),
                                       Utils.getInt(_conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING)));
-            _transport = connectionRetry.doConnectWithRetry(transportPlugin, underlyingTransport, _host, _asUser);
+            _transport = connectionRetry.doConnectWithRetry(transportPlugin, socket, _host, _asUser);
         } catch (IOException ex) {
             throw new RuntimeException(ex);
         }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java b/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java
index 64243ce..fdbdc7c 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/ThriftServer.java
@@ -53,12 +53,10 @@
     }
 
     /**
-     * Is ThriftServer listening to requests?
-     * @return
+     * @return true if ThriftServer is listening to requests?
      */
     public boolean isServing() {
-        if (_server == null) return false;
-        return _server.isServing();
+        return _server != null && _server.isServing();
     }
     
     public void serve()  {
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java
index e1bb077..04269dd 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java
@@ -26,7 +26,7 @@
 import org.slf4j.LoggerFactory;
 
 public abstract class DRPCAuthorizerBase implements IAuthorizer {
-    public static Logger LOG = LoggerFactory.getLogger(DRPCAuthorizerBase.class);
+    public static final Logger LOG = LoggerFactory.getLogger(DRPCAuthorizerBase.class);
 
     /**
      * A key name for the function requested to be executed by a user.
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
index 8ce26ed..4a9b379 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java
@@ -28,7 +28,6 @@
 
 import backtype.storm.Config;
 import backtype.storm.security.auth.ReqContext;
-import backtype.storm.security.auth.authorizer.DRPCAuthorizerBase;
 import backtype.storm.security.auth.AuthUtils;
 import backtype.storm.security.auth.IPrincipalToLocal;
 import backtype.storm.utils.Utils;
@@ -37,7 +36,7 @@
 import org.slf4j.LoggerFactory;
 
 public class DRPCSimpleACLAuthorizer extends DRPCAuthorizerBase {
-    public static Logger LOG =
+    public static final Logger LOG =
         LoggerFactory.getLogger(DRPCSimpleACLAuthorizer.class);
 
     public static final String CLIENT_USERS_KEY = "client.users";
@@ -54,7 +53,7 @@
         public AclFunctionEntry(Collection<String> clientUsers,
                 String invocationUser) {
             this.clientUsers = (clientUsers != null) ?
-                new HashSet<String>(clientUsers) : new HashSet<String>();
+                new HashSet<>(clientUsers) : new HashSet<String>();
             this.invocationUser = invocationUser;
         }
     }
@@ -68,27 +67,27 @@
         //change is atomic
         long now = System.currentTimeMillis();
         if ((now - 5000) > _lastUpdate || _acl == null) {
-            Map<String,AclFunctionEntry> acl = new HashMap<String,AclFunctionEntry>();
+            Map<String,AclFunctionEntry> acl = new HashMap<>();
             Map conf = Utils.findAndReadConfigFile(_aclFileName);
             if (conf.containsKey(Config.DRPC_AUTHORIZER_ACL)) {
                 Map<String,Map<String,?>> confAcl =
                     (Map<String,Map<String,?>>)
                     conf.get(Config.DRPC_AUTHORIZER_ACL);
 
-                for (String function : confAcl.keySet()) {
-                    Map<String,?> val = confAcl.get(function);
+                for (Map.Entry<String, Map<String, ?>> entry : confAcl.entrySet()) {
+                    Map<String,?> val = entry.getValue();
                     Collection<String> clientUsers =
                         val.containsKey(CLIENT_USERS_KEY) ?
                         (Collection<String>) val.get(CLIENT_USERS_KEY) : null;
                     String invocationUser =
                         val.containsKey(INVOCATION_USER_KEY) ?
                         (String) val.get(INVOCATION_USER_KEY) : null;
-                    acl.put(function,
+                    acl.put(entry.getKey(),
                             new AclFunctionEntry(clientUsers, invocationUser));
                 }
             } else if (!_permitWhenMissingFunctionEntry) {
                 LOG.warn("Requiring explicit ACL entries, but none given. " +
-                        "Therefore, all operiations will be denied.");
+                        "Therefore, all operations will be denied.");
             }
             _acl = acl;
             _lastUpdate = System.currentTimeMillis();
@@ -100,8 +99,8 @@
     public void prepare(Map conf) {
         Boolean isStrict = 
                 (Boolean) conf.get(Config.DRPC_AUTHORIZER_ACL_STRICT);
-        _permitWhenMissingFunctionEntry = 
-                (isStrict != null && !isStrict) ? true : false;
+        _permitWhenMissingFunctionEntry =
+                (isStrict != null && !isStrict);
         _aclFileName = (String) conf.get(Config.DRPC_AUTHORIZER_ACL_FILENAME);
         _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf);
     }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java
index 5e84b38..d1d6f87 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/DenyAuthorizer.java
@@ -19,19 +19,14 @@
 
 import java.util.Map;
 
-import backtype.storm.Config;
 import backtype.storm.security.auth.IAuthorizer;
 import backtype.storm.security.auth.ReqContext;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * An authorization implementation that denies everything, for testing purposes
  */
 public class DenyAuthorizer implements IAuthorizer {
-    private static final Logger LOG = LoggerFactory.getLogger(DenyAuthorizer.class);
-    
+
     /**
      * Invoked once immediately after construction
      * @param conf Storm configuration 
@@ -41,17 +36,12 @@
 
     /**
      * permit() method is invoked for each incoming Thrift request
-     * @param contrext request context 
+     * @param context request context
      * @param operation operation name
-     * @param topology_storm configuration of targeted topology 
+     * @param topology_conf configuration of targeted topology
      * @return true if the request is authorized, false if reject
      */
     public boolean permit(ReqContext context, String operation, Map topology_conf) {
-        LOG.info("[req "+ context.requestID()+ "] Access "
-                + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString())
-                + (context.principal() == null? "" : (" principal:"+ context.principal()))
-                +" op:"+operation
-                + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME))));
         return false;
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/ImpersonationAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/ImpersonationAuthorizer.java
index c1573a6..07e6447 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/ImpersonationAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/ImpersonationAuthorizer.java
@@ -38,14 +38,15 @@
 
     @Override
     public void prepare(Map conf) {
-        userImpersonationACL = new HashMap<String, ImpersonationACL>();
+        userImpersonationACL = new HashMap<>();
 
         Map<String, Map<String, List<String>>> userToHostAndGroup = (Map<String, Map<String, List<String>>>) conf.get(Config.NIMBUS_IMPERSONATION_ACL);
 
         if (userToHostAndGroup != null) {
-            for (String user : userToHostAndGroup.keySet()) {
-                Set<String> groups = ImmutableSet.copyOf(userToHostAndGroup.get(user).get("groups"));
-                Set<String> hosts = ImmutableSet.copyOf(userToHostAndGroup.get(user).get("hosts"));
+            for (Map.Entry<String, Map<String, List<String>>> entry : userToHostAndGroup.entrySet()) {
+                String user = entry.getKey();
+                Set<String> groups = ImmutableSet.copyOf(entry.getValue().get("groups"));
+                Set<String> hosts = ImmutableSet.copyOf(entry.getValue().get("hosts"));
                 userImpersonationACL.put(user, new ImpersonationACL(user, groups, hosts));
             }
         }
@@ -66,7 +67,7 @@
         String userBeingImpersonated = _ptol.toLocal(context.principal());
         InetAddress remoteAddress = context.remoteAddress();
 
-        LOG.info("user = {}, principal = {} is attmepting to impersonate user = {} for operation = {} from host = {}",
+        LOG.info("user = {}, principal = {} is attempting to impersonate user = {} for operation = {} from host = {}",
                 impersonatingUser, impersonatingPrincipal, userBeingImpersonated, operation, remoteAddress);
 
         /**
@@ -82,8 +83,8 @@
         ImpersonationACL principalACL = userImpersonationACL.get(impersonatingPrincipal);
         ImpersonationACL userACL = userImpersonationACL.get(impersonatingUser);
 
-        Set<String> authorizedHosts = new HashSet<String>();
-        Set<String> authorizedGroups = new HashSet<String>();
+        Set<String> authorizedHosts = new HashSet<>();
+        Set<String> authorizedGroups = new HashSet<>();
 
         if (principalACL != null) {
             authorizedHosts.addAll(principalACL.authorizedHosts);
@@ -126,7 +127,7 @@
             return true;
         }
 
-        Set<String> groups = null;
+        Set<String> groups;
         try {
             groups = _groupMappingProvider.getGroups(userBeingImpersonated);
         } catch (IOException e) {
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java
index 9af44d3..ab5bd4b 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/NoopAuthorizer.java
@@ -19,18 +19,13 @@
 
 import java.util.Map;
 
-import backtype.storm.Config;
 import backtype.storm.security.auth.IAuthorizer;
 import backtype.storm.security.auth.ReqContext;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * A no-op authorization implementation that illustrate info available for authorization decisions.
  */
 public class NoopAuthorizer implements IAuthorizer {
-    private static final Logger LOG = LoggerFactory.getLogger(NoopAuthorizer.class);
 
     /**
      * Invoked once immediately after construction
@@ -43,15 +38,10 @@
      * permit() method is invoked for each incoming Thrift request
      * @param context request context includes info about 
      * @param operation operation name
-     * @param topology_storm configuration of targeted topology 
+     * @param topology_conf configuration of targeted topology
      * @return true if the request is authorized, false if reject
      */
     public boolean permit(ReqContext context, String operation, Map topology_conf) {
-        LOG.info("[req "+ context.requestID()+ "] Access "
-                + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString())
-                + (context.principal() == null? "" : (" principal:"+ context.principal()))
-                +" op:"+operation
-                + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME))));
         return true;
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java
index e50a587..0063f92 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java
@@ -42,9 +42,30 @@
 public class SimpleACLAuthorizer implements IAuthorizer {
     private static final Logger LOG = LoggerFactory.getLogger(SimpleACLAuthorizer.class);
 
-    protected Set<String> _userCommands = new HashSet<String>(Arrays.asList("submitTopology", "fileUpload", "getNimbusConf", "getClusterInfo"));
-    protected Set<String> _supervisorCommands = new HashSet<String>(Arrays.asList("fileDownload"));
-    protected Set<String> _topoCommands = new HashSet<String>(Arrays.asList("killTopology","rebalance","activate","deactivate","getTopologyConf","getTopology","getUserTopology","getTopologyInfo","uploadNewCredentials"));
+    protected Set<String> _userCommands = new HashSet<>(Arrays.asList("submitTopology", "fileUpload", "getNimbusConf", "getClusterInfo"));
+    protected Set<String> _supervisorCommands = new HashSet<>(Arrays.asList("fileDownload"));
+    protected Set<String> _topoCommands = new HashSet<>(Arrays.asList(
+            "killTopology",
+            "rebalance",
+            "activate",
+            "deactivate",
+            "getTopologyConf",
+            "getTopology",
+            "getUserTopology",
+            "getTopologyInfo",
+            "getTopologyPageInfo",
+            "getComponentPageInfo",
+            "uploadNewCredentials",
+            "setLogConfig",
+            "setWorkerProfiler",
+            "getWorkerProfileActionExpiry",
+            "getComponentPendingProfileActions",
+            "startProfiling",
+            "stopProfiling",
+            "dumpProfile",
+            "dumpJstack",
+            "dumpHeap",
+            "getLogConfig"));
 
     protected Set<String> _admins;
     protected Set<String> _supervisors;
@@ -58,10 +79,10 @@
      */
     @Override
     public void prepare(Map conf) {
-        _admins = new HashSet<String>();
-        _supervisors = new HashSet<String>();
-        _nimbusUsers = new HashSet<String>();
-        _nimbusGroups = new HashSet<String>();
+        _admins = new HashSet<>();
+        _supervisors = new HashSet<>();
+        _nimbusUsers = new HashSet<>();
+        _nimbusGroups = new HashSet<>();
 
         if (conf.containsKey(Config.NIMBUS_ADMINS)) {
             _admins.addAll((Collection<String>)conf.get(Config.NIMBUS_ADMINS));
@@ -90,15 +111,9 @@
      */
     @Override
     public boolean permit(ReqContext context, String operation, Map topology_conf) {
-        LOG.info("[req " + context.requestID() + "] Access "
-                + " from: " + (context.remoteAddress() == null ? "null" : context.remoteAddress().toString())
-                + (context.principal() == null ? "" : (" principal:" + context.principal()))
-                + " op:" + operation
-                + (topology_conf == null ? "" : (" topoology:" + topology_conf.get(Config.TOPOLOGY_NAME))));
-
         String principal = context.principal().getName();
         String user = _ptol.toLocal(context.principal());
-        Set<String> userGroups = new HashSet<String>();
+        Set<String> userGroups = new HashSet<>();
 
         if (_groupMappingProvider != null) {
             try {
@@ -130,7 +145,7 @@
                 return true;
             }
 
-            Set<String> topoGroups = new HashSet<String>();
+            Set<String> topoGroups = new HashSet<>();
             if (topology_conf.containsKey(Config.TOPOLOGY_GROUPS) && topology_conf.get(Config.TOPOLOGY_GROUPS) != null) {
                 topoGroups.addAll((Collection<String>)topology_conf.get(Config.TOPOLOGY_GROUPS));
             }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
index 55109f9..5731f06 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java
@@ -23,20 +23,15 @@
 import java.util.HashSet;
 import java.util.Collection;
 
-import backtype.storm.Config;
 import backtype.storm.security.auth.IAuthorizer;
 import backtype.storm.security.auth.ReqContext;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /**
  * An authorization implementation that simply checks a whitelist of users that
  * are allowed to use the cluster.
  */
 public class SimpleWhitelistAuthorizer implements IAuthorizer {
-    private static final Logger LOG = LoggerFactory.getLogger(SimpleWhitelistAuthorizer.class);
-    public static String WHITELIST_USERS_CONF = "storm.auth.simple-white-list.users";
+    public static final String WHITELIST_USERS_CONF = "storm.auth.simple-white-list.users";
     protected Set<String> users;
 
     /**
@@ -45,7 +40,7 @@
      */
     @Override
     public void prepare(Map conf) {
-        users = new HashSet<String>();
+        users = new HashSet<>();
         if (conf.containsKey(WHITELIST_USERS_CONF)) {
             users.addAll((Collection<String>)conf.get(WHITELIST_USERS_CONF));
         }
@@ -55,16 +50,11 @@
      * permit() method is invoked for each incoming Thrift request
      * @param context request context includes info about 
      * @param operation operation name
-     * @param topology_storm configuration of targeted topology 
+     * @param topology_conf configuration of targeted topology
      * @return true if the request is authorized, false if reject
      */
     @Override
     public boolean permit(ReqContext context, String operation, Map topology_conf) {
-        LOG.info("[req "+ context.requestID()+ "] Access "
-                 + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString())
-                 + (context.principal() == null? "" : (" principal:"+ context.principal()))
-                 +" op:"+operation
-                 + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME))));
         return context.principal() != null ? users.contains(context.principal().getName()) : false;
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/digest/ClientCallbackHandler.java b/storm-core/src/jvm/backtype/storm/security/auth/digest/ClientCallbackHandler.java
index 3caacaa..420326c 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/digest/ClientCallbackHandler.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/digest/ClientCallbackHandler.java
@@ -46,8 +46,6 @@
      * Constructor based on a JAAS configuration
      * 
      * For digest, you should have a pair of user name and password defined.
-     * 
-     * @param configuration
      * @throws IOException
      */
     public ClientCallbackHandler(Configuration configuration) throws IOException {
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java b/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java
index ad642d8..09d6f78 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java
@@ -18,10 +18,8 @@
 package backtype.storm.security.auth.digest;
 
 import java.io.IOException;
-import java.util.Map;
 
 import javax.security.auth.callback.CallbackHandler;
-import javax.security.auth.login.Configuration;
 
 import org.apache.thrift.transport.TSaslClientTransport;
 import org.apache.thrift.transport.TSaslServerTransport;
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/digest/ServerCallbackHandler.java b/storm-core/src/jvm/backtype/storm/security/auth/digest/ServerCallbackHandler.java
index 1788dab..e80072c 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/digest/ServerCallbackHandler.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/digest/ServerCallbackHandler.java
@@ -26,7 +26,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.security.auth.Subject;
 import javax.security.auth.callback.Callback;
 import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.callback.NameCallback;
@@ -40,7 +39,7 @@
 import backtype.storm.security.auth.AuthUtils;
 
 /**
- * SASL server side collback handler
+ * SASL server side callback handler
  */
 public class ServerCallbackHandler implements CallbackHandler {
     private static final String USER_PREFIX = "user_";
@@ -48,7 +47,7 @@
     private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword";
 
     private String userName;
-    private final Map<String,String> credentials = new HashMap<String,String>();
+    private final Map<String,String> credentials = new HashMap<>();
 
     public ServerCallbackHandler(Configuration configuration) throws IOException {
         if (configuration==null) return;
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
index aed1c4f..dd54e02 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/AutoTGT.java
@@ -22,7 +22,6 @@
 import backtype.storm.security.auth.ICredentialsRenewer;
 import backtype.storm.security.auth.AuthUtils;
 
-import java.util.Date;
 import java.util.Map;
 import java.util.Set;
 import java.io.ByteArrayInputStream;
@@ -38,7 +37,6 @@
 import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.auth.kerberos.KerberosPrincipal;
 import javax.security.auth.login.Configuration;
-import javax.security.auth.login.LoginException;
 import javax.security.auth.login.LoginContext;
 import javax.security.auth.DestroyFailedException;
 import javax.security.auth.RefreshFailedException;
@@ -54,7 +52,7 @@
 public class AutoTGT implements IAutoCredentials, ICredentialsRenewer {
     private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class);
     private static final float TICKET_RENEW_WINDOW = 0.80f;
-    protected static AtomicReference<KerberosTicket> kerbTicket = new AtomicReference<KerberosTicket>();
+    protected static final AtomicReference<KerberosTicket> kerbTicket = new AtomicReference<>();
     private Map conf;
 
     public void prepare(Map conf) {
@@ -66,11 +64,9 @@
         for(KerberosTicket ticket: tickets) {
             KerberosPrincipal server = ticket.getServer();
             if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) {
-                tickets = null;
                 return ticket;
             }
         }
-        tickets = null;
         return null;
     } 
 
@@ -165,7 +161,7 @@
                         try {
                             t.destroy();
                         } catch (DestroyFailedException  e) {
-                            LOG.warn("Failed to destory ticket ", e);
+                            LOG.warn("Failed to destroy ticket ", e);
                         }
                     }
                 }
@@ -184,7 +180,7 @@
      * @param subject the subject that should have a TGT in it.
      */
     private void loginHadoopUser(Subject subject) {
-        Class<?> ugi = null;
+        Class<?> ugi;
         try {
             ugi = Class.forName("org.apache.hadoop.security.UserGroupInformation");
         } catch (ClassNotFoundException e) {
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/NoOpTTrasport.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/NoOpTTrasport.java
index 074930d..c0721fc 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/NoOpTTrasport.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/NoOpTTrasport.java
@@ -1,12 +1,26 @@
+/**
+ * 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.security.auth.kerberos;
 
 import org.apache.thrift.transport.TSaslServerTransport;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 
-/**
- * Created by pshah on 8/12/15.
- */
 public class NoOpTTrasport extends TSaslServerTransport {
 
     public NoOpTTrasport(TTransport transport) {
diff --git a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java
index 7b143f0..46ffa61 100644
--- a/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java
+++ b/storm-core/src/jvm/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java
@@ -87,6 +87,8 @@
         //add the authNid as the real user in reqContext's subject which will be used during authorization.
         if(!ac.getAuthenticationID().equals(ac.getAuthorizationID())) {
             ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID()));
+        } else {
+            ReqContext.context().setRealPrincipal(null);
         }
 
         ac.setAuthorized(true);
diff --git a/storm-core/src/jvm/backtype/storm/security/serialization/BlowfishTupleSerializer.java b/storm-core/src/jvm/backtype/storm/security/serialization/BlowfishTupleSerializer.java
index 8e66cdf..fca0c46 100644
--- a/storm-core/src/jvm/backtype/storm/security/serialization/BlowfishTupleSerializer.java
+++ b/storm-core/src/jvm/backtype/storm/security/serialization/BlowfishTupleSerializer.java
@@ -36,19 +36,19 @@
 import backtype.storm.Config;
 
 /**
- * Apply Blowfish encrption for tuple communication to bolts
+ * Apply Blowfish encryption for tuple communication to bolts
  */
 public class BlowfishTupleSerializer extends Serializer<ListDelegate> {
     /**
      * The secret key (if any) for data encryption by blowfish payload serialization factory (BlowfishSerializationFactory). 
      * You should use in via "storm -c topology.tuple.serializer.blowfish.key=YOURKEY -c topology.tuple.serializer=backtype.storm.security.serialization.BlowfishTupleSerializer jar ...".
      */
-    public static String SECRET_KEY = "topology.tuple.serializer.blowfish.key";
+    public static final String SECRET_KEY = "topology.tuple.serializer.blowfish.key";
     private static final Logger LOG = LoggerFactory.getLogger(BlowfishTupleSerializer.class);
     private BlowfishSerializer _serializer;
 
     public BlowfishTupleSerializer(Kryo kryo, Map storm_conf) {
-        String encryption_key = null;
+        String encryption_key;
         try {
             encryption_key = (String)storm_conf.get(SECRET_KEY);
             LOG.debug("Blowfish serializer being constructed ...");
diff --git a/storm-core/src/jvm/backtype/storm/serialization/GzipThriftSerializationDelegate.java b/storm-core/src/jvm/backtype/storm/serialization/GzipThriftSerializationDelegate.java
index 933a125..54193da 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/GzipThriftSerializationDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/GzipThriftSerializationDelegate.java
@@ -17,7 +17,6 @@
  */
 package backtype.storm.serialization;
 
-import java.io.IOException;
 import java.util.Map;
 import backtype.storm.utils.Utils;
 import org.apache.thrift.TBase;
diff --git a/storm-core/src/jvm/backtype/storm/serialization/ITupleDeserializer.java b/storm-core/src/jvm/backtype/storm/serialization/ITupleDeserializer.java
index 4e68658..09cc422 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/ITupleDeserializer.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/ITupleDeserializer.java
@@ -18,7 +18,6 @@
 package backtype.storm.serialization;
 
 import backtype.storm.tuple.Tuple;
-import java.io.IOException;
 
 public interface ITupleDeserializer {
     Tuple deserialize(byte[] ser);        
diff --git a/storm-core/src/jvm/backtype/storm/serialization/KryoTupleDeserializer.java b/storm-core/src/jvm/backtype/storm/serialization/KryoTupleDeserializer.java
index 5a5e3a4..07b9cd5 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/KryoTupleDeserializer.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/KryoTupleDeserializer.java
@@ -21,10 +21,7 @@
 import backtype.storm.tuple.MessageId;
 import backtype.storm.tuple.Tuple;
 import backtype.storm.tuple.TupleImpl;
-import backtype.storm.utils.WritableUtils;
 import com.esotericsoftware.kryo.io.Input;
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
diff --git a/storm-core/src/jvm/backtype/storm/serialization/KryoValuesDeserializer.java b/storm-core/src/jvm/backtype/storm/serialization/KryoValuesDeserializer.java
index 209ae53..418068e 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/KryoValuesDeserializer.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/KryoValuesDeserializer.java
@@ -21,7 +21,6 @@
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.io.Input;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -35,7 +34,7 @@
     }
     
     public List<Object> deserializeFrom(Input input) {
-    	ListDelegate delegate = (ListDelegate) _kryo.readObject(input, ListDelegate.class);
+    	ListDelegate delegate = _kryo.readObject(input, ListDelegate.class);
    	return delegate.getDelegate();
     }
     
diff --git a/storm-core/src/jvm/backtype/storm/serialization/SerializationFactory.java b/storm-core/src/jvm/backtype/storm/serialization/SerializationFactory.java
index 5549b1d..417f102 100644
--- a/storm-core/src/jvm/backtype/storm/serialization/SerializationFactory.java
+++ b/storm-core/src/jvm/backtype/storm/serialization/SerializationFactory.java
@@ -21,7 +21,6 @@
 import backtype.storm.generated.ComponentCommon;
 import backtype.storm.generated.StormTopology;
 import backtype.storm.serialization.types.ArrayListSerializer;
-import backtype.storm.serialization.types.ListDelegateSerializer;
 import backtype.storm.serialization.types.HashMapSerializer;
 import backtype.storm.serialization.types.HashSetSerializer;
 import backtype.storm.transactional.TransactionAttempt;
@@ -81,10 +80,10 @@
         kryoFactory.preRegister(k, conf);
 
         boolean skipMissing = (Boolean) conf.get(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS);
-        for(String klassName: registrations.keySet()) {
-            String serializerClassName = registrations.get(klassName);
+        for(Map.Entry<String, String> entry: registrations.entrySet()) {
+            String serializerClassName = entry.getValue();
             try {
-                Class klass = Class.forName(klassName);
+                Class klass = Class.forName(entry.getKey());
                 Class serializerClass = null;
                 if(serializerClassName!=null)
                     serializerClass = Class.forName(serializerClassName);
@@ -130,17 +129,17 @@
     }
 
     public static class IdDictionary {
-        Map<String, Map<String, Integer>> streamNametoId = new HashMap<String, Map<String, Integer>>();
-        Map<String, Map<Integer, String>> streamIdToName = new HashMap<String, Map<Integer, String>>();
+        Map<String, Map<String, Integer>> streamNametoId = new HashMap<>();
+        Map<String, Map<Integer, String>> streamIdToName = new HashMap<>();
 
         public IdDictionary(StormTopology topology) {
-            List<String> componentNames = new ArrayList<String>(topology.get_spouts().keySet());
+            List<String> componentNames = new ArrayList<>(topology.get_spouts().keySet());
             componentNames.addAll(topology.get_bolts().keySet());
             componentNames.addAll(topology.get_state_spouts().keySet());
 
             for(String name: componentNames) {
                 ComponentCommon common = Utils.getComponentCommon(topology, name);
-                List<String> streams = new ArrayList<String>(common.get_streams().keySet());
+                List<String> streams = new ArrayList<>(common.get_streams().keySet());
                 streamNametoId.put(name, idify(streams));
                 streamIdToName.put(name, Utils.reverseMap(streamNametoId.get(name)));
             }
@@ -156,7 +155,7 @@
 
         private static Map<String, Integer> idify(List<String> names) {
             Collections.sort(names);
-            Map<String, Integer> ret = new HashMap<String, Integer>();
+            Map<String, Integer> ret = new HashMap<>();
             int i = 1;
             for(String name: names) {
                 ret.put(name, i);
@@ -204,8 +203,8 @@
     private static Map<String, String> normalizeKryoRegister(Map conf) {
         // TODO: de-duplicate this logic with the code in nimbus
         Object res = conf.get(Config.TOPOLOGY_KRYO_REGISTER);
-        if(res==null) return new TreeMap<String, String>();
-        Map<String, String> ret = new HashMap<String, String>();
+        if(res==null) return new TreeMap<>();
+        Map<String, String> ret = new HashMap<>();
         if(res instanceof Map) {
             ret = (Map<String, String>) res;
         } else {
@@ -219,6 +218,6 @@
         }
 
         //ensure always same order for registrations with TreeMap
-        return new TreeMap<String, String>(ret);
+        return new TreeMap<>(ret);
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/spout/MultiScheme.java b/storm-core/src/jvm/backtype/storm/spout/MultiScheme.java
index ca2ce91..57bf4ce 100644
--- a/storm-core/src/jvm/backtype/storm/spout/MultiScheme.java
+++ b/storm-core/src/jvm/backtype/storm/spout/MultiScheme.java
@@ -17,12 +17,13 @@
  */
 package backtype.storm.spout;
 
+import java.nio.ByteBuffer;
 import java.util.List;
 import java.io.Serializable;
 
 import backtype.storm.tuple.Fields;
 
 public interface MultiScheme extends Serializable {
-  public Iterable<List<Object>> deserialize(byte[] ser);
+  public Iterable<List<Object>> deserialize(ByteBuffer ser);
   public Fields getOutputFields();
 }
diff --git a/storm-core/src/jvm/backtype/storm/spout/RawMultiScheme.java b/storm-core/src/jvm/backtype/storm/spout/RawMultiScheme.java
index 7f73975..824d16c 100644
--- a/storm-core/src/jvm/backtype/storm/spout/RawMultiScheme.java
+++ b/storm-core/src/jvm/backtype/storm/spout/RawMultiScheme.java
@@ -17,6 +17,7 @@
  */
 package backtype.storm.spout;
 
+import java.nio.ByteBuffer;
 import java.util.List;
 
 import backtype.storm.tuple.Fields;
@@ -27,7 +28,7 @@
 
 public class RawMultiScheme implements MultiScheme {
   @Override
-  public Iterable<List<Object>> deserialize(byte[] ser) {
+  public Iterable<List<Object>> deserialize(ByteBuffer ser) {
     return asList(tuple(ser));
   }
 
diff --git a/storm-core/src/jvm/backtype/storm/spout/RawScheme.java b/storm-core/src/jvm/backtype/storm/spout/RawScheme.java
index 7e26770..937acb7 100644
--- a/storm-core/src/jvm/backtype/storm/spout/RawScheme.java
+++ b/storm-core/src/jvm/backtype/storm/spout/RawScheme.java
@@ -18,12 +18,17 @@
 package backtype.storm.spout;
 
 import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+import java.nio.ByteBuffer;
 import java.util.List;
 import static backtype.storm.utils.Utils.tuple;
 
 public class RawScheme implements Scheme {
-    public List<Object> deserialize(byte[] ser) {
-        return tuple(ser);
+    public List<Object> deserialize(ByteBuffer ser) {
+        // Maintain backward compatibility for 0.10
+        byte[] b = Utils.toByteArray(ser);
+        return tuple(new Object[]{b});
     }
 
     public Fields getOutputFields() {
diff --git a/storm-core/src/jvm/backtype/storm/spout/Scheme.java b/storm-core/src/jvm/backtype/storm/spout/Scheme.java
index ca68954..d696a9c 100644
--- a/storm-core/src/jvm/backtype/storm/spout/Scheme.java
+++ b/storm-core/src/jvm/backtype/storm/spout/Scheme.java
@@ -19,10 +19,11 @@
 
 import backtype.storm.tuple.Fields;
 import java.io.Serializable;
+import java.nio.ByteBuffer;
 import java.util.List;
 
 
 public interface Scheme extends Serializable {
-    public List<Object> deserialize(byte[] ser);
+    List<Object> deserialize(ByteBuffer ser);
     public Fields getOutputFields();
 }
diff --git a/storm-core/src/jvm/backtype/storm/spout/SchemeAsMultiScheme.java b/storm-core/src/jvm/backtype/storm/spout/SchemeAsMultiScheme.java
index 29f7fce..a49d55f 100644
--- a/storm-core/src/jvm/backtype/storm/spout/SchemeAsMultiScheme.java
+++ b/storm-core/src/jvm/backtype/storm/spout/SchemeAsMultiScheme.java
@@ -17,6 +17,7 @@
  */
 package backtype.storm.spout;
 
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 
@@ -29,7 +30,7 @@
     this.scheme = scheme;
   }
 
-  @Override public Iterable<List<Object>> deserialize(final byte[] ser) {
+  @Override public Iterable<List<Object>> deserialize(final ByteBuffer ser) {
     List<Object> o = scheme.deserialize(ser);
     if(o == null) return null;
     else return Arrays.asList(o);
diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
index a69b9b2..bfdfe67 100644
--- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
+++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -41,11 +41,11 @@
 
 
 public class ShellSpout implements ISpout {
-    public static Logger LOG = LoggerFactory.getLogger(ShellSpout.class);
+    public static final Logger LOG = LoggerFactory.getLogger(ShellSpout.class);
 
     private SpoutOutputCollector _collector;
     private String[] _command;
-    private Map<String, String> env = new HashMap<String, String>();
+    private Map<String, String> env = new HashMap<>();
     private ShellProcess _process;
     
     private TopologyContext _context;
@@ -74,7 +74,11 @@
         _collector = collector;
         _context = context;
 
-        workerTimeoutMills = 1000 * RT.intCast(stormConf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS));
+        if (stormConf.containsKey(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS)) {
+            workerTimeoutMills = 1000 * RT.intCast(stormConf.get(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS));
+        } else {
+            workerTimeoutMills = 1000 * RT.intCast(stormConf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS));
+        }
 
         _process = new ShellProcess(_command);
         if (!env.isEmpty()) {
diff --git a/storm-core/src/jvm/backtype/storm/task/GeneralTopologyContext.java b/storm-core/src/jvm/backtype/storm/task/GeneralTopologyContext.java
index 53141b1..11bdc19 100644
--- a/storm-core/src/jvm/backtype/storm/task/GeneralTopologyContext.java
+++ b/storm-core/src/jvm/backtype/storm/task/GeneralTopologyContext.java
@@ -100,8 +100,8 @@
      */
     public List<Integer> getComponentTasks(String componentId) {
         List<Integer> ret = _componentToTasks.get(componentId);
-        if(ret==null) return new ArrayList<Integer>();
-        else return new ArrayList<Integer>(ret);
+        if(ret==null) return new ArrayList<>();
+        else return new ArrayList<>(ret);
     }
 
     /**
@@ -138,14 +138,15 @@
      * @return Map from stream id to component id to the Grouping used.
      */
     public Map<String, Map<String, Grouping>> getTargets(String componentId) {
-        Map<String, Map<String, Grouping>> ret = new HashMap<String, Map<String, Grouping>>();
+        Map<String, Map<String, Grouping>> ret = new HashMap<>();
         for(String otherComponentId: getComponentIds()) {
             Map<GlobalStreamId, Grouping> inputs = getComponentCommon(otherComponentId).get_inputs();
-            for(GlobalStreamId id: inputs.keySet()) {
+            for(Map.Entry<GlobalStreamId, Grouping> entry: inputs.entrySet()) {
+                GlobalStreamId id = entry.getKey();
                 if(id.get_componentId().equals(componentId)) {
                     Map<String, Grouping> curr = ret.get(id.get_streamId());
-                    if(curr==null) curr = new HashMap<String, Grouping>();
-                    curr.put(otherComponentId, inputs.get(id));
+                    if(curr==null) curr = new HashMap<>();
+                    curr.put(otherComponentId, entry.getValue());
                     ret.put(id.get_streamId(), curr);
                 }
             }
@@ -195,4 +196,4 @@
         }
         return max;
     }
-}
\ No newline at end of file
+}
diff --git a/storm-core/src/jvm/backtype/storm/task/OutputCollector.java b/storm-core/src/jvm/backtype/storm/task/OutputCollector.java
index 620d33d..0214eaa 100644
--- a/storm-core/src/jvm/backtype/storm/task/OutputCollector.java
+++ b/storm-core/src/jvm/backtype/storm/task/OutputCollector.java
@@ -115,7 +115,7 @@
      * @param tuple the new output tuple from this bolt
      */
     public void emitDirect(int taskId, String streamId, Tuple anchor, List<Object> tuple) {
-        emitDirect(taskId, streamId, Arrays.asList(anchor), tuple);
+        emitDirect(taskId, streamId, anchor == null ? (List) null : Arrays.asList(anchor), tuple);
     }
 
     /**
diff --git a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
index 44b8e48..0103715 100644
--- a/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
+++ b/storm-core/src/jvm/backtype/storm/task/ShellBolt.java
@@ -68,13 +68,12 @@
  */
 public class ShellBolt implements IBolt {
     public static final String HEARTBEAT_STREAM_ID = "__heartbeat";
-    public static Logger LOG = LoggerFactory.getLogger(ShellBolt.class);
-    Process _subprocess;
+    public static final Logger LOG = LoggerFactory.getLogger(ShellBolt.class);
     OutputCollector _collector;
-    Map<String, Tuple> _inputs = new ConcurrentHashMap<String, Tuple>();
+    Map<String, Tuple> _inputs = new ConcurrentHashMap<>();
 
     private String[] _command;
-    private Map<String, String> env = new HashMap<String, String>();
+    private Map<String, String> env = new HashMap<>();
     private ShellProcess _process;
     private volatile boolean _running = true;
     private volatile Throwable _exception;
@@ -115,7 +114,11 @@
 
         _context = context;
 
-        workerTimeoutMills = 1000 * RT.intCast(stormConf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS));
+        if (stormConf.containsKey(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS)) {
+            workerTimeoutMills = 1000 * RT.intCast(stormConf.get(Config.TOPOLOGY_SUBPROCESS_TIMEOUT_SECS));
+        } else {
+            workerTimeoutMills = 1000 * RT.intCast(stormConf.get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS));
+        }
 
         _process = new ShellProcess(_command);
         if (!env.isEmpty()) {
@@ -198,7 +201,7 @@
     }
 
     private void handleEmit(ShellMsg shellMsg) throws InterruptedException {
-        List<Tuple> anchors = new ArrayList<Tuple>();
+        List<Tuple> anchors = new ArrayList<>();
         List<String> recvAnchors = shellMsg.getAnchors();
         if (recvAnchors != null) {
             for (String anchor : recvAnchors) {
@@ -337,18 +340,25 @@
                     setHeartbeat();
 
                     // We don't need to take care of sync, cause we're always updating heartbeat
-                    if(command.equals("ack")) {
-                        handleAck(shellMsg.getId());
-                    } else if (command.equals("fail")) {
-                        handleFail(shellMsg.getId());
-                    } else if (command.equals("error")) {
-                        handleError(shellMsg.getMsg());
-                    } else if (command.equals("log")) {
-                        handleLog(shellMsg);
-                    } else if (command.equals("emit")) {
-                        handleEmit(shellMsg);
-                    } else if (command.equals("metrics")) {
-                        handleMetrics(shellMsg);
+                    switch (command) {
+                        case "ack":
+                            handleAck(shellMsg.getId());
+                            break;
+                        case "fail":
+                            handleFail(shellMsg.getId());
+                            break;
+                        case "error":
+                            handleError(shellMsg.getMsg());
+                            break;
+                        case "log":
+                            handleLog(shellMsg);
+                            break;
+                        case "emit":
+                            handleEmit(shellMsg);
+                            break;
+                        case "metrics":
+                            handleMetrics(shellMsg);
+                            break;
                     }
                 } catch (InterruptedException e) {
                 } catch (Throwable t) {
@@ -390,7 +400,7 @@
             msg.setId(genId);
             msg.setTask(Constants.SYSTEM_TASK_ID);
             msg.setStream(HEARTBEAT_STREAM_ID);
-            msg.setTuple(new ArrayList<Object>());
+            msg.setTuple(new ArrayList<>());
             return msg;
         }
     }
diff --git a/storm-core/src/jvm/backtype/storm/task/TopologyContext.java b/storm-core/src/jvm/backtype/storm/task/TopologyContext.java
index cefa207..e9cdb5b 100644
--- a/storm-core/src/jvm/backtype/storm/task/TopologyContext.java
+++ b/storm-core/src/jvm/backtype/storm/task/TopologyContext.java
@@ -51,8 +51,8 @@
  */
 public class TopologyContext extends WorkerTopologyContext implements IMetricsContext {
     private Integer _taskId;
-    private Map<String, Object> _taskData = new HashMap<String, Object>();
-    private List<ITaskHook> _hooks = new ArrayList<ITaskHook>();
+    private Map<String, Object> _taskData = new HashMap<>();
+    private List<ITaskHook> _hooks = new ArrayList<>();
     private Map<String, Object> _executorData;
     private Map<Integer,Map<Integer, Map<String, IMetric>>> _registeredMetrics;
     private clojure.lang.Atom _openOrPrepareWasCalled;
@@ -139,9 +139,8 @@
     }
 
     /**
-     * Gets the component id for this task. The component id maps
+     * @return the component id for this task. The component id maps
      * to a component id specified for a Spout or Bolt in the topology definition.
-     * @return
      */
     public String getThisComponentId() {
         return getComponentId(_taskId);
@@ -308,7 +307,7 @@
      * @return The IMetric argument unchanged.
      */
     public <T extends IMetric> T registerMetric(String name, T metric, int timeBucketSizeInSecs) {
-        if((Boolean)_openOrPrepareWasCalled.deref() == true) {
+        if((Boolean) _openOrPrepareWasCalled.deref()) {
             throw new RuntimeException("TopologyContext.registerMetric can only be called from within overridden " +
                                        "IBolt::prepare() or ISpout::open() method.");
         }
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..f885056
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/testing/AlternateRackDNSToSwitchMapping.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.testing;
+
+import backtype.storm.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
diff --git a/storm-core/src/jvm/backtype/storm/testing/MemoryTransactionalSpout.java b/storm-core/src/jvm/backtype/storm/testing/MemoryTransactionalSpout.java
index 3b492e1..faca77d 100644
--- a/storm-core/src/jvm/backtype/storm/testing/MemoryTransactionalSpout.java
+++ b/storm-core/src/jvm/backtype/storm/testing/MemoryTransactionalSpout.java
@@ -23,7 +23,6 @@
 import backtype.storm.transactional.TransactionAttempt;
 import backtype.storm.coordination.BatchOutputCollector;
 import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout;
-import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout.Emitter;
 import backtype.storm.tuple.Fields;
 import backtype.storm.utils.RegisteredGlobalState;
 import backtype.storm.utils.Utils;
@@ -34,7 +33,7 @@
 import java.util.Map;
 
 public class MemoryTransactionalSpout implements IPartitionedTransactionalSpout<MemoryTransactionalSpoutMeta> {
-    public static String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id";
+    public static final String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id";
     
     private String _id;
     private String _finishedPartitionsId;
@@ -81,7 +80,7 @@
     class Emitter implements IPartitionedTransactionalSpout.Emitter<MemoryTransactionalSpoutMeta> {
         
         Integer _maxSpoutPending;
-        Map<Integer, Integer> _emptyPartitions = new HashMap<Integer, Integer>();
+        Map<Integer, Integer> _emptyPartitions = new HashMap<>();
         
         public Emitter(Map conf) {
             Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING);
@@ -125,7 +124,7 @@
         public void emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, MemoryTransactionalSpoutMeta partitionMeta) {
             List<List<Object>> queue = getQueues().get(partition);
             for(int i=partitionMeta.index; i < partitionMeta.index + partitionMeta.amt; i++) {
-                List<Object> toEmit = new ArrayList<Object>(queue.get(i));
+                List<Object> toEmit = new ArrayList<>(queue.get(i));
                 toEmit.add(0, tx);
                 collector.emit(toEmit);                
             }
@@ -148,7 +147,7 @@
 
     @Override
     public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        List<String> toDeclare = new ArrayList<String>(_outFields.toList());
+        List<String> toDeclare = new ArrayList<>(_outFields.toList());
         toDeclare.add(0, TX_FIELD);
         declarer.declare(new Fields(toDeclare));
     }
diff --git a/storm-core/src/jvm/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java b/storm-core/src/jvm/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java
index 0151392..876bf51 100644
--- a/storm-core/src/jvm/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java
+++ b/storm-core/src/jvm/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java
@@ -36,7 +36,7 @@
  * This spout only works in local mode.
  */
 public class OpaqueMemoryTransactionalSpout implements IOpaquePartitionedTransactionalSpout<MemoryTransactionalSpoutMeta> {
-    public static String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id";
+    public static final String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id";
     
     private String _id;
     private String _finishedPartitionsId;
@@ -95,7 +95,7 @@
     class Emitter implements IOpaquePartitionedTransactionalSpout.Emitter<MemoryTransactionalSpoutMeta> {
         
         Integer _maxSpoutPending;
-        Map<Integer, Integer> _emptyPartitions = new HashMap<Integer, Integer>();
+        Map<Integer, Integer> _emptyPartitions = new HashMap<>();
         
         public Emitter(Map conf) {
             Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING);
@@ -120,7 +120,7 @@
 
                 MemoryTransactionalSpoutMeta ret = new MemoryTransactionalSpoutMeta(index, toTake);
                 for(int i=ret.index; i < ret.index + ret.amt; i++) {
-                    List<Object> toEmit = new ArrayList<Object>(queue.get(i));
+                    List<Object> toEmit = new ArrayList<>(queue.get(i));
                     toEmit.add(0, tx);
                     collector.emit(toEmit);                
                 }
@@ -151,7 +151,7 @@
 
     @Override
     public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        List<String> toDeclare = new ArrayList<String>(_outFields.toList());
+        List<String> toDeclare = new ArrayList<>(_outFields.toList());
         toDeclare.add(0, TX_FIELD);
         declarer.declare(new Fields(toDeclare));
     }
diff --git a/storm-core/src/jvm/backtype/storm/testing/TupleCaptureBolt.java b/storm-core/src/jvm/backtype/storm/testing/TupleCaptureBolt.java
index e163576..b085c54 100644
--- a/storm-core/src/jvm/backtype/storm/testing/TupleCaptureBolt.java
+++ b/storm-core/src/jvm/backtype/storm/testing/TupleCaptureBolt.java
@@ -30,7 +30,7 @@
 
 
 public class TupleCaptureBolt implements IRichBolt {
-    public static transient Map<String, Map<String, List<FixedTuple>>> emitted_tuples = new HashMap<String, Map<String, List<FixedTuple>>>();
+    public static final transient Map<String, Map<String, List<FixedTuple>>> emitted_tuples = new HashMap<>();
 
     private String _name;
     private OutputCollector _collector;
@@ -66,7 +66,7 @@
     }
 
     public Map<String, List<FixedTuple>> getAndClearResults() {
-        Map<String, List<FixedTuple>> ret = new HashMap<String, List<FixedTuple>>(emitted_tuples.get(_name));
+        Map<String, List<FixedTuple>> ret = new HashMap<>(emitted_tuples.get(_name));
         emitted_tuples.get(_name).clear();
         return ret;
     }
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 @@
 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 @@
     
     @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;
+    }
 }
diff --git a/storm-core/src/jvm/backtype/storm/topology/BasicBoltExecutor.java b/storm-core/src/jvm/backtype/storm/topology/BasicBoltExecutor.java
index 6c9cdc1..cf828fe 100644
--- a/storm-core/src/jvm/backtype/storm/topology/BasicBoltExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/topology/BasicBoltExecutor.java
@@ -25,7 +25,7 @@
 import org.slf4j.LoggerFactory;
 
 public class BasicBoltExecutor implements IRichBolt {
-    public static Logger LOG = LoggerFactory.getLogger(BasicBoltExecutor.class);    
+    public static final Logger LOG = LoggerFactory.getLogger(BasicBoltExecutor.class);
     
     private IBasicBolt _bolt;
     private transient BasicOutputCollector _collector;
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 @@
     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);
 }
diff --git a/storm-core/src/jvm/backtype/storm/topology/IWindowedBolt.java b/storm-core/src/jvm/backtype/storm/topology/IWindowedBolt.java
new file mode 100644
index 0000000..8363717
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/IWindowedBolt.java
@@ -0,0 +1,40 @@
+/**
+ * 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.topology;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.windowing.TupleWindow;
+
+import java.util.Map;
+
+/**
+ * A bolt abstraction for supporting time and count based sliding & tumbling windows.
+ */
+public interface IWindowedBolt extends IComponent {
+    /**
+     * This is similar to the {@link backtype.storm.task.IBolt#prepare(Map, TopologyContext, OutputCollector)} except
+     * that while emitting, the tuples are automatically anchored to the tuples in the inputWindow.
+     */
+    void prepare(Map stormConf, TopologyContext context, OutputCollector collector);
+    /**
+     * Process the tuple window and optionally emit new tuples based on the tuples in the input window.
+     */
+    void execute(TupleWindow inputWindow);
+    void cleanup();
+}
diff --git a/storm-core/src/jvm/backtype/storm/topology/OutputFieldsGetter.java b/storm-core/src/jvm/backtype/storm/topology/OutputFieldsGetter.java
index 0e7fd59..26a791e 100644
--- a/storm-core/src/jvm/backtype/storm/topology/OutputFieldsGetter.java
+++ b/storm-core/src/jvm/backtype/storm/topology/OutputFieldsGetter.java
@@ -24,7 +24,7 @@
 import java.util.Map;
 
 public class OutputFieldsGetter implements OutputFieldsDeclarer {
-    private Map<String, StreamInfo> _fields = new HashMap<String, StreamInfo>();
+    private Map<String, StreamInfo> _fields = new HashMap<>();
 
     public void declare(Fields fields) {
         declare(false, fields);
diff --git a/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java b/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
index 38b30d7..9d2ef61 100644
--- a/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
+++ b/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java
@@ -18,23 +18,20 @@
 package backtype.storm.topology;
 
 import backtype.storm.Config;
-import backtype.storm.generated.Bolt;
-import backtype.storm.generated.ComponentCommon;
-import backtype.storm.generated.ComponentObject;
-import backtype.storm.generated.GlobalStreamId;
-import backtype.storm.generated.Grouping;
-import backtype.storm.generated.NullStruct;
-import backtype.storm.generated.SpoutSpec;
-import backtype.storm.generated.StateSpoutSpec;
-import backtype.storm.generated.StormTopology;
+import backtype.storm.generated.*;
 import backtype.storm.grouping.CustomStreamGrouping;
 import backtype.storm.grouping.PartialKeyGrouping;
+import backtype.storm.hooks.IWorkerHook;
 import backtype.storm.tuple.Fields;
 import backtype.storm.utils.Utils;
+import org.json.simple.JSONValue;
+
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-import org.json.simple.JSONValue;
+import backtype.storm.windowing.TupleWindow;
 
 /**
  * TopologyBuilder exposes the Java API for specifying a topology for Storm
@@ -89,18 +86,20 @@
  * the inputs for that component.</p>
  */
 public class TopologyBuilder {
-    private Map<String, IRichBolt> _bolts = new HashMap<String, IRichBolt>();
-    private Map<String, IRichSpout> _spouts = new HashMap<String, IRichSpout>();
-    private Map<String, ComponentCommon> _commons = new HashMap<String, ComponentCommon>();
+    private Map<String, IRichBolt> _bolts = new HashMap<>();
+    private Map<String, IRichSpout> _spouts = new HashMap<>();
+    private Map<String, ComponentCommon> _commons = new HashMap<>();
 
 //    private Map<String, Map<GlobalStreamId, Grouping>> _inputs = new HashMap<String, Map<GlobalStreamId, Grouping>>();
 
-    private Map<String, StateSpoutSpec> _stateSpouts = new HashMap<String, StateSpoutSpec>();
-    
-    
+    private Map<String, StateSpoutSpec> _stateSpouts = new HashMap<>();
+    private List<ByteBuffer> _workerHooks = new ArrayList<>();
+
+
     public StormTopology createTopology() {
-        Map<String, Bolt> boltSpecs = new HashMap<String, Bolt>();
-        Map<String, SpoutSpec> spoutSpecs = new HashMap<String, SpoutSpec>();
+        Map<String, Bolt> boltSpecs = new HashMap<>();
+        Map<String, SpoutSpec> spoutSpecs = new HashMap<>();
+
         for(String boltId: _bolts.keySet()) {
             IRichBolt bolt = _bolts.get(boltId);
             ComponentCommon common = getComponentCommon(boltId, bolt);
@@ -110,11 +109,15 @@
             IRichSpout spout = _spouts.get(spoutId);
             ComponentCommon common = getComponentCommon(spoutId, spout);
             spoutSpecs.put(spoutId, new SpoutSpec(ComponentObject.serialized_java(Utils.javaSerialize(spout)), common));
-            
         }
-        return new StormTopology(spoutSpecs,
-                                 boltSpecs,
-                                 new HashMap<String, StateSpoutSpec>());
+
+        StormTopology stormTopology = new StormTopology(spoutSpecs,
+                boltSpecs,
+                new HashMap<String, StateSpoutSpec>());
+
+        stormTopology.set_worker_hooks(_workerHooks);
+
+        return stormTopology;
     }
 
     /**
@@ -168,7 +171,7 @@
      *
      * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs.
      * @param bolt the basic bolt
-     * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somwehere around the cluster.
+     * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somewhere around the cluster.
      * @return use the returned object to declare the inputs to this component
      * @throws IllegalArgumentException if {@code parallelism_hint} is not positive
      */
@@ -177,6 +180,21 @@
     }
 
     /**
+     * Define a new bolt in this topology. This defines a windowed bolt, intended
+     * for windowing operations. The {@link IWindowedBolt#execute(TupleWindow)} method
+     * is triggered for each window interval with the list of current events in the window.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs.
+     * @param bolt the windowed bolt
+     * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somwehere around the cluster.
+     * @return use the returned object to declare the inputs to this component
+     * @throws IllegalArgumentException if {@code parallelism_hint} is not positive
+     */
+    public BoltDeclarer setBolt(String id, IWindowedBolt bolt, Number parallelism_hint) throws IllegalArgumentException {
+        return setBolt(id, new WindowedBoltExecutor(bolt), parallelism_hint);
+    }
+
+    /**
      * Define a new spout in this topology.
      *
      * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs.
@@ -193,7 +211,7 @@
      * will be allocated to this component.
      *
      * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs.
-     * @param parallelism_hint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a process somwehere around the cluster.
+     * @param parallelism_hint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a process somewhere around the cluster.
      * @param spout the spout
      * @throws IllegalArgumentException if {@code parallelism_hint} is not positive
      */
@@ -213,6 +231,18 @@
         // TODO: finish
     }
 
+    /**
+     * Add a new worker lifecycle hook
+     *
+     * @param workerHook the lifecycle hook to add
+     */
+    public void addWorkerHook(IWorkerHook workerHook) {
+        if(null == workerHook) {
+            throw new IllegalArgumentException("WorkerHook must not be null.");
+        }
+
+        _workerHooks.add(ByteBuffer.wrap(Utils.javaSerialize(workerHook)));
+    }
 
     private void validateUnusedId(String id) {
         if(_bolts.containsKey(id)) {
diff --git a/storm-core/src/jvm/backtype/storm/topology/WindowedBoltExecutor.java b/storm-core/src/jvm/backtype/storm/topology/WindowedBoltExecutor.java
new file mode 100644
index 0000000..c7d6f70
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/WindowedBoltExecutor.java
@@ -0,0 +1,224 @@
+/**
+ * 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.topology;
+
+import backtype.storm.Config;
+import backtype.storm.task.IOutputCollector;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.windowing.TupleWindowImpl;
+import backtype.storm.windowing.WindowLifecycleListener;
+import backtype.storm.windowing.WindowManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static backtype.storm.topology.base.BaseWindowedBolt.Count;
+import static backtype.storm.topology.base.BaseWindowedBolt.Duration;
+
+/**
+ * An {@link IWindowedBolt} wrapper that does the windowing of tuples.
+ */
+public class WindowedBoltExecutor implements IRichBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(WindowedBoltExecutor.class);
+
+    private IWindowedBolt bolt;
+    private transient WindowedOutputCollector windowedOutputCollector;
+    private transient WindowLifecycleListener<Tuple> listener;
+    private transient WindowManager<Tuple> windowManager;
+
+    public WindowedBoltExecutor(IWindowedBolt bolt) {
+        this.bolt = bolt;
+    }
+
+    private int getTopologyTimeoutMillis(Map stormConf) {
+        if (stormConf.get(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS) != null) {
+            boolean timeOutsEnabled = (boolean) stormConf.get(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS);
+            if (!timeOutsEnabled) {
+                return Integer.MAX_VALUE;
+            }
+        }
+        int timeout = 0;
+        if (stormConf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS) != null) {
+            timeout = ((Number) stormConf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)).intValue();
+        }
+        return timeout * 1000;
+    }
+
+    private int getMaxSpoutPending(Map stormConf) {
+        int maxPending = Integer.MAX_VALUE;
+        if (stormConf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING) != null) {
+            maxPending = ((Number) stormConf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING)).intValue();
+        }
+        return maxPending;
+    }
+
+    private void ensureDurationLessThanTimeout(int duration, int timeout) {
+        if (duration > timeout) {
+            throw new IllegalArgumentException("Window duration (length + sliding interval) value " + duration +
+                                                       " is more than " + Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS +
+                                                       " value " + timeout);
+        }
+    }
+
+    private void ensureCountLessThanMaxPending(int count, int maxPending) {
+        if (count > maxPending) {
+            throw new IllegalArgumentException("Window count (length + sliding interval) value " + count +
+                                                       " is more than " + Config.TOPOLOGY_MAX_SPOUT_PENDING +
+                                                       " value " + maxPending);
+        }
+    }
+
+    private void validate(Map stormConf, Count windowLengthCount, Duration windowLengthDuration,
+                          Count slidingIntervalCount, Duration slidingIntervalDuration) {
+
+        int topologyTimeout = getTopologyTimeoutMillis(stormConf);
+        int maxSpoutPending = getMaxSpoutPending(stormConf);
+        if (windowLengthDuration != null && slidingIntervalDuration != null) {
+            ensureDurationLessThanTimeout(windowLengthDuration.value + slidingIntervalDuration.value, topologyTimeout);
+        } else if (windowLengthDuration != null) {
+            ensureDurationLessThanTimeout(windowLengthDuration.value, topologyTimeout);
+        } else if (slidingIntervalDuration != null) {
+            ensureDurationLessThanTimeout(slidingIntervalDuration.value, topologyTimeout);
+        }
+
+        if (windowLengthCount != null && slidingIntervalCount != null) {
+            ensureCountLessThanMaxPending(windowLengthCount.value + slidingIntervalCount.value, maxSpoutPending);
+        } else if (windowLengthCount != null) {
+            ensureCountLessThanMaxPending(windowLengthCount.value, maxSpoutPending);
+        } else if (slidingIntervalCount != null) {
+            ensureCountLessThanMaxPending(slidingIntervalCount.value, maxSpoutPending);
+        }
+    }
+
+    private WindowManager<Tuple> initWindowManager(WindowLifecycleListener<Tuple> lifecycleListener, Map stormConf) {
+        WindowManager<Tuple> manager = new WindowManager<>(lifecycleListener);
+        Duration windowLengthDuration = null;
+        Count windowLengthCount = null;
+        Duration slidingIntervalDuration = null;
+        Count slidingIntervalCount = null;
+        if (stormConf.containsKey(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT)) {
+            windowLengthCount = new Count(((Number) stormConf.get(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT)).intValue());
+        } else if (stormConf.containsKey(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS)) {
+            windowLengthDuration = new Duration(
+                    ((Number) stormConf.get(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS)).intValue(),
+                    TimeUnit.MILLISECONDS);
+        }
+
+        if (stormConf.containsKey(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT)) {
+            slidingIntervalCount = new Count(((Number) stormConf.get(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT)).intValue());
+        } else if (stormConf.containsKey(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS)) {
+            slidingIntervalDuration = new Duration(((Number) stormConf.get(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS)).intValue(), TimeUnit.MILLISECONDS);
+        } else {
+            // default is a sliding window of count 1
+            slidingIntervalCount = new Count(1);
+        }
+        // validate
+        validate(stormConf, windowLengthCount, windowLengthDuration,
+                 slidingIntervalCount, slidingIntervalDuration);
+        if (windowLengthCount != null) {
+            manager.setWindowLength(windowLengthCount);
+        } else {
+            manager.setWindowLength(windowLengthDuration);
+        }
+        if (slidingIntervalCount != null) {
+            manager.setSlidingInterval(slidingIntervalCount);
+        } else {
+            manager.setSlidingInterval(slidingIntervalDuration);
+        }
+        return manager;
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.windowedOutputCollector = new WindowedOutputCollector(collector);
+        bolt.prepare(stormConf, context, windowedOutputCollector);
+        this.listener = newWindowLifecycleListener();
+        this.windowManager = initWindowManager(listener, stormConf);
+        LOG.debug("Initialized window manager {} ", this.windowManager);
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        windowManager.add(input);
+    }
+
+    @Override
+    public void cleanup() {
+        windowManager.shutdown();
+        bolt.cleanup();
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        bolt.declareOutputFields(declarer);
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return bolt.getComponentConfiguration();
+    }
+
+    private WindowLifecycleListener<Tuple> newWindowLifecycleListener() {
+        return new WindowLifecycleListener<Tuple>() {
+            @Override
+            public void onExpiry(List<Tuple> tuples) {
+                for (Tuple tuple : tuples) {
+                    windowedOutputCollector.ack(tuple);
+                }
+            }
+
+            @Override
+            public void onActivation(List<Tuple> tuples, List<Tuple> newTuples, List<Tuple> expiredTuples) {
+                windowedOutputCollector.setContext(tuples);
+                bolt.execute(new TupleWindowImpl(tuples, newTuples, expiredTuples));
+            }
+        };
+    }
+
+    /**
+     * Creates an {@link OutputCollector} wrapper that automatically
+     * anchors the tuples to inputTuples while emitting.
+     */
+    private static class WindowedOutputCollector extends OutputCollector {
+        private List<Tuple> inputTuples;
+
+        WindowedOutputCollector(IOutputCollector delegate) {
+            super(delegate);
+        }
+
+        void setContext(List<Tuple> inputTuples) {
+            this.inputTuples = inputTuples;
+        }
+
+        @Override
+        public List<Integer> emit(String streamId, List<Object> tuple) {
+            return emit(streamId, inputTuples, tuple);
+        }
+
+        @Override
+        public void emitDirect(int taskId, String streamId, List<Object> tuple) {
+            emitDirect(taskId, streamId, inputTuples, tuple);
+        }
+    }
+
+}
diff --git a/storm-core/src/jvm/backtype/storm/topology/base/BaseBatchBolt.java b/storm-core/src/jvm/backtype/storm/topology/base/BaseBatchBolt.java
index 3206941..3f35149 100644
--- a/storm-core/src/jvm/backtype/storm/topology/base/BaseBatchBolt.java
+++ b/storm-core/src/jvm/backtype/storm/topology/base/BaseBatchBolt.java
@@ -18,7 +18,6 @@
 package backtype.storm.topology.base;
 
 import backtype.storm.coordination.IBatchBolt;
-import java.util.Map;
 
 public abstract class BaseBatchBolt<T> extends BaseComponent implements IBatchBolt<T> {
  
diff --git a/storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalSpout.java b/storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalSpout.java
index 704a95b..e2b3a93 100644
--- a/storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalSpout.java
+++ b/storm-core/src/jvm/backtype/storm/topology/base/BaseTransactionalSpout.java
@@ -18,7 +18,6 @@
 package backtype.storm.topology.base;
 
 import backtype.storm.transactional.ITransactionalSpout;
-import java.util.Map;
 
 public abstract class BaseTransactionalSpout<T> extends BaseComponent implements ITransactionalSpout<T> {
 
diff --git a/storm-core/src/jvm/backtype/storm/topology/base/BaseWindowedBolt.java b/storm-core/src/jvm/backtype/storm/topology/base/BaseWindowedBolt.java
new file mode 100644
index 0000000..fd4af90
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/topology/base/BaseWindowedBolt.java
@@ -0,0 +1,179 @@
+/**
+ * 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.topology.base;
+
+import backtype.storm.Config;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IWindowedBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.windowing.TupleWindow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public abstract class BaseWindowedBolt implements IWindowedBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(BaseWindowedBolt.class);
+
+    private final transient Map<String, Object> windowConfiguration;
+
+    /**
+     * Holds a count value for count based windows and sliding intervals.
+     */
+    public static class Count {
+        public final int value;
+
+        public Count(int value) {
+            this.value = value;
+        }
+    }
+
+    /**
+     * Holds a Time duration for time based windows and sliding intervals.
+     */
+    public static class Duration {
+        public final int value;
+
+        public Duration(int value, TimeUnit timeUnit) {
+            this.value = (int) timeUnit.toMillis(value);
+        }
+    }
+
+    protected BaseWindowedBolt() {
+        windowConfiguration = new HashMap<>();
+    }
+
+    private BaseWindowedBolt withWindowLength(Count count) {
+        windowConfiguration.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_COUNT, count.value);
+        return this;
+    }
+
+    private BaseWindowedBolt withWindowLength(Duration duration) {
+        windowConfiguration.put(Config.TOPOLOGY_BOLTS_WINDOW_LENGTH_DURATION_MS, duration.value);
+        return this;
+    }
+
+    private BaseWindowedBolt withSlidingInterval(Count count) {
+        windowConfiguration.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_COUNT, count.value);
+        return this;
+    }
+
+    private BaseWindowedBolt withSlidingInterval(Duration duration) {
+        windowConfiguration.put(Config.TOPOLOGY_BOLTS_SLIDING_INTERVAL_DURATION_MS, duration.value);
+        return this;
+    }
+
+    /**
+     * Tuple count based sliding window configuration.
+     *
+     * @param windowLength    the number of tuples in the window
+     * @param slidingInterval the number of tuples after which the window slides
+     */
+    public BaseWindowedBolt withWindow(Count windowLength, Count slidingInterval) {
+        return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
+    }
+
+    /**
+     * Tuple count and time duration based sliding window configuration.
+     *
+     * @param windowLength    the number of tuples in the window
+     * @param slidingInterval the time duration after which the window slides
+     */
+    public BaseWindowedBolt withWindow(Count windowLength, Duration slidingInterval) {
+        return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
+    }
+
+    /**
+     * Time duration and count based sliding window configuration.
+     *
+     * @param windowLength    the time duration of the window
+     * @param slidingInterval the number of tuples after which the window slides
+     */
+    public BaseWindowedBolt withWindow(Duration windowLength, Count slidingInterval) {
+        return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
+    }
+
+    /**
+     * Time duration based sliding window configuration.
+     *
+     * @param windowLength    the time duration of the window
+     * @param slidingInterval the time duration after which the window slides
+     */
+    public BaseWindowedBolt withWindow(Duration windowLength, Duration slidingInterval) {
+        return withWindowLength(windowLength).withSlidingInterval(slidingInterval);
+    }
+
+    /**
+     * A tuple count based window that slides with every incoming tuple.
+     *
+     * @param windowLength the number of tuples in the window
+     */
+    public BaseWindowedBolt withWindow(Count windowLength) {
+        return withWindowLength(windowLength).withSlidingInterval(new Count(1));
+    }
+
+    /**
+     * A time duration based window that slides with every incoming tuple.
+     *
+     * @param windowLength the time duration of the window
+     */
+    public BaseWindowedBolt withWindow(Duration windowLength) {
+        return withWindowLength(windowLength).withSlidingInterval(new Count(1));
+    }
+
+    /**
+     * A count based tumbling window.
+     *
+     * @param count the number of tuples after which the window tumbles
+     */
+    public BaseWindowedBolt withTumblingWindow(Count count) {
+        return withWindowLength(count).withSlidingInterval(count);
+    }
+
+    /**
+     * A time duration based tumbling window.
+     *
+     * @param duration the time duration after which the window tumbles
+     */
+    public BaseWindowedBolt withTumblingWindow(Duration duration) {
+        return withWindowLength(duration).withSlidingInterval(duration);
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        // NOOP
+    }
+
+    @Override
+    public void cleanup() {
+        // NOOP
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        // NOOP
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return windowConfiguration;
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java b/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java
index 53aacae..101abe6 100644
--- a/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java
@@ -31,13 +31,13 @@
 import org.slf4j.LoggerFactory;
 
 public class TransactionalSpoutBatchExecutor implements IRichBolt {
-    public static Logger LOG = LoggerFactory.getLogger(TransactionalSpoutBatchExecutor.class);    
+    public static final Logger LOG = LoggerFactory.getLogger(TransactionalSpoutBatchExecutor.class);
 
     BatchOutputCollectorImpl _collector;
     ITransactionalSpout _spout;
     ITransactionalSpout.Emitter _emitter;
     
-    TreeMap<BigInteger, TransactionAttempt> _activeTransactions = new TreeMap<BigInteger, TransactionAttempt>();
+    TreeMap<BigInteger, TransactionAttempt> _activeTransactions = new TreeMap<>();
 
     public TransactionalSpoutBatchExecutor(ITransactionalSpout spout) {
         _spout = spout;
diff --git a/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutCoordinator.java b/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutCoordinator.java
index f7ce534..f8f73f6 100644
--- a/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutCoordinator.java
+++ b/storm-core/src/jvm/backtype/storm/transactional/TransactionalSpoutCoordinator.java
@@ -52,7 +52,7 @@
     private TransactionalState _state;
     private RotatingTransactionalState _coordinatorState;
     
-    TreeMap<BigInteger, TransactionStatus> _activeTx = new TreeMap<BigInteger, TransactionStatus>();
+    TreeMap<BigInteger, TransactionStatus> _activeTx = new TreeMap<>();
     
     private SpoutOutputCollector _collector;
     private Random _rand;
diff --git a/storm-core/src/jvm/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java b/storm-core/src/jvm/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java
index aabcb7a..02185f4 100644
--- a/storm-core/src/jvm/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java
@@ -63,8 +63,8 @@
     public class Emitter implements ICommitterTransactionalSpout.Emitter {
         IOpaquePartitionedTransactionalSpout.Emitter _emitter;
         TransactionalState _state;
-        TreeMap<BigInteger, Map<Integer, Object>> _cachedMetas = new TreeMap<BigInteger, Map<Integer, Object>>();
-        Map<Integer, RotatingTransactionalState> _partitionStates = new HashMap<Integer, RotatingTransactionalState>();
+        TreeMap<BigInteger, Map<Integer, Object>> _cachedMetas = new TreeMap<>();
+        Map<Integer, RotatingTransactionalState> _partitionStates = new HashMap<>();
         int _index;
         int _numTasks;
         
@@ -84,7 +84,7 @@
         
         @Override
         public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, BatchOutputCollector collector) {
-            Map<Integer, Object> metas = new HashMap<Integer, Object>();
+            Map<Integer, Object> metas = new HashMap<>();
             _cachedMetas.put(tx.getTransactionId(), metas);
             int partitions = _emitter.numPartitions();
             Entry<BigInteger, Map<Integer, Object>> entry = _cachedMetas.lowerEntry(tx.getTransactionId());
@@ -92,7 +92,7 @@
             if(entry!=null) {
                 prevCached = entry.getValue();
             } else {
-                prevCached = new HashMap<Integer, Object>();
+                prevCached = new HashMap<>();
             }
             
             for(int i=_index; i < partitions; i+=_numTasks) {
@@ -120,8 +120,9 @@
         public void commit(TransactionAttempt attempt) {
             BigInteger txid = attempt.getTransactionId();
             Map<Integer, Object> metas = _cachedMetas.remove(txid);
-            for(Integer partition: metas.keySet()) {
-                Object meta = metas.get(partition);
+            for(Entry<Integer, Object> entry: metas.entrySet()) {
+                Integer partition = entry.getKey();
+                Object meta = entry.getValue();
                 _partitionStates.get(partition).overrideState(txid, meta);
             }
         }
diff --git a/storm-core/src/jvm/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java b/storm-core/src/jvm/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java
index 479dda4..76859cf 100644
--- a/storm-core/src/jvm/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java
+++ b/storm-core/src/jvm/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java
@@ -67,7 +67,7 @@
     class Emitter implements ITransactionalSpout.Emitter<Integer> {
         private IPartitionedTransactionalSpout.Emitter _emitter;
         private TransactionalState _state;
-        private Map<Integer, RotatingTransactionalState> _partitionStates = new HashMap<Integer, RotatingTransactionalState>();
+        private Map<Integer, RotatingTransactionalState> _partitionStates = new HashMap<>();
         private int _index;
         private int _numTasks;
         
diff --git a/storm-core/src/jvm/backtype/storm/tuple/AddressedTuple.java b/storm-core/src/jvm/backtype/storm/tuple/AddressedTuple.java
new file mode 100644
index 0000000..c3aec72
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/tuple/AddressedTuple.java
@@ -0,0 +1,48 @@
+/**
+ * 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.tuple;
+
+/**
+ * A Tuple that is addressed to a destination.
+ */
+public class AddressedTuple {
+    /**
+     * Destination used when broadcasting a tuple.
+     */
+    public static final int BROADCAST_DEST = -2;
+    public final Tuple tuple;
+    public final int dest;
+
+    public AddressedTuple(int dest, Tuple tuple) {
+        this.dest = dest;
+        this.tuple = tuple;
+    }
+
+    public Tuple getTuple() {
+        return tuple;
+    }
+
+    public int getDest() {
+        return dest;
+    }
+
+    @Override
+    public String toString() {
+        return "[dest: "+dest+" tuple: "+tuple+"]";
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/tuple/Fields.java b/storm-core/src/jvm/backtype/storm/tuple/Fields.java
index 3eed409..b52b798 100644
--- a/storm-core/src/jvm/backtype/storm/tuple/Fields.java
+++ b/storm-core/src/jvm/backtype/storm/tuple/Fields.java
@@ -30,14 +30,14 @@
  */
 public class Fields implements Iterable<String>, Serializable {
     private List<String> _fields;
-    private Map<String, Integer> _index = new HashMap<String, Integer>();
+    private Map<String, Integer> _index = new HashMap<>();
     
     public Fields(String... fields) {
         this(Arrays.asList(fields));
     }
     
     public Fields(List<String> fields) {
-        _fields = new ArrayList<String>(fields.size());
+        _fields = new ArrayList<>(fields.size());
         for (String field : fields) {
             if (_fields.contains(field))
                 throw new IllegalArgumentException(
@@ -49,7 +49,7 @@
     }
     
     public List<Object> select(Fields selector, List<Object> tuple) {
-        List<Object> ret = new ArrayList<Object>(selector.size());
+        List<Object> ret = new ArrayList<>(selector.size());
         for(String s: selector) {
             ret.add(tuple.get(_index.get(s)));
         }
@@ -57,7 +57,7 @@
     }
 
     public List<String> toList() {
-        return new ArrayList<String>(_fields);
+        return new ArrayList<>(_fields);
     }
     
     /**
@@ -98,7 +98,7 @@
     }
     
     /**
-     * @returns true if this contains the specified name of the field.
+     * @return true if this contains the specified name of the field.
      */
     public boolean contains(String field) {
         return _index.containsKey(field);
diff --git a/storm-core/src/jvm/backtype/storm/tuple/MessageId.java b/storm-core/src/jvm/backtype/storm/tuple/MessageId.java
index 680af38..554bab6 100644
--- a/storm-core/src/jvm/backtype/storm/tuple/MessageId.java
+++ b/storm-core/src/jvm/backtype/storm/tuple/MessageId.java
@@ -43,7 +43,7 @@
     }
         
     public static MessageId makeRootId(long id, long val) {
-        Map<Long, Long> anchorsToIds = new HashMap<Long, Long>();
+        Map<Long, Long> anchorsToIds = new HashMap<>();
         anchorsToIds.put(id, val);
         return new MessageId(anchorsToIds);
     }
@@ -67,11 +67,7 @@
 
     @Override
     public boolean equals(Object other) {
-        if(other instanceof MessageId) {
-            return _anchorsToIds.equals(((MessageId) other)._anchorsToIds);
-        } else {
-            return false;
-        }
+        return other instanceof MessageId && _anchorsToIds.equals(((MessageId) other)._anchorsToIds);
     }
 
     @Override
@@ -89,7 +85,7 @@
 
     public static MessageId deserialize(Input in) throws IOException {
         int numAnchors = in.readInt(true);
-        Map<Long, Long> anchorsToIds = new HashMap<Long, Long>();
+        Map<Long, Long> anchorsToIds = new HashMap<>();
         for(int i=0; i<numAnchors; i++) {
             anchorsToIds.put(in.readLong(), in.readLong());
         }
diff --git a/storm-core/src/jvm/backtype/storm/tuple/Tuple.java b/storm-core/src/jvm/backtype/storm/tuple/Tuple.java
index 799bf68..19d0cb4 100644
--- a/storm-core/src/jvm/backtype/storm/tuple/Tuple.java
+++ b/storm-core/src/jvm/backtype/storm/tuple/Tuple.java
@@ -18,7 +18,6 @@
 package backtype.storm.tuple;
 
 import backtype.storm.generated.GlobalStreamId;
-import java.util.List;
 
 /**
  * The tuple is the main data structure in Storm. A tuple is a named list of values, 
@@ -36,10 +35,18 @@
 
     /**
      * Returns the global stream id (component + stream) of this tuple.
+     * 
+     * @deprecated replaced by {@link #getSourceGlobalStreamId()} due to broken naming convention
      */
+    @Deprecated
     public GlobalStreamId getSourceGlobalStreamid();
     
     /**
+     * Returns the global stream id (component + stream) of this tuple.
+     */
+    public GlobalStreamId getSourceGlobalStreamId();
+
+    /**
      * Gets the id of the component that created this tuple.
      */
     public String getSourceComponent();
diff --git a/storm-core/src/jvm/backtype/storm/tuple/TupleImpl.java b/storm-core/src/jvm/backtype/storm/tuple/TupleImpl.java
index 818eff1..dd31c96 100644
--- a/storm-core/src/jvm/backtype/storm/tuple/TupleImpl.java
+++ b/storm-core/src/jvm/backtype/storm/tuple/TupleImpl.java
@@ -40,7 +40,7 @@
     private String streamId;
     private GeneralTopologyContext context;
     private MessageId id;
-    private IPersistentMap _meta = null;
+    private IPersistentMap _meta;
     
     public TupleImpl(GeneralTopologyContext context, List<Object> values, int taskId, String streamId, MessageId id) {
         this.values = values;
@@ -63,8 +63,8 @@
         this(context, values, taskId, streamId, MessageId.makeUnanchored());
     }    
     
-    Long _processSampleStartTime = null;
-    Long _executeSampleStartTime = null;
+    Long _processSampleStartTime;
+    Long _executeSampleStartTime;
     
     public void setProcessSampleStartTime(long ms) {
         _processSampleStartTime = ms;
@@ -196,8 +196,13 @@
     public List<Object> select(Fields selector) {
         return getFields().select(selector, values);
     }
-      
+    
+    @Deprecated
     public GlobalStreamId getSourceGlobalStreamid() {
+        return getSourceGlobalStreamId();
+    }
+    
+    public GlobalStreamId getSourceGlobalStreamId() {
         return new GlobalStreamId(getSourceComponent(), streamId);
     }
     
@@ -232,7 +237,7 @@
         return System.identityHashCode(this);
     }
 
-    private final Keyword makeKeyword(String name) {
+    private Keyword makeKeyword(String name) {
         return Keyword.intern(Symbol.create(name));
     }    
 
@@ -245,7 +250,7 @@
             } else if(o instanceof String) {
                 return getValueByField((String) o);
             }
-        } catch(IllegalArgumentException e) {
+        } catch(IllegalArgumentException ignored) {
         }
         return null;
     }
diff --git a/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java b/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java
index b2a2a7d..23deb28 100644
--- a/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/DRPCClient.java
@@ -17,7 +17,6 @@
  */
 package backtype.storm.utils;
 
-import backtype.storm.Config;
 import backtype.storm.generated.DRPCExecutionException;
 import backtype.storm.generated.DistributedRPC;
 import backtype.storm.generated.AuthorizationException;
diff --git a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
index e0053e9..5d68909 100644
--- a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java
@@ -6,9 +6,9 @@
  * 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.
@@ -18,102 +18,403 @@
 package backtype.storm.utils;
 
 import com.lmax.disruptor.AlertException;
-import com.lmax.disruptor.ClaimStrategy;
 import com.lmax.disruptor.EventFactory;
 import com.lmax.disruptor.EventHandler;
 import com.lmax.disruptor.InsufficientCapacityException;
+import com.lmax.disruptor.LiteBlockingWaitStrategy;
 import com.lmax.disruptor.RingBuffer;
 import com.lmax.disruptor.Sequence;
 import com.lmax.disruptor.SequenceBarrier;
-import com.lmax.disruptor.SingleThreadedClaimStrategy;
+import com.lmax.disruptor.TimeoutBlockingWaitStrategy;
+import com.lmax.disruptor.TimeoutException;
 import com.lmax.disruptor.WaitStrategy;
+import com.lmax.disruptor.dsl.ProducerType;
 
-import java.util.Random;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import backtype.storm.metric.api.IStatefulObject;
-
+import backtype.storm.metric.internal.RateTracker;
 
 /**
  * A single consumer queue that uses the LMAX Disruptor. They key to the performance is
  * the ability to catch up to the producer by processing tuples in batches.
  */
 public class DisruptorQueue implements IStatefulObject {
-    static final Object FLUSH_CACHE = new Object();
-    static final Object INTERRUPT = new Object();
+    private static final Logger LOG = LoggerFactory.getLogger(DisruptorQueue.class);    
+    private static final Object INTERRUPT = new Object();
+    private static final String PREFIX = "disruptor-";
+    private static final FlusherPool FLUSHER = new FlusherPool();
 
-    RingBuffer<MutableObject> _buffer;
-    Sequence _consumer;
-    SequenceBarrier _barrier;
+    private static class FlusherPool { 
+        private Timer _timer = new Timer("disruptor-flush-trigger", true);
+        private ThreadPoolExecutor _exec = new ThreadPoolExecutor(1, 100, 10, TimeUnit.SECONDS, new ArrayBlockingQueue<Runnable>(1024), new ThreadPoolExecutor.DiscardPolicy());
+        private HashMap<Long, ArrayList<Flusher>> _pendingFlush = new HashMap<>();
+        private HashMap<Long, TimerTask> _tt = new HashMap<>();
 
-    // TODO: consider having a threadlocal cache of this variable to speed up reads?
-    volatile boolean consumerStartedFlag = false;
-    ConcurrentLinkedQueue<Object> _cache = new ConcurrentLinkedQueue();
+        public synchronized void start(Flusher flusher, final long flushInterval) {
+            ArrayList<Flusher> pending = _pendingFlush.get(flushInterval);
+            if (pending == null) {
+                pending = new ArrayList<>();
+                TimerTask t = new TimerTask() {
+                    @Override
+                    public void run() {
+                        invokeAll(flushInterval);
+                    }
+                };
+                _pendingFlush.put(flushInterval, pending);
+                _timer.schedule(t, flushInterval, flushInterval);
+                _tt.put(flushInterval, t);
+            }
+            pending.add(flusher);
+        }
 
-    private final ReentrantReadWriteLock cacheLock = new ReentrantReadWriteLock();
-    private final Lock readLock = cacheLock.readLock();
-    private final Lock writeLock = cacheLock.writeLock();
+        private synchronized void invokeAll(long flushInterval) {
+            ArrayList<Flusher> tasks = _pendingFlush.get(flushInterval);
+            if (tasks != null) {
+                for (Flusher f: tasks) {
+                    _exec.submit(f);
+                }
+            }
+        }
 
-    private static String PREFIX = "disruptor-";
-    private String _queueName = "";
+        public synchronized void stop(Flusher flusher, long flushInterval) {
+            ArrayList<Flusher> pending = _pendingFlush.get(flushInterval);
+            pending.remove(flusher);
+            if (pending.size() == 0) {
+                _pendingFlush.remove(flushInterval);
+                _tt.remove(flushInterval).cancel();
+            }
+        }
+    }
 
-    private long _waitTimeout;
+    private static class ObjectEventFactory implements EventFactory<AtomicReference<Object>> {
+        @Override
+        public AtomicReference<Object> newInstance() {
+            return new AtomicReference<Object>();
+        }
+    }
 
+    private interface ThreadLocalInserter {
+        public void add(Object obj);
+        public void forceBatch();
+        public void flush(boolean block);
+    }
+
+    private class ThreadLocalJustInserter implements ThreadLocalInserter {
+        private final ReentrantLock _flushLock;
+        private final ConcurrentLinkedQueue<Object> _overflow;
+
+        public ThreadLocalJustInserter() {
+            _flushLock = new ReentrantLock();
+            _overflow = new ConcurrentLinkedQueue<>();
+        }
+
+        //called by the main thread and should not block for an undefined period of time
+        public synchronized void add(Object obj) {
+            boolean inserted = false;
+            if (_overflow.isEmpty()) {
+                try {
+                    publishDirectSingle(obj, false);
+                    inserted = true;
+                } catch (InsufficientCapacityException e) {
+                    //Ignored
+                }
+            }
+
+            if (!inserted) {
+                _overflowCount.incrementAndGet();
+                _overflow.add(obj);
+            }
+
+            if (_enableBackpressure && _cb != null && (_metrics.population() + _overflowCount.get()) >= _highWaterMark) {
+                try {
+                    if (!_throttleOn) {
+                        _cb.highWaterMark();
+                        _throttleOn = true;
+                    }
+                } catch (Exception e) {
+                    throw new RuntimeException("Exception during calling highWaterMark callback!", e);
+                }
+            }
+        }
+
+        //May be called by a background thread
+        public void forceBatch() {
+            //NOOP
+        }
+
+        //May be called by a background thread
+        public void flush(boolean block) {
+            if (block) {
+                _flushLock.lock();
+            } else if (!_flushLock.tryLock()) {
+               //Someone else if flushing so don't do anything
+               return;
+            }
+            try {
+                while (!_overflow.isEmpty()) {
+                    publishDirectSingle(_overflow.peek(), block);
+                    _overflowCount.addAndGet(-1);
+                    _overflow.poll();
+                }
+            } catch (InsufficientCapacityException e) {
+                //Ignored we should not block
+            } finally {
+                _flushLock.unlock();
+            }
+        }
+    }
+
+    private class ThreadLocalBatcher implements ThreadLocalInserter {
+        private final ReentrantLock _flushLock;
+        private final ConcurrentLinkedQueue<ArrayList<Object>> _overflow;
+        private ArrayList<Object> _currentBatch;
+
+        public ThreadLocalBatcher() {
+            _flushLock = new ReentrantLock();
+            _overflow = new ConcurrentLinkedQueue<ArrayList<Object>>();
+            _currentBatch = new ArrayList<Object>(_inputBatchSize);
+        }
+
+        //called by the main thread and should not block for an undefined period of time
+        public synchronized void add(Object obj) {
+            _currentBatch.add(obj);
+            _overflowCount.incrementAndGet();
+            if (_enableBackpressure && _cb != null && (_metrics.population() + _overflowCount.get()) >= _highWaterMark) {
+                try {
+                    if (!_throttleOn) {
+                        _cb.highWaterMark();
+                        _throttleOn = true;
+                    }
+                } catch (Exception e) {
+                    throw new RuntimeException("Exception during calling highWaterMark callback!", e);
+                }
+            }
+            if (_currentBatch.size() >= _inputBatchSize) {
+                boolean flushed = false;
+                if (_overflow.isEmpty()) {
+                    try {
+                        publishDirect(_currentBatch, false);
+                        _overflowCount.addAndGet(0 - _currentBatch.size());
+                        _currentBatch.clear();
+                        flushed = true;
+                    } catch (InsufficientCapacityException e) {
+                        //Ignored we will flush later
+                    }
+                }
+
+                if (!flushed) {        
+                    _overflow.add(_currentBatch);
+                    _currentBatch = new ArrayList<Object>(_inputBatchSize);
+                }
+            }
+        }
+
+        //May be called by a background thread
+        public synchronized void forceBatch() {
+            if (!_currentBatch.isEmpty()) {
+                _overflow.add(_currentBatch);
+                _currentBatch = new ArrayList<Object>(_inputBatchSize);
+            }
+        }
+
+        //May be called by a background thread
+        public void flush(boolean block) {
+            if (block) {
+                _flushLock.lock();
+            } else if (!_flushLock.tryLock()) {
+               //Someone else if flushing so don't do anything
+               return;
+            }
+            try {
+                while (!_overflow.isEmpty()) {
+                    publishDirect(_overflow.peek(), block);
+                    _overflowCount.addAndGet(0 - _overflow.poll().size());
+                }
+            } catch (InsufficientCapacityException e) {
+                //Ignored we should not block
+            } finally {
+                _flushLock.unlock();
+            }
+        }
+    }
+
+    private class Flusher implements Runnable {
+        private AtomicBoolean _isFlushing = new AtomicBoolean(false);
+        private final long _flushInterval;
+
+        public Flusher(long flushInterval, String name) {
+            _flushInterval = flushInterval;
+        }
+
+        public void run() {
+            if (_isFlushing.compareAndSet(false, true)) {
+                for (ThreadLocalInserter batcher: _batchers.values()) {
+                    batcher.forceBatch();
+                    batcher.flush(true);
+                }
+                _isFlushing.set(false);
+            }
+        }
+
+        public void start() {
+            FLUSHER.start(this, _flushInterval);
+        }
+
+        public void close() {
+            FLUSHER.stop(this, _flushInterval);
+        }
+    }
+
+    /**
+     * This inner class provides methods to access the metrics of the disruptor queue.
+     */
+    public class QueueMetrics {
+        private final RateTracker _rateTracker = new RateTracker(10000, 10);
+
+        public long writePos() {
+            return _buffer.getCursor();
+        }
+
+        public long readPos() {
+            return _consumer.get();
+        }
+
+        public long overflow() {
+            return _overflowCount.get();
+        }
+
+        public long population() {
+            return writePos() - readPos();
+        }
+
+        public long capacity() {
+            return _buffer.getBufferSize();
+        }
+
+        public float pctFull() {
+            return (1.0F * population() / capacity());
+        }
+
+        public Object getState() {
+            Map state = new HashMap<String, Object>();
+
+            // get readPos then writePos so it's never an under-estimate
+            long rp = readPos();
+            long wp = writePos();
+
+            final double arrivalRateInSecs = _rateTracker.reportRate();
+
+            //Assume the queue is stable, in which the arrival rate is equal to the consumption rate.
+            // If this assumption does not hold, the calculation of sojourn time should also consider
+            // departure rate according to Queuing Theory.
+            final double sojournTime = (wp - rp) / Math.max(arrivalRateInSecs, 0.00001) * 1000.0;
+
+            state.put("capacity", capacity());
+            state.put("population", wp - rp);
+            state.put("write_pos", wp);
+            state.put("read_pos", rp);
+            state.put("arrival_rate_secs", arrivalRateInSecs);
+            state.put("sojourn_time_ms", sojournTime); //element sojourn time in milliseconds
+            state.put("overflow", _overflowCount.get());
+
+            return state;
+        }
+
+        public void notifyArrivals(long counts) {
+            _rateTracker.notify(counts);
+        }
+    }
+
+    private final RingBuffer<AtomicReference<Object>> _buffer;
+    private final Sequence _consumer;
+    private final SequenceBarrier _barrier;
+    private final int _inputBatchSize;
+    private final ConcurrentHashMap<Long, ThreadLocalInserter> _batchers = new ConcurrentHashMap<Long, ThreadLocalInserter>();
+    private final Flusher _flusher;
     private final QueueMetrics _metrics;
+
+    private String _queueName = "";
     private DisruptorBackpressureCallback _cb = null;
     private int _highWaterMark = 0;
     private int _lowWaterMark = 0;
     private boolean _enableBackpressure = false;
+    private final AtomicLong _overflowCount = new AtomicLong(0);
+    private volatile boolean _throttleOn = false;
 
-    public DisruptorQueue(String queueName, ClaimStrategy claim, WaitStrategy wait, long timeout) {
+    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval) {
         this._queueName = PREFIX + queueName;
-        _buffer = new RingBuffer<MutableObject>(new ObjectEventFactory(), claim, wait);
-        _consumer = new Sequence();
-        _barrier = _buffer.newBarrier();
-        _buffer.setGatingSequences(_consumer);
-        _metrics = new QueueMetrics((float) 0.05);
-
-        if (claim instanceof SingleThreadedClaimStrategy) {
-            consumerStartedFlag = true;
+        WaitStrategy wait;
+        if (readTimeout <= 0) {
+            wait = new LiteBlockingWaitStrategy();
         } else {
-            // make sure we flush the pending messages in cache first
-            try {
-                publishDirect(FLUSH_CACHE, true);
-            } catch (InsufficientCapacityException e) {
-                throw new RuntimeException("This code should be unreachable!", e);
-            }
+            wait = new TimeoutBlockingWaitStrategy(readTimeout, TimeUnit.MILLISECONDS);
         }
 
-        _waitTimeout = timeout;
+        _buffer = RingBuffer.create(type, new ObjectEventFactory(), size, wait);
+        _consumer = new Sequence();
+        _barrier = _buffer.newBarrier();
+        _buffer.addGatingSequences(_consumer);
+        _metrics = new QueueMetrics();
+        //The batch size can be no larger than half the full queue size.
+        //This is mostly to avoid contention issues.
+        _inputBatchSize = Math.max(1, Math.min(inputBatchSize, size/2));
+
+        _flusher = new Flusher(Math.max(flushInterval, 1), _queueName);
+        _flusher.start();
     }
 
     public String getName() {
         return _queueName;
     }
 
-    public void consumeBatch(EventHandler<Object> handler) {
-        consumeBatchToCursor(_barrier.getCursor(), handler);
+    public boolean isFull() {
+        return (_metrics.population() + _overflowCount.get()) >= _metrics.capacity();
     }
 
     public void haltWithInterrupt() {
-        publish(INTERRUPT);
+        try {
+            publishDirect(new ArrayList<Object>(Arrays.asList(INTERRUPT)), true);
+            _flusher.close();
+        } catch (InsufficientCapacityException e) {
+            //This should be impossible
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void consumeBatch(EventHandler<Object> handler) {
+        if (_metrics.population() > 0) {
+            consumeBatchWhenAvailable(handler);
+        }
     }
 
     public void consumeBatchWhenAvailable(EventHandler<Object> handler) {
         try {
             final long nextSequence = _consumer.get() + 1;
-            final long availableSequence =
-                    _waitTimeout == 0L ? _barrier.waitFor(nextSequence) : _barrier.waitFor(nextSequence, _waitTimeout,
-                            TimeUnit.MILLISECONDS);
+            long availableSequence = _barrier.waitFor(nextSequence);
 
             if (availableSequence >= nextSequence) {
                 consumeBatchToCursor(availableSequence, handler);
             }
+        } catch (TimeoutException te) {
+            //Ignored
         } catch (AlertException e) {
             throw new RuntimeException(e);
         } catch (InterruptedException e) {
@@ -121,27 +422,23 @@
         }
     }
 
-
     private void consumeBatchToCursor(long cursor, EventHandler<Object> handler) {
         for (long curr = _consumer.get() + 1; curr <= cursor; curr++) {
             try {
-                MutableObject mo = _buffer.get(curr);
-                Object o = mo.o;
-                mo.setObject(null);
-                if (o == FLUSH_CACHE) {
-                    Object c = null;
-                    while (true) {
-                        c = _cache.poll();
-                        if (c == null) break;
-                        else handler.onEvent(c, curr, true);
-                    }
-                } else if (o == INTERRUPT) {
+                AtomicReference<Object> mo = _buffer.get(curr);
+                Object o = mo.getAndSet(null);
+                if (o == INTERRUPT) {
                     throw new InterruptedException("Disruptor processing interrupted");
+                } else if (o == null) {
+                    LOG.error("NULL found in {}:{}", this.getName(), cursor);
                 } else {
                     handler.onEvent(o, curr, curr == cursor);
-                    if (_enableBackpressure && _cb != null && _metrics.writePos() - curr <= _lowWaterMark) {
+                    if (_enableBackpressure && _cb != null && (_metrics.writePos() - curr + _overflowCount.get()) <= _lowWaterMark) {
                         try {
-                            _cb.lowWaterMark();
+                            if (_throttleOn) {
+                                _throttleOn = false;
+                                _cb.lowWaterMark();
+                            }
                         } catch (Exception e) {
                             throw new RuntimeException("Exception during calling lowWaterMark callback!");
                         }
@@ -151,7 +448,6 @@
                 throw new RuntimeException(e);
             }
         }
-        //TODO: only set this if the consumer cursor has changed?
         _consumer.set(cursor);
     }
 
@@ -159,69 +455,58 @@
         this._cb = cb;
     }
 
-    /*
-     * Caches until consumerStarted is called, upon which the cache is flushed to the consumer
-     */
-    public void publish(Object obj) {
-        try {
-            publish(obj, true);
-        } catch (InsufficientCapacityException ex) {
-            throw new RuntimeException("This code should be unreachable!");
-        }
+    private static Long getId() {
+        return Thread.currentThread().getId();
     }
 
-    public void tryPublish(Object obj) throws InsufficientCapacityException {
-        publish(obj, false);
-    }
-
-    public void publish(Object obj, boolean block) throws InsufficientCapacityException {
-
-        boolean publishNow = consumerStartedFlag;
-
-        if (!publishNow) {
-            readLock.lock();
-            try {
-                publishNow = consumerStartedFlag;
-                if (!publishNow) {
-                    _cache.add(obj);
-                }
-            } finally {
-                readLock.unlock();
-            }
-        }
-
-        if (publishNow) {
-            publishDirect(obj, block);
-        }
-    }
-
-    private void publishDirect(Object obj, boolean block) throws InsufficientCapacityException {
-        final long id;
+    private void publishDirectSingle(Object obj, boolean block) throws InsufficientCapacityException {
+        long at;
         if (block) {
-            id = _buffer.next();
+            at = _buffer.next();
         } else {
-            id = _buffer.tryNext(1);
+            at = _buffer.tryNext();
         }
-        final MutableObject m = _buffer.get(id);
-        m.setObject(obj);
-        _buffer.publish(id);
+        AtomicReference<Object> m = _buffer.get(at);
+        m.set(obj);
+        _buffer.publish(at);
         _metrics.notifyArrivals(1);
-        if (_enableBackpressure && _cb != null && _metrics.population() >= _highWaterMark) {
-           try {
-               _cb.highWaterMark();
-           } catch (Exception e) {
-               throw new RuntimeException("Exception during calling highWaterMark callback!");
-           }
+    }
+
+    private void publishDirect(ArrayList<Object> objs, boolean block) throws InsufficientCapacityException {
+        int size = objs.size();
+        if (size > 0) {
+            long end;
+            if (block) {
+                end = _buffer.next(size);
+            } else {
+                end = _buffer.tryNext(size);
+            }
+            long begin = end - (size - 1);
+            long at = begin;
+            for (Object obj: objs) {
+                AtomicReference<Object> m = _buffer.get(at);
+                m.set(obj);
+                at++;
+            }
+            _buffer.publish(begin, end);
+            _metrics.notifyArrivals(size);
         }
     }
 
-    public void consumerStarted() {
-
-        consumerStartedFlag = true;
-
-        // Use writeLock to make sure all pending cache add opearation completed
-        writeLock.lock();
-        writeLock.unlock();
+    public void publish(Object obj) {
+        Long id = getId();
+        ThreadLocalInserter batcher = _batchers.get(id);
+        if (batcher == null) {
+            //This thread is the only one ever creating this, so this is safe
+            if (_inputBatchSize > 1) {
+                batcher = new ThreadLocalBatcher();
+            } else {
+                batcher = new ThreadLocalJustInserter();
+            }
+            _batchers.put(id, batcher);
+        }
+        batcher.add(obj);
+        batcher.flush(false);
     }
 
     @Override
@@ -252,103 +537,8 @@
         return this;
     }
 
-    public static class ObjectEventFactory implements EventFactory<MutableObject> {
-        @Override
-        public MutableObject newInstance() {
-            return new MutableObject();
-        }
-    }
-
     //This method enables the metrics to be accessed from outside of the DisruptorQueue class
     public QueueMetrics getMetrics() {
         return _metrics;
     }
-
-
-    /**
-     * This inner class provides methods to access the metrics of the disruptor queue.
-     */
-    public class QueueMetrics {
-
-        private final RateTracker _rateTracker = new RateTracker(10000, 10);
-        private final float _sampleRate;
-        private Random _random;
-
-        public QueueMetrics() throws IllegalArgumentException {
-            this(1);
-        }
-
-        /**
-         * @param sampleRate a number between 0 and 1. The higher it is, the accurate the metrics
-         *                   will be. Using a reasonable sampleRate, e.g., 0.1, could effectively reduce the
-         *                   metric maintenance cost while providing good accuracy.
-         */
-        public QueueMetrics(float sampleRate) throws IllegalArgumentException {
-
-            if (sampleRate <= 0 || sampleRate > 1)
-                throw new IllegalArgumentException("sampleRate should be a value between (0,1].");
-
-            _sampleRate = sampleRate;
-
-            _random = new Random();
-        }
-
-        public long writePos() {
-            return _buffer.getCursor();
-        }
-
-        public long readPos() {
-            return _consumer.get();
-        }
-
-        public long population() {
-            return writePos() - readPos();
-        }
-
-        public long capacity() {
-            return _buffer.getBufferSize();
-        }
-
-        public float pctFull() {
-            return (1.0F * population() / capacity());
-        }
-
-        public Object getState() {
-            Map state = new HashMap<String, Object>();
-
-            // get readPos then writePos so it's never an under-estimate
-            long rp = readPos();
-            long wp = writePos();
-
-            final float arrivalRateInMils = _rateTracker.reportRate() / _sampleRate;
-
-            /*
-            Assume the queue is stable, in which the arrival rate is equal to the consumption rate.
-            If this assumption does not hold, the calculation of sojourn time should also consider
-            departure rate according to Queuing Theory.
-             */
-            final float sojournTime = (wp - rp) / (float) Math.max(arrivalRateInMils, 0.00001);
-
-            state.put("capacity", capacity());
-            state.put("population", wp - rp);
-            state.put("write_pos", wp);
-            state.put("read_pos", rp);
-            state.put("arrival_rate", arrivalRateInMils); //arrivals per millisecond
-            state.put("sojourn_time", sojournTime); //element sojourn time in milliseconds
-
-            return state;
-        }
-
-        public void notifyArrivals(long counts) {
-            if (sample())
-                _rateTracker.notify(counts);
-        }
-
-        final private boolean sample() {
-            if (_sampleRate == 1 || _random.nextFloat() < _sampleRate)
-                return true;
-            return false;
-        }
-    }
-
 }
diff --git a/storm-core/src/jvm/backtype/storm/utils/InprocMessaging.java b/storm-core/src/jvm/backtype/storm/utils/InprocMessaging.java
index b20c775..4abbc3e 100644
--- a/storm-core/src/jvm/backtype/storm/utils/InprocMessaging.java
+++ b/storm-core/src/jvm/backtype/storm/utils/InprocMessaging.java
@@ -22,7 +22,7 @@
 import java.util.concurrent.LinkedBlockingQueue;
 
 public class InprocMessaging {
-    private static Map<Integer, LinkedBlockingQueue<Object>> _queues = new HashMap<Integer, LinkedBlockingQueue<Object>>();
+    private static Map<Integer, LinkedBlockingQueue<Object>> _queues = new HashMap<>();
     private static final Object _lock = new Object();
     private static int port = 1;
     
@@ -50,7 +50,7 @@
     private static LinkedBlockingQueue<Object> getQueue(int port) {
         synchronized(_lock) {
             if(!_queues.containsKey(port)) {
-              _queues.put(port, new LinkedBlockingQueue<Object>());   
+              _queues.put(port, new LinkedBlockingQueue<>());
             }
             return _queues.get(port);
         }
diff --git a/storm-core/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java b/storm-core/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java
index 3cb455d..4ecdda6 100644
--- a/storm-core/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java
+++ b/storm-core/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java
@@ -28,15 +28,15 @@
 public class KeyedRoundRobinQueue<V> {
     private final Object _lock = new Object();
     private Semaphore _size = new Semaphore(0);
-    private Map<Object, Queue<V>> _queues = new HashMap<Object, Queue<V>>();
-    private List<Object> _keyOrder = new ArrayList<Object>();
+    private Map<Object, Queue<V>> _queues = new HashMap<>();
+    private List<Object> _keyOrder = new ArrayList<>();
     private int _currIndex = 0;
 
     public void add(Object key, V val) {
         synchronized(_lock) {
             Queue<V> queue = _queues.get(key);
             if(queue==null) {
-                queue = new LinkedList<V>();
+                queue = new LinkedList<>();
                 _queues.put(key, queue);
                 _keyOrder.add(key);
             }
diff --git a/storm-core/src/jvm/backtype/storm/utils/ListDelegate.java b/storm-core/src/jvm/backtype/storm/utils/ListDelegate.java
index 1e091f0..d198a72 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ListDelegate.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ListDelegate.java
@@ -27,7 +27,7 @@
     private List<Object> _delegate;
     
     public ListDelegate() {
-    	_delegate = new ArrayList<Object>();
+    	_delegate = new ArrayList<>();
     }
     
     public void setDelegate(List<Object> delegate) {
@@ -84,12 +84,12 @@
     }
 
     @Override
-    public boolean addAll(Collection<? extends Object> clctn) {
+    public boolean addAll(Collection<?> clctn) {
         return _delegate.addAll(clctn);
     }
 
     @Override
-    public boolean addAll(int i, Collection<? extends Object> clctn) {
+    public boolean addAll(int i, Collection<?> clctn) {
         return _delegate.addAll(i, clctn);
     }
 
diff --git a/storm-core/src/jvm/backtype/storm/utils/LocalState.java b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
index b6ecba2..847fea4 100644
--- a/storm-core/src/jvm/backtype/storm/utils/LocalState.java
+++ b/storm-core/src/jvm/backtype/storm/utils/LocalState.java
@@ -42,7 +42,7 @@
  * Every read/write hits disk.
  */
 public class LocalState {
-    public static Logger LOG = LoggerFactory.getLogger(LocalState.class);
+    public static final Logger LOG = LoggerFactory.getLogger(LocalState.class);
     private VersionedStore _vs;
     
     public LocalState(String backingDir) throws IOException {
@@ -65,7 +65,7 @@
     }
 
     private Map<String, TBase> deserializeLatestVersion() throws IOException {
-        Map<String, TBase> result = new HashMap<String, TBase>();
+        Map<String, TBase> result = new HashMap<>();
         TDeserializer td = new TDeserializer();
         for (Map.Entry<String, ThriftSerializedObject> ent: partialDeserializeLatestVersion(td).entrySet()) {
             result.put(ent.getKey(), deserialize(ent.getValue(), td));
@@ -87,7 +87,7 @@
     private Map<String, ThriftSerializedObject> partialDeserializeLatestVersion(TDeserializer td) {
         try {
             String latestPath = _vs.mostRecentVersionPath();
-            Map<String, ThriftSerializedObject> result = new HashMap<String, ThriftSerializedObject>();
+            Map<String, ThriftSerializedObject> result = new HashMap<>();
             if (latestPath != null) {
                 byte[] serialized = FileUtils.readFileToByteArray(new File(latestPath));
                 if (serialized.length == 0) {
@@ -181,21 +181,7 @@
 
     private ThriftSerializedObject serialize(TBase o, TSerializer ser) {
         try {
-            return new ThriftSerializedObject(o.getClass().getName(), ByteBuffer.wrap(ser.serialize((TBase)o)));
-        } catch(Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private void persist(Map<String, TBase> val, boolean cleanup) {
-        try {
-            TSerializer ser = new TSerializer();
-            Map<String, ThriftSerializedObject> serialized = new HashMap<String, ThriftSerializedObject>();
-            for (Map.Entry<String, TBase> ent: val.entrySet()) {
-                Object o = ent.getValue();
-                serialized.put(ent.getKey(), serialize(ent.getValue(), ser));
-            }
-            persistInternal(serialized, ser, cleanup);
+            return new ThriftSerializedObject(o.getClass().getName(), ByteBuffer.wrap(ser.serialize(o)));
         } catch(Exception e) {
             throw new RuntimeException(e);
         }
diff --git a/storm-core/src/jvm/backtype/storm/utils/Monitor.java b/storm-core/src/jvm/backtype/storm/utils/Monitor.java
index 36fedc4..41717a9 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Monitor.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Monitor.java
@@ -20,7 +20,6 @@
 import backtype.storm.generated.*;
 
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 
 public class Monitor {
@@ -98,7 +97,7 @@
     }
 
     private HashSet<String> getComponents(Nimbus.Client client, String topology) throws Exception{
-        HashSet<String> components = new HashSet<String>();
+        HashSet<String> components = new HashSet<>();
         ClusterSummary clusterSummary = client.getClusterInfo();
         TopologySummary topologySummary = null;
         for (TopologySummary ts: clusterSummary.get_topologies()) {
diff --git a/storm-core/src/jvm/backtype/storm/utils/MutableObject.java b/storm-core/src/jvm/backtype/storm/utils/MutableObject.java
index d5cb7db..2bd9bb1 100644
--- a/storm-core/src/jvm/backtype/storm/utils/MutableObject.java
+++ b/storm-core/src/jvm/backtype/storm/utils/MutableObject.java
@@ -18,7 +18,7 @@
 package backtype.storm.utils;
 
 public class MutableObject {
-    Object o = null;
+    private Object o = null;
     
     public MutableObject() {
         
@@ -28,11 +28,11 @@
         this.o = o;
     }
     
-    public void setObject(Object o) {
+    public synchronized void setObject(Object o) {
         this.o = o;
     }
     
-    public Object getObject() {
+    public synchronized Object getObject() {
         return o;
     }
 }
diff --git a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
index a715de2..a9306f7 100644
--- a/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -22,13 +22,8 @@
 import backtype.storm.generated.ClusterSummary;
 import backtype.storm.generated.Nimbus;
 import backtype.storm.generated.NimbusSummary;
-import backtype.storm.nimbus.ILeaderElector;
-import backtype.storm.nimbus.NimbusInfo;
 import backtype.storm.security.auth.ThriftClient;
 import backtype.storm.security.auth.ThriftConnectionType;
-import clojure.lang.IFn;
-import clojure.lang.PersistentArrayMap;
-import com.google.common.base.Splitter;
 import com.google.common.collect.Lists;
 import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
@@ -38,7 +33,6 @@
 import java.util.Map;
 
 public class NimbusClient extends ThriftClient {
-    public static final String DELIMITER = ":";
     private Nimbus.Client _client;
     private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class);
 
@@ -56,7 +50,7 @@
             asUser = (String) conf.get(Config.STORM_DO_AS_USER);
         }
 
-        List<String> seeds = null;
+        List<String> seeds;
         if(conf.containsKey(Config.NIMBUS_HOST)) {
             LOG.warn("Using deprecated config {} for backward compatibility. Please update your storm.yaml so it only has config {}",
                     Config.NIMBUS_HOST, Config.NIMBUS_SEEDS);
@@ -67,7 +61,7 @@
 
         for (String host : seeds) {
             int port = Integer.parseInt(conf.get(Config.NIMBUS_THRIFT_PORT).toString());
-            ClusterSummary clusterInfo = null;
+            ClusterSummary clusterInfo;
             try {
                 NimbusClient client = new NimbusClient(conf, host, port);
                 clusterInfo = client.getClient().getClusterInfo();
diff --git a/storm-core/src/jvm/backtype/storm/utils/RateTracker.java b/storm-core/src/jvm/backtype/storm/utils/RateTracker.java
deleted file mode 100644
index f937b1f..0000000
--- a/storm-core/src/jvm/backtype/storm/utils/RateTracker.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <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.utils;
-
-import java.util.*;
-
-/**
- * This class is a utility to track the rate.
- */
-public class RateTracker{
-    /* number of slides to keep in the history. */
-    public final int _numOfSlides; // number of slides to keep in the history
-
-    public final int _slideSizeInMils;
-    private final long[] _histograms;// an array storing the number of element for each time slide.
-
-    private int _currentValidSlideNum;
-
-    private static Timer _timer = new Timer();
-
-    /**
-     * @param validTimeWindowInMils events that happened before validTimeWindowInMils are not considered
-     *                        when reporting the rate.
-     * @param numOfSlides the number of time sildes to divide validTimeWindows. The more slides,
-     *                    the smother the reported results will be.
-     */
-    public RateTracker(int validTimeWindowInMils, int numOfSlides) {
-        this(validTimeWindowInMils, numOfSlides, false);
-    }
-
-    /**
-     * Constructor
-     * @param validTimeWindowInMils events that happened before validTimeWindow are not considered
-     *                        when reporting the rate.
-     * @param numOfSlides the number of time sildes to divide validTimeWindows. The more slides,
-     *                    the smother the reported results will be.
-     * @param simulate set true if it use simulated time rather than system time for testing purpose.
-     */
-    public RateTracker(int validTimeWindowInMils, int numOfSlides, boolean simulate ){
-        _numOfSlides = Math.max(numOfSlides, 1);
-        _slideSizeInMils = validTimeWindowInMils / _numOfSlides;
-        if (_slideSizeInMils < 1 ) {
-            throw new IllegalArgumentException("Illeggal argument for RateTracker");
-        }
-        assert(_slideSizeInMils > 1);
-        _histograms = new long[_numOfSlides];
-        Arrays.fill(_histograms,0L);
-        if(!simulate) {
-            _timer.scheduleAtFixedRate(new Fresher(), _slideSizeInMils, _slideSizeInMils);
-        }
-        _currentValidSlideNum = 1;
-    }
-
-    /**
-     * Notify the tracker upon new arrivals
-     *
-     * @param count number of arrivals
-     */
-    public void notify(long count) {
-        _histograms[_histograms.length-1]+=count;
-    }
-
-    /**
-     * Return the average rate in slides.
-     *
-     * @return the average rate
-     */
-    public final float reportRate() {
-        long sum = 0;
-        long duration = _currentValidSlideNum * _slideSizeInMils;
-        for(int i=_numOfSlides - _currentValidSlideNum; i < _numOfSlides; i++ ){
-            sum += _histograms[i];
-        }
-
-        return sum / (float) duration * 1000;
-    }
-
-    public final void forceUpdateSlides(int numToEclipse) {
-
-        for(int i=0; i< numToEclipse; i++) {
-            updateSlides();
-        }
-
-    }
-
-    private void updateSlides(){
-
-        for (int i = 0; i < _numOfSlides - 1; i++) {
-            _histograms[i] = _histograms[i + 1];
-        }
-
-        _histograms[_histograms.length - 1] = 0;
-
-        _currentValidSlideNum = Math.min(_currentValidSlideNum + 1, _numOfSlides);
-    }
-
-    private class Fresher extends TimerTask {
-        public void run () {
-            updateSlides();
-        }
-    }
-
-
-}
diff --git a/storm-core/src/jvm/backtype/storm/utils/RegisteredGlobalState.java b/storm-core/src/jvm/backtype/storm/utils/RegisteredGlobalState.java
index 48053fc..159a7c1 100644
--- a/storm-core/src/jvm/backtype/storm/utils/RegisteredGlobalState.java
+++ b/storm-core/src/jvm/backtype/storm/utils/RegisteredGlobalState.java
@@ -27,7 +27,7 @@
  * tuples.
  */
 public class RegisteredGlobalState {
-    private static HashMap<String, Object> _states = new HashMap<String, Object>();
+    private static HashMap<String, Object> _states = new HashMap<>();
     private static final Object _lock = new Object();
     
     public static Object globalLock() {
@@ -50,9 +50,7 @@
     
     public static Object getState(String id) {
         synchronized(_lock) {
-            Object ret = _states.get(id);
-            //System.out.println("State: " + ret.toString());
-            return ret;
+            return _states.get(id);
         }        
     }
     
diff --git a/storm-core/src/jvm/backtype/storm/utils/RotatingMap.java b/storm-core/src/jvm/backtype/storm/utils/RotatingMap.java
index 2b8d66b..03d1bf8 100644
--- a/storm-core/src/jvm/backtype/storm/utils/RotatingMap.java
+++ b/storm-core/src/jvm/backtype/storm/utils/RotatingMap.java
@@ -53,7 +53,7 @@
         if(numBuckets<2) {
             throw new IllegalArgumentException("numBuckets must be >= 2");
         }
-        _buckets = new LinkedList<HashMap<K, V>>();
+        _buckets = new LinkedList<>();
         for(int i=0; i<numBuckets; i++) {
             _buckets.add(new HashMap<K, V>());
         }
diff --git a/storm-core/src/jvm/backtype/storm/utils/ServiceRegistry.java b/storm-core/src/jvm/backtype/storm/utils/ServiceRegistry.java
index 724bc3e..d5c2eca 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ServiceRegistry.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ServiceRegistry.java
@@ -22,7 +22,7 @@
 
 // this class should be combined with RegisteredGlobalState
 public class ServiceRegistry {
-    private static HashMap<String, Object> _services = new HashMap<String, Object>();
+    private static HashMap<String, Object> _services = new HashMap<>();
     private static final Object _lock = new Object();
     
     public static String registerService(Object service) {
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
index d1b2800..0b2d34e 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellProcess.java
@@ -36,12 +36,12 @@
 import org.slf4j.LoggerFactory;
 
 public class ShellProcess implements Serializable {
-    public static Logger LOG = LoggerFactory.getLogger(ShellProcess.class);
+    public static final Logger LOG = LoggerFactory.getLogger(ShellProcess.class);
     public static Logger ShellLogger;
     private Process      _subprocess;
     private InputStream  processErrorStream;
     private String[]     command;
-    private Map<String, String> env = new HashMap<String, String>();
+    private Map<String, String> env = new HashMap<>();
     public ISerializer   serializer;
     public Number pid;
     public String componentName;
@@ -98,7 +98,7 @@
         String serializer_className = (String)conf.get(Config.TOPOLOGY_MULTILANG_SERIALIZER);
         LOG.info("Storm multilang serializer: " + serializer_className);
 
-        ISerializer serializer = null;
+        ISerializer serializer;
         try {
             //create a factory class
             Class klass = Class.forName(serializer_className);
diff --git a/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java b/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java
index 1065ff9..8595b71 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ShellUtils.java
@@ -34,7 +34,7 @@
 
 
 abstract public class ShellUtils {
-    public static Logger LOG = LoggerFactory.getLogger(ShellUtils.class);
+    public static final Logger LOG = LoggerFactory.getLogger(ShellUtils.class);
 
     // OSType detection
     public enum OSType {
diff --git a/storm-core/src/jvm/backtype/storm/utils/StormBoundedExponentialBackoffRetry.java b/storm-core/src/jvm/backtype/storm/utils/StormBoundedExponentialBackoffRetry.java
index 5422c2e..be6f0e9 100644
--- a/storm-core/src/jvm/backtype/storm/utils/StormBoundedExponentialBackoffRetry.java
+++ b/storm-core/src/jvm/backtype/storm/utils/StormBoundedExponentialBackoffRetry.java
@@ -42,8 +42,9 @@
     public StormBoundedExponentialBackoffRetry(int baseSleepTimeMs, int maxSleepTimeMs, int maxRetries) {
         super(baseSleepTimeMs, maxSleepTimeMs, maxRetries);
         expRetriesThreshold = 1;
-        while ((1 << (expRetriesThreshold + 1)) < ((maxSleepTimeMs - baseSleepTimeMs) / 2))
+        while ((1 << (expRetriesThreshold + 1)) < ((maxSleepTimeMs - baseSleepTimeMs) / 2)) {
             expRetriesThreshold++;
+        }
         LOG.debug("The baseSleepTimeMs [{}] the maxSleepTimeMs [{}] the maxRetries [{}]", 
 				baseSleepTimeMs, maxSleepTimeMs, maxRetries);
         if (baseSleepTimeMs > maxSleepTimeMs) {
diff --git a/storm-core/src/jvm/backtype/storm/utils/ThriftTopologyUtils.java b/storm-core/src/jvm/backtype/storm/utils/ThriftTopologyUtils.java
index 8306d9b..d5c460f 100644
--- a/storm-core/src/jvm/backtype/storm/utils/ThriftTopologyUtils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/ThriftTopologyUtils.java
@@ -27,30 +27,38 @@
 import java.util.Set;
 
 public class ThriftTopologyUtils {
+    public static boolean isWorkerHook(StormTopology._Fields f) {
+        return f.equals(StormTopology._Fields.WORKER_HOOKS);
+    }
+
     public static Set<String> getComponentIds(StormTopology topology) {
         Set<String> ret = new HashSet<String>();
         for(StormTopology._Fields f: StormTopology.metaDataMap.keySet()) {
-            Map<String, Object> componentMap = (Map<String, Object>) topology.getFieldValue(f);
-            ret.addAll(componentMap.keySet());
+            if(StormTopology.metaDataMap.get(f).valueMetaData.type == org.apache.thrift.protocol.TType.MAP) {
+                Map<String, Object> componentMap = (Map<String, Object>) topology.getFieldValue(f);
+                ret.addAll(componentMap.keySet());
+            }
         }
         return ret;
     }
 
     public static ComponentCommon getComponentCommon(StormTopology topology, String componentId) {
         for(StormTopology._Fields f: StormTopology.metaDataMap.keySet()) {
-            Map<String, Object> componentMap = (Map<String, Object>) topology.getFieldValue(f);
-            if(componentMap.containsKey(componentId)) {
-                Object component = componentMap.get(componentId);
-                if(component instanceof Bolt) {
-                    return ((Bolt) component).get_common();
+            if(StormTopology.metaDataMap.get(f).valueMetaData.type == org.apache.thrift.protocol.TType.MAP) {
+                Map<String, Object> componentMap = (Map<String, Object>) topology.getFieldValue(f);
+                if(componentMap.containsKey(componentId)) {
+                    Object component = componentMap.get(componentId);
+                    if(component instanceof Bolt) {
+                        return ((Bolt) component).get_common();
+                    }
+                    if(component instanceof SpoutSpec) {
+                        return ((SpoutSpec) component).get_common();
+                    }
+                    if(component instanceof StateSpoutSpec) {
+                        return ((StateSpoutSpec) component).get_common();
+                    }
+                    throw new RuntimeException("Unreachable code! No get_common conversion for component " + component);
                 }
-                if(component instanceof SpoutSpec) {
-                    return ((SpoutSpec) component).get_common();
-                }
-                if(component instanceof StateSpoutSpec) {
-                    return ((StateSpoutSpec) component).get_common();
-                }
-                throw new RuntimeException("Unreachable code! No get_common conversion for component " + component);
             }
         }
         throw new IllegalArgumentException("Could not find component common for " + componentId);
diff --git a/storm-core/src/jvm/backtype/storm/utils/Time.java b/storm-core/src/jvm/backtype/storm/utils/Time.java
index 50a79fd..8aa871b 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Time.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Time.java
@@ -26,7 +26,7 @@
 
 
 public class Time {
-    public static Logger LOG = LoggerFactory.getLogger(Time.class);    
+    public static final Logger LOG = LoggerFactory.getLogger(Time.class);
     
     private static AtomicBoolean simulating = new AtomicBoolean(false);
     //TODO: should probably use weak references here or something
@@ -39,7 +39,7 @@
         synchronized(sleepTimesLock) {
             simulating.set(true);
             simulatedCurrTimeMs = new AtomicLong(0);
-            threadSleepTimes = new ConcurrentHashMap<Thread, AtomicLong>();
+            threadSleepTimes = new ConcurrentHashMap<>();
         }
     }
     
@@ -58,14 +58,24 @@
         if(simulating.get()) {
             try {
                 synchronized(sleepTimesLock) {
+                    if (threadSleepTimes == null) {
+                        LOG.debug("{} is still sleeping after simulated time disabled.", Thread.currentThread(), new RuntimeException("STACK TRACE"));
+                        throw new InterruptedException();
+                    }
                     threadSleepTimes.put(Thread.currentThread(), new AtomicLong(targetTimeMs));
                 }
                 while(simulatedCurrTimeMs.get() < targetTimeMs) {
+                    synchronized(sleepTimesLock) {
+                        if (threadSleepTimes == null) {
+                            LOG.debug("{} is still sleeping after simulated time disabled.", Thread.currentThread(), new RuntimeException("STACK TRACE"));
+                            throw new InterruptedException();
+                        }
+                    }
                     Thread.sleep(10);
                 }
             } finally {
                 synchronized(sleepTimesLock) {
-                    if (simulating.get()) {
+                    if (simulating.get() && threadSleepTimes != null) {
                         threadSleepTimes.remove(Thread.currentThread());
                     }
                 }
diff --git a/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java b/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
index 5a111ce..feb0328 100644
--- a/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
+++ b/storm-core/src/jvm/backtype/storm/utils/TransferDrainer.java
@@ -20,6 +20,7 @@
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.Map;
 
 import backtype.storm.messaging.IConnection;
 import backtype.storm.messaging.TaskMessage;
@@ -30,18 +31,19 @@
   private HashMap<Integer, ArrayList<ArrayList<TaskMessage>>> bundles = new HashMap();
   
   public void add(HashMap<Integer, ArrayList<TaskMessage>> taskTupleSetMap) {
-    for (Integer task : taskTupleSetMap.keySet()) {
-      addListRefToMap(this.bundles, task, taskTupleSetMap.get(task));
+    for (Map.Entry<Integer, ArrayList<TaskMessage>> entry : taskTupleSetMap.entrySet()) {
+      addListRefToMap(this.bundles, entry.getKey(), entry.getValue());
     }
   }
   
   public void send(HashMap<Integer, String> taskToNode, HashMap<String, IConnection> connections) {
     HashMap<String, ArrayList<ArrayList<TaskMessage>>> bundleMapByDestination = groupBundleByDestination(taskToNode);
 
-    for (String hostPort : bundleMapByDestination.keySet()) {
+    for (Map.Entry<String, ArrayList<ArrayList<TaskMessage>>> entry : bundleMapByDestination.entrySet()) {
+      String hostPort = entry.getKey();
       IConnection connection = connections.get(hostPort);
       if (null != connection) {
-        ArrayList<ArrayList<TaskMessage>> bundle = bundleMapByDestination.get(hostPort);
+        ArrayList<ArrayList<TaskMessage>> bundle = entry.getValue();
         Iterator<TaskMessage> iter = getBundleIterator(bundle);
         if (null != iter && iter.hasNext()) {
           connection.send(iter);
@@ -98,15 +100,12 @@
       
       @Override
       public boolean hasNext() {
-        if (offset < size) {
-          return true;
-        }
-        return false;
+          return offset < size;
       }
 
       @Override
       public TaskMessage next() {
-        TaskMessage msg = null;
+        TaskMessage msg;
         if (iter.hasNext()) {
           msg = iter.next(); 
         } else {
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index b80aa11..47d2332 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -42,7 +42,6 @@
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.SafeConstructor;
 
-import java.io.*;
 import java.net.URL;
 import java.net.URLDecoder;
 import java.nio.ByteBuffer;
@@ -50,13 +49,44 @@
 import java.nio.channels.WritableByteChannel;
 import java.io.File;
 import java.io.FileInputStream;
-import java.util.*;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ByteArrayInputStream;
+import java.io.OutputStreamWriter;
+import java.io.InputStreamReader;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.FileOutputStream;
+import java.io.BufferedReader;
+import java.io.RandomAccessFile;
+import java.io.Serializable;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Iterator;
+import java.util.Enumeration;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+import org.apache.thrift.TBase;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TSerializer;
 
 public class Utils {
     private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
     public static final String DEFAULT_STREAM_ID = "default";
+    private static ThreadLocal<TSerializer> threadSer = new ThreadLocal<TSerializer>();
+    private static ThreadLocal<TDeserializer> threadDes = new ThreadLocal<TDeserializer>();
 
     private static SerializationDelegate serializationDelegate;
     private static ClassLoader cl = ClassLoader.getSystemClassLoader();
@@ -83,6 +113,51 @@
         return serializationDelegate.deserialize(serialized, clazz);
     }
 
+    public static byte[] thriftSerialize(TBase t) {
+        try {
+            TSerializer ser = threadSer.get();
+            if (ser == null) {
+                ser = new TSerializer();
+                threadSer.set(ser);
+            } 
+            return ser.serialize(t);
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static TDeserializer getDes() {
+        TDeserializer des = threadDes.get();
+        if(des == null) {
+            des = new TDeserializer();
+            threadDes.set(des);
+        }
+        return des;
+    }
+
+    public static <T> T thriftDeserialize(Class c, byte[] b, int offset, int length) {
+        try {
+            T ret = (T) c.newInstance();
+            TDeserializer des = getDes();
+            des.deserialize((TBase)ret, b, offset, length);
+            return ret;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> T thriftDeserialize(Class c, byte[] b) {
+        try {
+            T ret = (T) c.newInstance();
+            TDeserializer des = getDes();
+            des.deserialize((TBase) ret, b);
+            return ret;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+
+    }
+    
     public static byte[] javaSerialize(Object obj) {
         try {
             ByteArrayOutputStream bos = new ByteArrayOutputStream();
@@ -102,9 +177,9 @@
             Object ret = ois.readObject();
             ois.close();
             return (T)ret;
-        } catch(IOException ioe) {
+        } catch (IOException ioe) {
             throw new RuntimeException(ioe);
-        } catch(ClassNotFoundException e) {
+        } catch (ClassNotFoundException e) {
             throw new RuntimeException(e);
         }
     }
@@ -158,23 +233,23 @@
             Object ret = JSONValue.parseWithException(in);
             in.close();
             return (Map<String,Object>)ret;
-        } catch(IOException ioe) {
+        } catch (IOException ioe) {
             throw new RuntimeException(ioe);
-        } catch(ParseException e) {
+        } catch (ParseException e) {
             throw new RuntimeException(e);
         }
     }
 
     public static <T> String join(Iterable<T> coll, String sep) {
         Iterator<T> it = coll.iterator();
-        String ret = "";
+        StringBuilder ret = new StringBuilder();
         while(it.hasNext()) {
-            ret = ret + it.next();
+            ret.append(it.next());
             if(it.hasNext()) {
-                ret = ret + sep;
+                ret.append(sep);
             }
         }
-        return ret;
+        return ret.toString();
     }
 
     public static void sleep(long millis) {
@@ -189,7 +264,7 @@
         try {
             Enumeration<URL> resources = Thread.currentThread().getContextClassLoader().getResources(name);
             List<URL> ret = new ArrayList<URL>();
-            while(resources.hasMoreElements()) {
+            while (resources.hasMoreElements()) {
                 ret.add(resources.nextElement());
             }
             return ret;
@@ -253,7 +328,7 @@
                             + " resources. You're probably bundling the Storm jars with your topology jar. "
                             + resources);
         } else {
-            LOG.info("Using "+configFilePath+" from resources");
+            LOG.debug("Using "+configFilePath+" from resources");
             URL resource = resources.iterator().next();
             return resource.openStream();
         }
@@ -262,7 +337,7 @@
 
 
     public static Map findAndReadConfigFile(String name) {
-       return findAndReadConfigFile(name, true);
+        return findAndReadConfigFile(name, true);
     }
 
     public static Map readDefaultConfig() {
@@ -272,7 +347,7 @@
     public static Map readCommandLineOpts() {
         Map ret = new HashMap();
         String commandOptions = System.getProperty("storm.options");
-        if(commandOptions != null) {
+        if (commandOptions != null) {
             String[] configs = commandOptions.split(",");
             for (String config : configs) {
                 config = URLDecoder.decode(config);
@@ -295,7 +370,7 @@
         Map ret = readDefaultConfig();
         String confFile = System.getProperty("storm.conf.file");
         Map storm;
-        if (confFile==null || confFile.equals("")) {
+        if (confFile == null || confFile.equals("")) {
             storm = findAndReadConfigFile("storm.yaml", false);
         } else {
             storm = findAndReadConfigFile(confFile, true);
@@ -306,24 +381,23 @@
     }
 
     private static Object normalizeConf(Object conf) {
-        if(conf==null) return new HashMap();
-        if(conf instanceof Map) {
-            Map confMap = new HashMap((Map) conf);
-            for(Object key: confMap.keySet()) {
-                Object val = confMap.get(key);
-                confMap.put(key, normalizeConf(val));
+        if (conf == null) return new HashMap();
+        if (conf instanceof Map) {
+            Map<Object, Object> confMap = new HashMap((Map) conf);
+            for (Map.Entry<Object, Object> entry : confMap.entrySet()) {
+                confMap.put(entry.getKey(), normalizeConf(entry.getValue()));
             }
             return confMap;
-        } else if(conf instanceof List) {
+        } else if (conf instanceof List) {
             List confList =  new ArrayList((List) conf);
-            for(int i=0; i<confList.size(); i++) {
+            for (int i = 0; i < confList.size(); i++) {
                 Object val = confList.get(i);
                 confList.set(i, normalizeConf(val));
             }
             return confList;
         } else if (conf instanceof Integer) {
             return ((Integer) conf).longValue();
-        } else if(conf instanceof Float) {
+        } else if (conf instanceof Float) {
             return ((Float) conf).doubleValue();
         } else {
             return conf;
@@ -335,9 +409,9 @@
     }
 
     public static Object getSetComponentObject(ComponentObject obj) {
-        if(obj.getSetField()==ComponentObject._Fields.SERIALIZED_JAVA) {
+        if (obj.getSetField() == ComponentObject._Fields.SERIALIZED_JAVA) {
             return Utils.javaDeserialize(obj.get_serialized_java(), Serializable.class);
-        } else if(obj.getSetField()==ComponentObject._Fields.JAVA_OBJECT) {
+        } else if (obj.getSetField() == ComponentObject._Fields.JAVA_OBJECT) {
             return obj.get_java_object();
         } else {
             return obj.get_shell();
@@ -346,7 +420,7 @@
 
     public static <S, T> T get(Map<S, T> m, S key, T def) {
         T ret = m.get(key);
-        if(ret==null) {
+        if (ret == null) {
             ret = def;
         }
         return ret;
@@ -354,7 +428,7 @@
 
     public static List<Object> tuple(Object... values) {
         List<Object> ret = new ArrayList<Object>();
-        for(Object v: values) {
+        for (Object v : values) {
             ret.add(v);
         }
         return ret;
@@ -363,18 +437,18 @@
     public static void downloadFromMaster(Map conf, String file, String localFile) throws AuthorizationException, IOException, TException {
         NimbusClient client = NimbusClient.getConfiguredClient(conf);
         try {
-        	download(client, file, localFile);
+            download(client, file, localFile);
         } finally {
-        	client.close();
+            client.close();
         }
     }
 
     public static void downloadFromHost(Map conf, String file, String localFile, String host, int port) throws IOException, TException, AuthorizationException {
         NimbusClient client = new NimbusClient (conf, host, port, null);
         try {
-        	download(client, file, localFile);
+            download(client, file, localFile);
         } finally {
-        	client.close();
+            client.close();
         }
     }
 
@@ -382,21 +456,21 @@
         WritableByteChannel out = Channels.newChannel(new FileOutputStream(localFile));
         try {
             String id = client.getClient().beginFileDownload(file);
-	        while(true) {
-	            ByteBuffer chunk = client.getClient().downloadChunk(id);
-	            int written = out.write(chunk);
-	            if(written==0) break;
-	        }
+            while (true) {
+                ByteBuffer chunk = client.getClient().downloadChunk(id);
+                int written = out.write(chunk);
+                if (written == 0) break;
+            }
         } finally {
-        	out.close();
+            out.close();
         }
     }
 
     public static IFn loadClojureFn(String namespace, String name) {
         try {
-          clojure.lang.Compiler.eval(RT.readString("(require '" + namespace + ")"));
+            clojure.lang.Compiler.eval(RT.readString("(require '" + namespace + ")"));
         } catch (Exception e) {
-          //if playing from the repl and defining functions, file won't exist
+            //if playing from the repl and defining functions, file won't exist
         }
         return (IFn) RT.var(namespace, name).deref();
     }
@@ -407,64 +481,93 @@
 
     public static <K, V> Map<V, K> reverseMap(Map<K, V> map) {
         Map<V, K> ret = new HashMap<V, K>();
-        for(K key: map.keySet()) {
-            ret.put(map.get(key), key);
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+            ret.put(entry.getValue(), entry.getKey());
         }
         return ret;
     }
 
     public static ComponentCommon getComponentCommon(StormTopology topology, String id) {
-        if(topology.get_spouts().containsKey(id)) {
+        if (topology.get_spouts().containsKey(id)) {
             return topology.get_spouts().get(id).get_common();
         }
-        if(topology.get_bolts().containsKey(id)) {
+        if (topology.get_bolts().containsKey(id)) {
             return topology.get_bolts().get(id).get_common();
         }
-        if(topology.get_state_spouts().containsKey(id)) {
+        if (topology.get_state_spouts().containsKey(id)) {
             return topology.get_state_spouts().get(id).get_common();
         }
         throw new IllegalArgumentException("Could not find component with id " + id);
     }
 
     public static Integer getInt(Object o) {
-      Integer result = getInt(o, null);
-      if (null == result) {
-        throw new IllegalArgumentException("Don't know how to convert null to int");
-      }
-      return result;
+        Integer result = getInt(o, null);
+        if (null == result) {
+            throw new IllegalArgumentException("Don't know how to convert null to int");
+        }
+        return result;
     }
 
     public static Integer getInt(Object o, Integer defaultValue) {
-      if (null == o) {
-        return defaultValue;
-      }
+        if (null == o) {
+            return defaultValue;
+        }
 
-      if (o instanceof Integer ||
-          o instanceof Short ||
-          o instanceof Byte) {
-          return ((Number) o).intValue();
-      } else if (o instanceof Long) {
-          final long l = (Long) o;
-          if (l <= Integer.MAX_VALUE && l >= Integer.MIN_VALUE) {
-              return (int) l;
-          }
-      } else if (o instanceof String) {
-          return Integer.parseInt((String) o);
-      }
+        if (o instanceof Integer ||
+                o instanceof Short ||
+                o instanceof Byte) {
+            return ((Number) o).intValue();
+        } else if (o instanceof Long) {
+            final long l = (Long) o;
+            if (l <= Integer.MAX_VALUE && l >= Integer.MIN_VALUE) {
+                return (int) l;
+            }
+        } else if (o instanceof String) {
+            return Integer.parseInt((String) o);
+        }
 
-      throw new IllegalArgumentException("Don't know how to convert " + o + " to int");
+        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;
-      }
+        if (null == o) {
+            return defaultValue;
+        }
+        if (o instanceof Boolean) {
+            return (Boolean) o;
+        } else {
+            throw new IllegalArgumentException("Don't know how to convert " + o + " + to boolean");
+        }
+    }
 
-      if(o instanceof Boolean) {
-          return (Boolean) o;
-      } else {
-          throw new IllegalArgumentException("Don't know how to convert " + o + " + to boolean");
-      }
+    public static String getString(Object o, String defaultValue) {
+        if (null == o) {
+            return defaultValue;
+        }
+        if (o instanceof String) {
+            return (String) o;
+        } else {
+            throw new IllegalArgumentException("Don't know how to convert " + o + " + to String");
+        }
     }
 
     public static long secureRandomLong() {
@@ -477,7 +580,7 @@
 
     public static CuratorFramework newCurator(Map conf, List<String> servers, Object port, String root, ZookeeperAuthInfo auth) {
         List<String> serverPorts = new ArrayList<String>();
-        for(String zkServer: (List<String>) servers) {
+        for (String zkServer : (List<String>) servers) {
             serverPorts.add(zkServer + ":" + Utils.getInt(port));
         }
         String zkStr = StringUtils.join(serverPorts, ",") + root;
@@ -491,14 +594,14 @@
     protected static void setupBuilder(CuratorFrameworkFactory.Builder builder, String zkStr, Map conf, ZookeeperAuthInfo auth)
     {
         builder.connectString(zkStr)
-            .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)))
-            .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)))
-            .retryPolicy(new StormBoundedExponentialBackoffRetry(
+                .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT)))
+                .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT)))
+                .retryPolicy(new StormBoundedExponentialBackoffRetry(
                         Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)),
                         Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)),
                         Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES))));
 
-        if(auth!=null && auth.scheme!=null && auth.payload!=null) {
+        if (auth != null && auth.scheme != null && auth.payload != null) {
             builder = builder.authorization(auth.scheme, auth.payload);
         }
     }
@@ -519,28 +622,13 @@
         return ret;
     }
 
-    /**
-     *
-(defn integer-divided [sum num-pieces]
-  (let [base (int (/ sum num-pieces))
-        num-inc (mod sum num-pieces)
-        num-bases (- num-pieces num-inc)]
-    (if (= num-inc 0)
-      {base num-bases}
-      {base num-bases (inc base) num-inc}
-      )))
-     * @param sum
-     * @param numPieces
-     * @return
-     */
-
     public static TreeMap<Integer, Integer> integerDivided(int sum, int numPieces) {
         int base = sum / numPieces;
         int numInc = sum % numPieces;
         int numBases = numPieces - numInc;
         TreeMap<Integer, Integer> ret = new TreeMap<Integer, Integer>();
         ret.put(base, numBases);
-        if(numInc!=0) {
+        if (numInc != 0) {
             ret.put(base+1, numInc);
         }
         return ret;
@@ -556,7 +644,7 @@
         try {
             BufferedReader r = new BufferedReader(new InputStreamReader(in));
             String line = null;
-            while ((line = r.readLine())!= null) {
+            while ((line = r.readLine()) != null) {
                 LOG.info("{}:{}", prefix, line);
             }
         } catch (IOException e) {
@@ -566,8 +654,8 @@
 
     public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) {
         Throwable t = throwable;
-        while(t != null) {
-            if(klass.isInstance(t)) {
+        while (t != null) {
+            if (klass.isInstance(t)) {
                 return true;
             }
             t = t.getCause();
@@ -583,9 +671,9 @@
      */
     public static boolean isZkAuthenticationConfiguredStormServer(Map conf) {
         return null != System.getProperty("java.security.auth.login.config")
-            || (conf != null
+                || (conf != null
                 && conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME) != null
-                && ! ((String)conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME)).isEmpty());
+                && !((String)conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME)).isEmpty());
     }
 
     /**
@@ -596,7 +684,7 @@
     public static boolean isZkAuthenticationConfiguredTopology(Map conf) {
         return (conf != null
                 && conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME) != null
-                && ! ((String)conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME)).isEmpty());
+                && !((String)conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME)).isEmpty());
     }
 
     public static List<ACL> getWorkerACL(Map conf) {
@@ -606,37 +694,37 @@
         }
         String stormZKUser = (String)conf.get(Config.STORM_ZOOKEEPER_SUPERACL);
         if (stormZKUser == null) {
-           throw new IllegalArgumentException("Authentication is enabled but "+Config.STORM_ZOOKEEPER_SUPERACL+" is not set");
+            throw new IllegalArgumentException("Authentication is enabled but "+Config.STORM_ZOOKEEPER_SUPERACL+" is not set");
         }
         String[] split = stormZKUser.split(":",2);
         if (split.length != 2) {
-          throw new IllegalArgumentException(Config.STORM_ZOOKEEPER_SUPERACL+" does not appear to be in the form scheme:acl, i.e. sasl:storm-user");
+            throw new IllegalArgumentException(Config.STORM_ZOOKEEPER_SUPERACL+" does not appear to be in the form scheme:acl, i.e. sasl:storm-user");
         }
         ArrayList<ACL> ret = new ArrayList<ACL>(ZooDefs.Ids.CREATOR_ALL_ACL);
         ret.add(new ACL(ZooDefs.Perms.ALL, new Id(split[0], split[1])));
         return ret;
     }
 
-   public static String threadDump() {
-       final StringBuilder dump = new StringBuilder();
-       final java.lang.management.ThreadMXBean threadMXBean =  java.lang.management.ManagementFactory.getThreadMXBean();
-       final java.lang.management.ThreadInfo[] threadInfos = threadMXBean.getThreadInfo(threadMXBean.getAllThreadIds(), 100);
-       for (java.lang.management.ThreadInfo threadInfo : threadInfos) {
-           dump.append('"');
-           dump.append(threadInfo.getThreadName());
-           dump.append("\" ");
-           final Thread.State state = threadInfo.getThreadState();
-           dump.append("\n   java.lang.Thread.State: ");
-           dump.append(state);
-           final StackTraceElement[] stackTraceElements = threadInfo.getStackTrace();
-           for (final StackTraceElement stackTraceElement : stackTraceElements) {
-               dump.append("\n        at ");
-               dump.append(stackTraceElement);
-           }
-           dump.append("\n\n");
-       }
-       return dump.toString();
-   }
+    public static String threadDump() {
+        final StringBuilder dump = new StringBuilder();
+        final java.lang.management.ThreadMXBean threadMXBean =  java.lang.management.ManagementFactory.getThreadMXBean();
+        final java.lang.management.ThreadInfo[] threadInfos = threadMXBean.getThreadInfo(threadMXBean.getAllThreadIds(), 100);
+        for (java.lang.management.ThreadInfo threadInfo : threadInfos) {
+            dump.append('"');
+            dump.append(threadInfo.getThreadName());
+            dump.append("\" ");
+            final Thread.State state = threadInfo.getThreadState();
+            dump.append("\n   java.lang.Thread.State: ");
+            dump.append(state);
+            final StackTraceElement[] stackTraceElements = threadInfo.getStackTrace();
+            for (final StackTraceElement stackTraceElement : stackTraceElements) {
+                dump.append("\n        at ");
+                dump.append(stackTraceElement);
+            }
+            dump.append("\n\n");
+        }
+        return dump.toString();
+    }
 
     // Assumes caller is synchronizing
     private static SerializationDelegate getSerializationDelegate(Map stormConf) {
@@ -659,30 +747,133 @@
         return delegate;
     }
 
-  public static void handleUncaughtException(Throwable t) {
-    if (t != null && t instanceof Error) {
-      if (t instanceof OutOfMemoryError) {
-        try {
-          System.err.println("Halting due to Out Of Memory Error..." + Thread.currentThread().getName());
-        } catch (Throwable err) {
-          //Again we don't want to exit because of logging issues.
+    public static void handleUncaughtException(Throwable t) {
+        if (t != null && t instanceof Error) {
+            if (t instanceof OutOfMemoryError) {
+                try {
+                    System.err.println("Halting due to Out Of Memory Error..." + Thread.currentThread().getName());
+                } catch (Throwable err) {
+                    //Again we don't want to exit because of logging issues.
+                }
+                Runtime.getRuntime().halt(-1);
+            } else {
+                //Running in daemon mode, we would pass Error to calling thread.
+                throw (Error) t;
+            }
         }
-        Runtime.getRuntime().halt(-1);
-      } else {
-        //Running in daemon mode, we would pass Error to calling thread.
-        throw (Error) t;
-      }
     }
-  }
 
-  @VisibleForTesting
-  public static void setClassLoaderForJavaDeSerialize(ClassLoader cl) {
-    Utils.cl = cl;
-  }
+    /**
+     * Given a File input it will unzip the file in a the unzip directory
+     * passed as the second parameter
+     * @param inFile The zip file as input
+     * @param unzipDir The unzip directory where to unzip the zip file.
+     * @throws IOException
+     */
+    public static void unZip(File inFile, File unzipDir) throws IOException {
+        Enumeration<? extends ZipEntry> entries;
+        ZipFile zipFile = new ZipFile(inFile);
 
-  @VisibleForTesting
-  public static void resetClassLoaderForJavaDeSerialize() {
-    Utils.cl = ClassLoader.getSystemClassLoader();
-  }
+        try {
+            entries = zipFile.entries();
+            while (entries.hasMoreElements()) {
+                ZipEntry entry = entries.nextElement();
+                if (!entry.isDirectory()) {
+                    InputStream in = zipFile.getInputStream(entry);
+                    try {
+                        File file = new File(unzipDir, entry.getName());
+                        if (!file.getParentFile().mkdirs()) {
+                            if (!file.getParentFile().isDirectory()) {
+                                throw new IOException("Mkdirs failed to create " +
+                                        file.getParentFile().toString());
+                            }
+                        }
+                        OutputStream out = new FileOutputStream(file);
+                        try {
+                            byte[] buffer = new byte[8192];
+                            int i;
+                            while ((i = in.read(buffer)) != -1) {
+                                out.write(buffer, 0, i);
+                            }
+                        } finally {
+                            out.close();
+                        }
+                    } finally {
+                        in.close();
+                    }
+                }
+            }
+        } finally {
+            zipFile.close();
+        }
+    }
+
+    /**
+     * Given a zip File input it will return its size
+     * Only works for zip files whose uncompressed size is less than 4 GB,
+     * otherwise returns the size module 2^32, per gzip specifications
+     * @param myFile The zip file as input
+     * @throws IOException
+     * @return zip file size as a long
+     */
+    public static long zipFileSize(File myFile) throws IOException{
+        RandomAccessFile raf = new RandomAccessFile(myFile, "r");
+        raf.seek(raf.length() - 4);
+        long b4 = raf.read();
+        long b3 = raf.read();
+        long b2 = raf.read();
+        long b1 = raf.read();
+        long val = (b1 << 24) | (b2 << 16) + (b3 << 8) + b4;
+        raf.close();
+        return val;
+    }
+
+    public static double zeroIfNaNOrInf(double x) {
+        return (Double.isNaN(x) || Double.isInfinite(x)) ? 0.0 : x;
+    }
+
+    /**
+     * parses the arguments to extract jvm heap memory size in MB.
+     * @param input
+     * @param defaultValue
+     * @return the value of the JVM heap memory setting (in MB) in a java command.
+     */
+    public static Double parseJvmHeapMemByChildOpts(String input, Double defaultValue) {
+        if (input != null) {
+            Pattern optsPattern = Pattern.compile("Xmx[0-9]+[mkgMKG]");
+            Matcher m = optsPattern.matcher(input);
+            String memoryOpts = null;
+            while (m.find()) {
+                memoryOpts = m.group();
+            }
+            if (memoryOpts != null) {
+                int unit = 1;
+                if (memoryOpts.toLowerCase().endsWith("k")) {
+                    unit = 1024;
+                } else if (memoryOpts.toLowerCase().endsWith("m")) {
+                    unit = 1024 * 1024;
+                } else if (memoryOpts.toLowerCase().endsWith("g")) {
+                    unit = 1024 * 1024 * 1024;
+                }
+                memoryOpts = memoryOpts.replaceAll("[a-zA-Z]", "");
+                Double result =  Double.parseDouble(memoryOpts) * unit / 1024.0 / 1024.0;
+                return (result < 1.0) ? 1.0 : result;
+            } else {
+                return defaultValue;
+            }
+        } else {
+            return defaultValue;
+        }
+    }
+
+    @VisibleForTesting
+    public static void setClassLoaderForJavaDeSerialize(ClassLoader cl) {
+        Utils.cl = cl;
+    }
+
+    @VisibleForTesting
+    public static void resetClassLoaderForJavaDeSerialize() {
+        Utils.cl = ClassLoader.getSystemClassLoader();
+    }
 }
 
diff --git a/storm-core/src/jvm/backtype/storm/utils/VersionInfo.java b/storm-core/src/jvm/backtype/storm/utils/VersionInfo.java
index 1740e18..f3586bb 100644
--- a/storm-core/src/jvm/backtype/storm/utils/VersionInfo.java
+++ b/storm-core/src/jvm/backtype/storm/utils/VersionInfo.java
@@ -86,7 +86,7 @@
   }
 
 
-  private static VersionInfo COMMON_VERSION_INFO = new VersionInfo("storm-core");
+  private static final VersionInfo COMMON_VERSION_INFO = new VersionInfo("storm-core");
 
   public static String getVersion() {
     return COMMON_VERSION_INFO._getVersion();
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
new file mode 100644
index 0000000..44ec967
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidation.java
@@ -0,0 +1,646 @@
+/**
+ * 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.validation;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Provides functionality for validating configuration fields.
+ */
+public class ConfigValidation {
+
+    private static final Class CONFIG_CLASS = backtype.storm.Config.class;
+
+    private static final Logger LOG = LoggerFactory.getLogger(ConfigValidation.class);
+
+    public static abstract class Validator {
+        public Validator(Map<String, Object> params) {}
+        public Validator() {}
+        public abstract void validateField(String name, Object o);
+    }
+
+    /**
+     * Validator definitions
+     */
+
+    /**
+     * Validates if an object is not null
+     */
+
+    public static class NotNullValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if (o == null) {
+                throw new IllegalArgumentException("Field " + name + "cannot be null! Actual value: " + o);
+            }
+        }
+    }
+
+    /**
+     * Validates basic types
+     */
+    public static class SimpleTypeValidator extends Validator {
+
+        private Class type;
+
+        public SimpleTypeValidator(Map<String, Object> params) {
+            this.type = (Class) params.get(ConfigValidationAnnotations.ValidatorParams.TYPE);
+        }
+
+        @Override
+        public void validateField(String name, Object o) {
+            validateField(name, this.type, o);
+        }
+
+        public static void validateField(String name, Class type, Object o) {
+            if (o == null) {
+                return;
+            }
+            if (type.isInstance(o)) {
+                return;
+            }
+            throw new IllegalArgumentException("Field " + name + " must be of type " + type + ". Object: " + o + " actual type: " + o.getClass());
+        }
+    }
+
+    public static class StringValidator extends Validator {
+
+        private HashSet<String> acceptedValues = null;
+
+        public StringValidator(){}
+
+        public StringValidator(Map<String, Object> params) {
+
+            this.acceptedValues = new HashSet<String>(Arrays.asList((String[])params.get(ConfigValidationAnnotations.ValidatorParams.ACCEPTED_VALUES)));
+
+            if(this.acceptedValues.isEmpty() || (this.acceptedValues.size() == 1 && this.acceptedValues.contains(""))) {
+                this.acceptedValues = null;
+            }
+        }
+
+        @Override
+        public void validateField(String name, Object o) {
+            SimpleTypeValidator.validateField(name, String.class, o);
+            if(this.acceptedValues != null) {
+                if (!this.acceptedValues.contains((String) o)) {
+                    throw new IllegalArgumentException("Field " + name + " is not an accepted value. Value: " + o + " Accepted values: " + this.acceptedValues);
+                }
+            }
+        }
+    }
+
+    public static class BooleanValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            SimpleTypeValidator.validateField(name, Boolean.class, o);
+        }
+    }
+
+    public static class NumberValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            SimpleTypeValidator.validateField(name, Number.class, o);
+        }
+    }
+
+    public static class DoubleValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            SimpleTypeValidator.validateField(name, Double.class, o);
+        }
+    }
+
+    /**
+     * Validates a Integer.
+     */
+    public static class IntegerValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            validateInteger(name, o);
+        }
+
+        public void validateInteger(String name, Object o) {
+            if (o == null) {
+                return;
+            }
+            final long i;
+            if (o instanceof Number &&
+                    (i = ((Number) o).longValue()) == ((Number) o).doubleValue()) {
+                if (i <= Integer.MAX_VALUE && i >= Integer.MIN_VALUE) {
+                    return;
+                }
+            }
+            throw new IllegalArgumentException("Field " + name + " must be an Integer within type range.");
+        }
+    }
+
+    /**
+     * Validates an entry for ImpersonationAclUser
+     */
+    public static class ImpersonationAclUserEntryValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if (o == null) {
+                return;
+            }
+            ConfigValidationUtils.NestableFieldValidator validator = ConfigValidationUtils.mapFv(ConfigValidationUtils.fv(String.class, false),
+                    ConfigValidationUtils.listFv(String.class, false), false);
+            validator.validateField(name, o);
+            Map<String, List<String>> mapObject = (Map<String, List<String>>) o;
+            if (!mapObject.containsKey("hosts")) {
+                throw new IllegalArgumentException(name + " should contain Map entry with key: hosts");
+            }
+            if (!mapObject.containsKey("groups")) {
+                throw new IllegalArgumentException(name + " should contain Map entry with key: groups");
+            }
+        }
+    }
+
+    /**
+     * validates a list of has no duplicates
+     */
+    public static class NoDuplicateInListValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object field) {
+            if (field == null) {
+                return;
+            }
+            //check if iterable
+            SimpleTypeValidator.validateField(name, Iterable.class, field);
+            HashSet<Object> objectSet = new HashSet<Object>();
+            for (Object o : (Iterable) field) {
+                if (objectSet.contains(o)) {
+                    throw new IllegalArgumentException(name + " should contain no duplicate elements. Duplicated element: " + o);
+                }
+                objectSet.add(o);
+            }
+        }
+    }
+
+    /**
+     * Validates a String or a list of Strings
+     */
+    public static class StringOrStringListValidator extends Validator {
+
+        private ConfigValidationUtils.FieldValidator fv = ConfigValidationUtils.listFv(String.class, false);
+
+        @Override
+        public void validateField(String name, Object o) {
+
+            if (o == null || o instanceof String) {
+                // A null value or a String value is acceptable
+                return;
+            }
+            this.fv.validateField(name, o);
+        }
+    }
+
+    /**
+     * Validates Kryo Registration
+     */
+    public static class KryoRegValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if (o == null) {
+                return;
+            }
+            if (o instanceof Iterable) {
+                for (Object e : (Iterable) o) {
+                    if (e instanceof Map) {
+                        for (Map.Entry<Object, Object> entry : ((Map<Object, Object>) e).entrySet()) {
+                            if (!(entry.getKey() instanceof String) ||
+                                    !(entry.getValue() instanceof String)) {
+                                throw new IllegalArgumentException(
+                                        "Each element of the list " + name + " must be a String or a Map of Strings");
+                            }
+                        }
+                    } else if (!(e instanceof String)) {
+                        throw new IllegalArgumentException(
+                                "Each element of the list " + name + " must be a String or a Map of Strings");
+                    }
+                }
+                return;
+            }
+            throw new IllegalArgumentException(
+                    "Field " + name + " must be an Iterable containing only Strings or Maps of Strings");
+        }
+    }
+
+    /**
+     * Validates if a number is a power of 2
+     */
+    public static class PowerOf2Validator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if (o == null) {
+                return;
+            }
+            final long i;
+            if (o instanceof Number &&
+                    (i = ((Number) o).longValue()) == ((Number) o).doubleValue()) {
+                // Test whether the integer is a power of 2.
+                if (i > 0 && (i & (i - 1)) == 0) {
+                    return;
+                }
+            }
+            throw new IllegalArgumentException("Field " + name + " must be a power of 2.");
+        }
+    }
+
+    /**
+     * Validates each entry in a list
+     */
+    public static class ListEntryTypeValidator extends Validator {
+
+        private Class type;
+
+        public ListEntryTypeValidator(Map<String, Object> params) {
+            this.type = (Class) params.get(ConfigValidationAnnotations.ValidatorParams.TYPE);
+        }
+
+        @Override
+        public void validateField(String name, Object o) {
+            validateField(name, this.type, o);
+        }
+
+        public static void validateField(String name, Class type, Object o) {
+            ConfigValidationUtils.NestableFieldValidator validator = ConfigValidationUtils.listFv(type, false);
+            validator.validateField(name, o);
+        }
+    }
+
+    /**
+     * Validates each entry in a list against a list of custom Validators
+     * Each validator in the list of validators must inherit or be an instance of Validator class
+     */
+    public static class ListEntryCustomValidator extends Validator{
+
+        private Class[] entryValidators;
+
+        public ListEntryCustomValidator(Map<String, Object> params) {
+            this.entryValidators = (Class[]) params.get(ConfigValidationAnnotations.ValidatorParams.ENTRY_VALIDATOR_CLASSES);
+        }
+
+        @Override
+        public void validateField(String name, Object o)  {
+            try {
+                validateField(name, this.entryValidators, o);
+            } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public static void validateField(String name, Class[] validators, Object o) throws IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException {
+            if (o == null) {
+                return;
+            }
+            //check if iterable
+            SimpleTypeValidator.validateField(name, Iterable.class, o);
+            for (Object entry : (Iterable) o) {
+                for (Class validator : validators) {
+                    Object v = validator.getConstructor().newInstance();
+                    if (v instanceof Validator) {
+                        ((Validator) v).validateField(name + " list entry", entry);
+                    } else {
+                        LOG.warn("validator: {} cannot be used in ListEntryCustomValidator.  Individual entry validators must a instance of Validator class", validator.getName());
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * validates each key and value in a map of a certain type
+     */
+    public static class MapEntryTypeValidator extends Validator{
+
+        private Class keyType;
+        private Class valueType;
+
+        public MapEntryTypeValidator(Map<String, Object> params) {
+            this.keyType = (Class) params.get(ConfigValidationAnnotations.ValidatorParams.KEY_TYPE);
+            this.valueType = (Class) params.get(ConfigValidationAnnotations.ValidatorParams.VALUE_TYPE);
+        }
+
+        @Override
+        public void validateField(String name, Object o) {
+            validateField(name, this.keyType, this.valueType, o);
+        }
+
+        public static void validateField(String name, Class keyType, Class valueType, Object o) {
+            ConfigValidationUtils.NestableFieldValidator validator = ConfigValidationUtils.mapFv(keyType, valueType, false);
+            validator.validateField(name, o);
+        }
+    }
+
+    /**
+     * validates each key and each value against the respective arrays of validators
+     */
+    public static class MapEntryCustomValidator extends Validator{
+
+        private Class[] keyValidators;
+        private Class[] valueValidators;
+
+        public MapEntryCustomValidator(Map<String, Object> params) {
+            this.keyValidators = (Class []) params.get(ConfigValidationAnnotations.ValidatorParams.KEY_VALIDATOR_CLASSES);
+            this.valueValidators = (Class []) params.get(ConfigValidationAnnotations.ValidatorParams.VALUE_VALIDATOR_CLASSES);
+        }
+
+        @Override
+        public void validateField(String name, Object o) {
+            try {
+                validateField(name, this.keyValidators, this.valueValidators, o);
+            } catch (IllegalAccessException | InstantiationException | NoSuchMethodException | InvocationTargetException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        public static void validateField(String name, Class[] keyValidators, Class[] valueValidators, Object o) throws IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException {
+            if (o == null) {
+                return;
+            }
+            //check if Map
+            SimpleTypeValidator.validateField(name, Map.class, o);
+            for (Map.Entry<Object, Object> entry : ((Map<Object, Object>) o).entrySet()) {
+                for (Class kv : keyValidators) {
+                    Object keyValidator = kv.getConstructor().newInstance();
+                    if (keyValidator instanceof Validator) {
+                        ((Validator) keyValidator).validateField(name + " Map key", entry.getKey());
+                    } else {
+                        LOG.warn("validator: {} cannot be used in MapEntryCustomValidator to validate keys.  Individual entry validators must a instance of Validator class", kv.getName());
+                    }
+                }
+                for (Class vv : valueValidators) {
+                    Object valueValidator = vv.getConstructor().newInstance();
+                    if (valueValidator instanceof Validator) {
+                        ((Validator) valueValidator).validateField(name + " Map value", entry.getValue());
+                    } else {
+                        LOG.warn("validator: {} cannot be used in MapEntryCustomValidator to validate values.  Individual entry validators must a instance of Validator class", vv.getName());
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Validates a positive number
+     */
+    public static class PositiveNumberValidator extends Validator{
+
+        private boolean includeZero;
+
+        public PositiveNumberValidator() {
+            this.includeZero = false;
+        }
+
+        public PositiveNumberValidator(Map<String, Object> params) {
+            this.includeZero = (boolean) params.get(ConfigValidationAnnotations.ValidatorParams.INCLUDE_ZERO);
+        }
+
+        @Override
+        public void validateField(String name, Object o) {
+            validateField(name, this.includeZero, o);
+        }
+
+        public static void validateField(String name, boolean includeZero, Object o) {
+            if (o == null) {
+                return;
+            }
+            if (o instanceof Number) {
+                if(includeZero) {
+                    if (((Number) o).doubleValue() >= 0.0) {
+                        return;
+                    }
+                } else {
+                    if (((Number) o).doubleValue() > 0.0) {
+                        return;
+                    }
+                }
+            }
+            throw new IllegalArgumentException("Field " + name + " must be a Positive Number");
+        }
+    }
+
+    public static class MetricRegistryValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if(o == null) {
+                return;
+            }
+            SimpleTypeValidator.validateField(name, Map.class, o);
+            if(!((Map) o).containsKey("class") ) {
+                throw new IllegalArgumentException( "Field " + name + " must have map entry with key: class");
+            }
+            if(!((Map) o).containsKey("parallelism.hint") ) {
+                throw new IllegalArgumentException("Field " + name + " must have map entry with key: parallelism.hint");
+            }
+
+            SimpleTypeValidator.validateField(name, String.class, ((Map) o).get("class"));
+            SimpleTypeValidator.validateField(name, Long.class, ((Map) o).get("parallelism.hint"));
+        }
+    }
+
+    public static class PacemakerAuthTypeValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if(o == null) {
+                throw new IllegalArgumentException( "Field " + name + " must be set.");
+            }
+
+            if(o instanceof String &&
+               (((String)o).equals("NONE") ||
+                ((String)o).equals("DIGEST") ||
+                ((String)o).equals("KERBEROS"))) {
+                return;
+            }
+            throw new IllegalArgumentException( "Field " + name + " must be one of \"NONE\", \"DIGEST\", or \"KERBEROS\"");
+        }
+    }
+
+    /**
+     * Methods for validating confs
+     */
+
+    /**
+     * Validates a field given field name as string uses Config.java as the default config class
+     *
+     * @param fieldName provided as a string
+     * @param conf      map of confs
+     */
+    public static void validateField(String fieldName, Map conf) {
+        validateField(fieldName, conf, CONFIG_CLASS);
+    }
+
+    /**
+     * Validates a field given field name as string
+     *
+     * @param fieldName   provided as a string
+     * @param conf        map of confs
+     * @param configClass config class
+     */
+    public static void validateField(String fieldName, Map conf, Class configClass) {
+        Field field = null;
+        try {
+            field = configClass.getField(fieldName);
+        } catch (NoSuchFieldException e) {
+            throw new RuntimeException(e);
+        }
+        validateField(field, conf);
+    }
+
+    /**
+     * Validates a field given field.  Calls correct ValidatorField method based on which fields are
+     * declared for the corresponding annotation.
+     *
+     * @param field field that needs to be validated
+     * @param conf  map of confs
+     */
+    public static void validateField(Field field, Map conf) {
+        Annotation[] annotations = field.getAnnotations();
+        if (annotations.length == 0) {
+            LOG.warn("Field {} does not have validator annotation", field);
+        }
+        try {
+            for (Annotation annotation : annotations) {
+                String type = annotation.annotationType().getName();
+                Class validatorClass = null;
+                Class<?>[] classes = ConfigValidationAnnotations.class.getDeclaredClasses();
+                //check if annotation is one of our
+                for (Class clazz : classes) {
+                    if (clazz.getName().equals(type)) {
+                        validatorClass = clazz;
+                        break;
+                    }
+                }
+                if (validatorClass != null) {
+                    Object v = validatorClass.cast(annotation);
+                    String key = (String) field.get(null);
+                    Class clazz = (Class) validatorClass
+                            .getMethod(ConfigValidationAnnotations.ValidatorParams.VALIDATOR_CLASS).invoke(v);
+                    Validator o = null;
+                    Map<String, Object> params = getParamsFromAnnotation(validatorClass, v);
+                    //two constructor signatures used to initialize validators.
+                    //One constructor takes input a Map of arguments, the other doesn't take any arguments (default constructor)
+                    //If validator has a constructor that takes a Map as an argument call that constructor
+                    if (hasConstructor(clazz, Map.class)) {
+                        o = (Validator) clazz.getConstructor(Map.class).newInstance(params);
+                    }
+                    //If not call default constructor
+                    else {
+                        o = (((Class<Validator>) clazz).newInstance());
+                    }
+                    o.validateField(field.getName(), conf.get(key));
+                }
+            }
+        }  catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Validate all confs in map
+     *
+     * @param conf map of configs
+     */
+    public static void validateFields(Map conf) {
+        validateFields(conf, CONFIG_CLASS);
+    }
+
+    /**
+     * Validate all confs in map
+     *
+     * @param conf        map of configs
+     * @param configClass config class
+     */
+    public static void validateFields(Map conf, Class configClass) {
+        for (Field field : configClass.getFields()) {
+            Object keyObj = null;
+            try {
+                keyObj = field.get(null);
+            } catch (IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+            //make sure that defined key is string in case wrong stuff got put into Config.java
+            if (keyObj instanceof String) {
+                String confKey = (String) keyObj;
+                if (conf.containsKey(confKey)) {
+                    validateField(field, conf);
+                }
+            }
+        }
+    }
+
+    private static Map<String,Object> getParamsFromAnnotation(Class validatorClass, Object v) throws InvocationTargetException, IllegalAccessException {
+        Map<String, Object> params = new HashMap<String, Object>();
+        for(Method method : validatorClass.getDeclaredMethods()) {
+
+            Object value = null;
+            try {
+                value = (Object) method.invoke(v);
+            } catch (IllegalArgumentException ex) {
+                value = null;
+            }
+            if(value != null) {
+                params.put(method.getName(), value);
+            }
+        }
+        return params;
+    }
+
+    private static boolean hasConstructor(Class clazz, Class paramClass) {
+        Class[] classes = {paramClass};
+        try {
+            clazz.getConstructor(classes);
+        } catch (NoSuchMethodException e) {
+            return false;
+        }
+        return true;
+    }
+
+    private static boolean hasMethod(Class clazz, String method) {
+        try {
+            clazz.getMethod(method);
+        } catch (NoSuchMethodException ex) {
+            return false;
+        }
+        return true;
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
new file mode 100644
index 0000000..ed93370
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationAnnotations.java
@@ -0,0 +1,214 @@
+/**
+ * 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.validation;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Note: every annotation interface must have method validatorClass()
+ * For every annotation there must validator class to do the validation
+ * To add another annotation for config validation, add another annotation @interface class.  Implement the corresponding
+ * validator logic in a class in ConfigValidation.  Make sure validateField method in ConfigValidation knows how to use the validator
+ * and which method definition/parameters to pass in based on what fields are in the annotation.  By default, params of annotations
+ * will be passed into a constructor that takes a Map as a parameter.
+ */
+public class ConfigValidationAnnotations {
+    /**
+     * Field names for annotations
+     */
+    public static class ValidatorParams {
+        static final String VALIDATOR_CLASS = "validatorClass";
+        static final String TYPE = "type";
+        static final String ENTRY_VALIDATOR_CLASSES = "entryValidatorClasses";
+        static final String KEY_VALIDATOR_CLASSES = "keyValidatorClasses";
+        static final String VALUE_VALIDATOR_CLASSES = "valueValidatorClasses";
+        static final String KEY_TYPE = "keyType";
+        static final String VALUE_TYPE = "valueType";
+        static final String INCLUDE_ZERO = "includeZero";
+        static final String ACCEPTED_VALUES = "acceptedValues";
+    }
+
+    /**
+     * Validators with fields: validatorClass and type
+     */
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isType {
+        Class validatorClass() default ConfigValidation.SimpleTypeValidator.class;
+
+        Class type();
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isStringList {
+        Class validatorClass() default ConfigValidation.ListEntryTypeValidator.class;
+
+        Class type() default String.class;
+    }
+
+    /**
+     * validates each entry in a list is of a certain type
+     */
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isListEntryType {
+        Class validatorClass() default ConfigValidation.ListEntryTypeValidator.class;
+
+        Class type();
+    }
+
+    /**
+     * Validators with fields: validatorClass
+     */
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isString {
+        Class validatorClass() default ConfigValidation.StringValidator.class;
+        String[] acceptedValues() default "";
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isNumber {
+        Class validatorClass() default ConfigValidation.NumberValidator.class;
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isBoolean {
+        Class validatorClass() default ConfigValidation.BooleanValidator.class;
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isInteger {
+        Class validatorClass() default ConfigValidation.IntegerValidator.class;
+    }
+
+    /**
+     * validates on object is not null
+     */
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface NotNull {
+        Class validatorClass() default ConfigValidation.NotNullValidator.class;
+    }
+
+    /**
+     * validates that there are no duplicates in a list
+     */
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isNoDuplicateInList {
+        Class validatorClass() default ConfigValidation.NoDuplicateInListValidator.class;
+    }
+
+    /**
+     * Validates each entry in a list with a list of validators
+     * Validators with fields: validatorClass and entryValidatorClass
+     */
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isListEntryCustom {
+        Class validatorClass() default ConfigValidation.ListEntryCustomValidator.class;
+
+        Class[] entryValidatorClasses();
+    }
+
+    /**
+     * Validates the type of each key and value in a map
+     * Validator with fields: validatorClass, keyValidatorClass, valueValidatorClass
+     */
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isMapEntryType {
+        Class validatorClass() default ConfigValidation.MapEntryTypeValidator.class;
+
+        Class keyType();
+
+        Class valueType();
+    }
+
+    /**
+     * Validates a each key and value in a Map with a list of validators
+     * Validator with fields: validatorClass, keyValidatorClasses, valueValidatorClasses
+     */
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isMapEntryCustom {
+        Class validatorClass() default ConfigValidation.MapEntryCustomValidator.class;
+
+        Class[] keyValidatorClasses();
+
+        Class[] valueValidatorClasses();
+    }
+
+    /**
+     * checks if a number is positive and whether zero inclusive
+     * Validator with fields: validatorClass, includeZero
+     */
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isPositiveNumber {
+        Class validatorClass() default ConfigValidation.PositiveNumberValidator.class;
+
+        boolean includeZero() default false;
+    }
+
+    /**
+     * Complex/custom type validators
+     */
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isStringOrStringList {
+        Class validatorClass() default ConfigValidation.StringOrStringListValidator.class;
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isKryoReg {
+        Class validatorClass() default ConfigValidation.KryoRegValidator.class;
+    }
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface isPowerOf2 {
+        Class validatorClass() default ConfigValidation.PowerOf2Validator.class;
+    }
+
+    /**
+     * For custom validators
+     */
+
+    @Retention(RetentionPolicy.RUNTIME)
+    @Target(ElementType.FIELD)
+    public @interface CustomValidator {
+        Class validatorClass();
+    }
+}
+
diff --git a/storm-core/src/jvm/backtype/storm/validation/ConfigValidationUtils.java b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationUtils.java
new file mode 100644
index 0000000..d55dd5d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/validation/ConfigValidationUtils.java
@@ -0,0 +1,175 @@
+/**
+ * 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.validation;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class ConfigValidationUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(ConfigValidationUtils.class);
+
+    /**
+     * Declares methods for validating configuration values.
+     */
+    public static interface FieldValidator {
+        /**
+         * Validates the given field.
+         * @param name the name of the field.
+         * @param field The field to be validated.
+         * @throws IllegalArgumentException if the field fails validation.
+         */
+        public void validateField(String name, Object field) throws IllegalArgumentException;
+    }
+
+    /**
+     * Declares a method for validating configuration values that is nestable.
+     */
+    public static abstract class NestableFieldValidator implements FieldValidator {
+        @Override
+        public void validateField(String name, Object field) throws IllegalArgumentException {
+            validateField(null, name, field);
+        }
+
+        /**
+         * Validates the given field.
+         * @param pd describes the parent wrapping this validator.
+         * @param name the name of the field.
+         * @param field The field to be validated.
+         * @throws IllegalArgumentException if the field fails validation.
+         */
+        public abstract void validateField(String pd, String name, Object field) throws IllegalArgumentException;
+    }
+
+    /**
+     * Returns a new NestableFieldValidator for a given class.
+     * @param cls the Class the field should be a type of
+     * @param notNull whether or not a value of null is valid
+     * @return a NestableFieldValidator for that class
+     */
+    public static NestableFieldValidator fv(final Class cls, final boolean notNull) {
+        return new NestableFieldValidator() {
+            @Override
+            public void validateField(String pd, String name, Object field)
+                    throws IllegalArgumentException {
+                if (field == null) {
+                    if (notNull) {
+                        throw new IllegalArgumentException("Field " + name + " must not be null");
+                    } else {
+                        return;
+                    }
+                }
+                if (!cls.isInstance(field)) {
+                    throw new IllegalArgumentException(
+                            pd + name + " must be a " + cls.getName() + ". (" + field + ")");
+                }
+            }
+        };
+    }
+
+    /**
+     * Returns a new NestableFieldValidator for a List of the given Class.
+     * @param cls the Class of elements composing the list
+     * @param notNull whether or not a value of null is valid
+     * @return a NestableFieldValidator for a list of the given class
+     */
+    public static NestableFieldValidator listFv(Class cls, boolean notNull) {
+        return listFv(fv(cls, notNull), notNull);
+    }
+
+    /**
+     * Returns a new NestableFieldValidator for a List where each item is validated by validator.
+     * @param validator used to validate each item in the list
+     * @param notNull whether or not a value of null is valid
+     * @return a NestableFieldValidator for a list with each item validated by a different validator.
+     */
+    public static NestableFieldValidator listFv(final NestableFieldValidator validator,
+                                                final boolean notNull) {
+        return new NestableFieldValidator() {
+            @Override
+            public void validateField(String pd, String name, Object field)
+                    throws IllegalArgumentException {
+
+                if (field == null) {
+                    if (notNull) {
+                        throw new IllegalArgumentException("Field " + name + " must not be null");
+                    } else {
+                        return;
+                    }
+                }
+                if (field instanceof Iterable) {
+                    for (Object e : (Iterable) field) {
+                        validator.validateField(pd + "Each element of the list ", name, e);
+                    }
+                    return;
+                }
+                throw new IllegalArgumentException(
+                        "Field " + name + " must be an Iterable but was " +
+                                ((field == null) ? "null" : ("a " + field.getClass())));
+            }
+        };
+    }
+
+    /**
+     * Returns a new NestableFieldValidator for a Map of key to val.
+     * @param key the Class of keys in the map
+     * @param val the Class of values in the map
+     * @param notNull whether or not a value of null is valid
+     * @return a NestableFieldValidator for a Map of key to val
+     */
+    public static NestableFieldValidator mapFv(Class key, Class val,
+                                               boolean notNull) {
+        return mapFv(fv(key, false), fv(val, false), notNull);
+    }
+
+    /**
+     * Returns a new NestableFieldValidator for a Map.
+     * @param key a validator for the keys in the map
+     * @param val a validator for the values in the map
+     * @param notNull whether or not a value of null is valid
+     * @return a NestableFieldValidator for a Map
+     */
+    public static NestableFieldValidator mapFv(final NestableFieldValidator key,
+                                               final NestableFieldValidator val, final boolean notNull) {
+        return new NestableFieldValidator() {
+            @SuppressWarnings("unchecked")
+            @Override
+            public void validateField(String pd, String name, Object field)
+                    throws IllegalArgumentException {
+                if (field == null) {
+                    if (notNull) {
+                        throw new IllegalArgumentException("Field " + name + " must not be null");
+                    } else {
+                        return;
+                    }
+                }
+                if (field instanceof Map) {
+                    for (Map.Entry<Object, Object> entry : ((Map<Object, Object>) field).entrySet()) {
+                        key.validateField("Each key of the map ", name, entry.getKey());
+                        val.validateField("Each value in the map ", name, entry.getValue());
+                    }
+                    return;
+                }
+                throw new IllegalArgumentException(
+                        "Field " + name + " must be a Map");
+            }
+        };
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/CountEvictionPolicy.java b/storm-core/src/jvm/backtype/storm/windowing/CountEvictionPolicy.java
new file mode 100644
index 0000000..a6779a8
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/CountEvictionPolicy.java
@@ -0,0 +1,68 @@
+/**
+ * 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.windowing;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * An eviction policy that tracks event counts and can
+ * evict based on a threshold count.
+ *
+ * @param <T> the type of event tracked by this policy.
+ */
+public class CountEvictionPolicy<T> implements EvictionPolicy<T> {
+    private final int threshold;
+    private final AtomicInteger currentCount;
+
+    public CountEvictionPolicy(int count) {
+        this.threshold = count;
+        this.currentCount = new AtomicInteger();
+    }
+
+    @Override
+    public boolean evict(Event<T> event) {
+        /*
+         * atomically decrement the count if its greater than threshold and
+         * return if the event should be evicted
+         */
+        while (true) {
+            int curVal = currentCount.get();
+            if (curVal > threshold) {
+                if (currentCount.compareAndSet(curVal, curVal - 1)) {
+                    return true;
+                }
+            } else {
+                break;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public void track(Event<T> event) {
+        currentCount.incrementAndGet();
+    }
+
+    @Override
+    public String toString() {
+        return "CountEvictionPolicy{" +
+                "threshold=" + threshold +
+                ", currentCount=" + currentCount +
+                '}';
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/CountTriggerPolicy.java b/storm-core/src/jvm/backtype/storm/windowing/CountTriggerPolicy.java
new file mode 100644
index 0000000..3b1bf9f
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/CountTriggerPolicy.java
@@ -0,0 +1,63 @@
+/**
+ * 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.windowing;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A trigger that tracks event counts and calls back {@link TriggerHandler#onTrigger()}
+ * when the count threshold is hit.
+ *
+ * @param <T> the type of event tracked by this policy.
+ */
+public class CountTriggerPolicy<T> implements TriggerPolicy<T> {
+    private final int count;
+    private final AtomicInteger currentCount;
+    private final TriggerHandler handler;
+
+    public CountTriggerPolicy(int count, TriggerHandler handler) {
+        this.count = count;
+        this.currentCount = new AtomicInteger();
+        this.handler = handler;
+    }
+
+    @Override
+    public void track(Event<T> event) {
+        if (currentCount.incrementAndGet() >= count) {
+            handler.onTrigger();
+        }
+    }
+
+    @Override
+    public void reset() {
+        currentCount.set(0);
+    }
+
+    @Override
+    public void shutdown() {
+        // NOOP
+    }
+
+    @Override
+    public String toString() {
+        return "CountTriggerPolicy{" +
+                "count=" + count +
+                ", currentCount=" + currentCount +
+                '}';
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/Event.java b/storm-core/src/jvm/backtype/storm/windowing/Event.java
new file mode 100644
index 0000000..4855701
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/Event.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.windowing;
+
+/**
+ * An event is a wrapper object that gets stored in the window.
+ *
+ * @param <T> the type of the object thats wrapped. E.g Tuple
+ */
+interface Event<T> {
+    /**
+     * The event timestamp in millis. This could be the time
+     * when the source generated the tuple or the time
+     * when the tuple was received by a bolt.
+     *
+     * @return the event timestamp in milliseconds.
+     */
+    long getTimestamp();
+
+    /**
+     * Returns the wrapped object, E.g. a tuple
+     *
+     * @return the wrapped object.
+     */
+    T get();
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/EventImpl.java b/storm-core/src/jvm/backtype/storm/windowing/EventImpl.java
new file mode 100644
index 0000000..09c974c
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/EventImpl.java
@@ -0,0 +1,38 @@
+/**
+ * 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.windowing;
+
+class EventImpl<T> implements Event<T> {
+    private final T event;
+    private long ts;
+
+    EventImpl(T event, long ts) {
+        this.event = event;
+        this.ts = ts;
+    }
+
+    @Override
+    public long getTimestamp() {
+        return ts;
+    }
+
+    @Override
+    public T get() {
+        return event;
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/EvictionPolicy.java b/storm-core/src/jvm/backtype/storm/windowing/EvictionPolicy.java
new file mode 100644
index 0000000..8820e92
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/EvictionPolicy.java
@@ -0,0 +1,42 @@
+/**
+ * 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.windowing;
+
+/**
+ * Eviction policy tracks events and decides whether
+ * an event should be evicted from the window or not.
+ *
+ * @param <T> the type of event that is tracked.
+ */
+public interface EvictionPolicy<T> {
+    /**
+     * Decides if an event should be evicted from the window or not.
+     *
+     * @param event the input event
+     * @return true if the event should be evicted, false otherwise
+     */
+    boolean evict(Event<T> event);
+
+    /**
+     * Tracks the event to later decide whether
+     * {@link EvictionPolicy#evict(Event)} should evict it or not.
+     *
+     * @param event the input event to be tracked
+     */
+    void track(Event<T> event);
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/TimeEvictionPolicy.java b/storm-core/src/jvm/backtype/storm/windowing/TimeEvictionPolicy.java
new file mode 100644
index 0000000..16408f3
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/TimeEvictionPolicy.java
@@ -0,0 +1,52 @@
+/**
+ * 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.windowing;
+
+/**
+ * Eviction policy that evicts events based on time duration.
+ */
+public class TimeEvictionPolicy<T> implements EvictionPolicy<T> {
+    private final long duration;
+
+    public TimeEvictionPolicy(long millis) {
+        this.duration = millis;
+    }
+
+    /**
+     * Returns true if the event falls out of the window based on the window duration
+     *
+     * @param event
+     * @return
+     */
+    @Override
+    public boolean evict(Event<T> event) {
+        return (System.currentTimeMillis() - event.getTimestamp()) >= duration;
+    }
+
+    @Override
+    public void track(Event<T> event) {
+        // NOOP
+    }
+
+    @Override
+    public String toString() {
+        return "TimeEvictionPolicy{" +
+                "duration=" + duration +
+                '}';
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/TimeTriggerPolicy.java b/storm-core/src/jvm/backtype/storm/windowing/TimeTriggerPolicy.java
new file mode 100644
index 0000000..db1fbeb
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/TimeTriggerPolicy.java
@@ -0,0 +1,115 @@
+/**
+ * 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.windowing;
+
+import backtype.storm.topology.FailedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Invokes {@link TriggerHandler#onTrigger()} after the duration.
+ */
+public class TimeTriggerPolicy<T> implements TriggerPolicy<T> {
+    private static final Logger LOG = LoggerFactory.getLogger(TimeTriggerPolicy.class);
+
+    private long duration;
+    private final TriggerHandler handler;
+    private final ScheduledExecutorService executor;
+    private final ScheduledFuture<?> executorFuture;
+
+    public TimeTriggerPolicy(long millis, TriggerHandler handler) {
+        this.duration = millis;
+        this.handler = handler;
+        this.executor = Executors.newSingleThreadScheduledExecutor();
+        this.executorFuture = executor.scheduleAtFixedRate(newTriggerTask(), duration, duration, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void track(Event<T> event) {
+        checkFailures();
+    }
+
+    @Override
+    public void reset() {
+        checkFailures();
+    }
+
+    @Override
+    public void shutdown() {
+        executor.shutdown();
+        try {
+            if (!executor.awaitTermination(2, TimeUnit.SECONDS)) {
+                executor.shutdownNow();
+            }
+        } catch (InterruptedException ie) {
+            executor.shutdownNow();
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public String toString() {
+        return "TimeTriggerPolicy{" +
+                "duration=" + duration +
+                '}';
+    }
+
+    /*
+    * Check for uncaught exceptions during the execution
+    * of the trigger and fail fast.
+    * The uncaught exceptions will be wrapped in
+    * ExecutionException and thrown when future.get() is invoked.
+    */
+    private void checkFailures() {
+        if (executorFuture.isDone()) {
+            try {
+                executorFuture.get();
+            } catch (InterruptedException ex) {
+                LOG.error("Got exception ", ex);
+                throw new FailedException(ex);
+            } catch (ExecutionException ex) {
+                LOG.error("Got exception ", ex);
+                throw new FailedException(ex.getCause());
+            }
+        }
+    }
+
+    private Runnable newTriggerTask() {
+        return new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    handler.onTrigger();
+                } catch (Throwable th) {
+                    LOG.error("handler.onTrigger failed ", th);
+                    /*
+                     * propagate it so that task gets canceled and the exception
+                     * can be retrieved from executorFuture.get()
+                     */
+                    throw th;
+                }
+            }
+        };
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/TriggerHandler.java b/storm-core/src/jvm/backtype/storm/windowing/TriggerHandler.java
new file mode 100644
index 0000000..f947951
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/TriggerHandler.java
@@ -0,0 +1,29 @@
+/**
+ * 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.windowing;
+
+/**
+ * The callback fired by {@link TriggerPolicy} when the trigger
+ * condition is satisfied.
+ */
+interface TriggerHandler {
+    /**
+     * the code to execute when the {@link TriggerPolicy} condition is satisfied.
+     */
+    void onTrigger();
+}
\ No newline at end of file
diff --git a/storm-core/src/jvm/backtype/storm/windowing/TriggerPolicy.java b/storm-core/src/jvm/backtype/storm/windowing/TriggerPolicy.java
new file mode 100644
index 0000000..d27af76
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/TriggerPolicy.java
@@ -0,0 +1,42 @@
+/**
+ * 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.windowing;
+
+/**
+ * Triggers the window calculations based on the policy.
+ *
+ * @param <T> the type of the event that is tracked
+ */
+public interface TriggerPolicy<T> {
+    /**
+     * Tracks the event and could use this to invoke the trigger.
+     *
+     * @param event the input event
+     */
+    void track(Event<T> event);
+
+    /**
+     * resets the trigger policy
+     */
+    void reset();
+
+    /**
+     * Any clean up could be handled here.
+     */
+    void shutdown();
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/TupleWindow.java b/storm-core/src/jvm/backtype/storm/windowing/TupleWindow.java
new file mode 100644
index 0000000..cf7408b
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/TupleWindow.java
@@ -0,0 +1,26 @@
+/**
+ * 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.windowing;
+
+import backtype.storm.tuple.Tuple;
+
+/**
+ * A {@link Window} that contains {@link Tuple} objects.
+ */
+public interface TupleWindow extends Window<Tuple> {
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/TupleWindowImpl.java b/storm-core/src/jvm/backtype/storm/windowing/TupleWindowImpl.java
new file mode 100644
index 0000000..56a395f
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/TupleWindowImpl.java
@@ -0,0 +1,61 @@
+/**
+ * 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.windowing;
+
+import backtype.storm.tuple.Tuple;
+
+import java.util.List;
+
+/**
+ * Holds the expired, new and current tuples in a window.
+ */
+public class TupleWindowImpl implements TupleWindow {
+    private final List<Tuple> tuples;
+    private final List<Tuple> newTuples;
+    private final List<Tuple> expiredTuples;
+
+    public TupleWindowImpl(List<Tuple> tuples, List<Tuple> newTuples, List<Tuple> expiredTuples) {
+        this.tuples = tuples;
+        this.newTuples = newTuples;
+        this.expiredTuples = expiredTuples;
+    }
+
+    @Override
+    public List<Tuple> get() {
+        return tuples;
+    }
+
+    @Override
+    public List<Tuple> getNew() {
+        return newTuples;
+    }
+
+    @Override
+    public List<Tuple> getExpired() {
+        return expiredTuples;
+    }
+
+    @Override
+    public String toString() {
+        return "TupleWindowImpl{" +
+                "tuples=" + tuples +
+                ", newTuples=" + newTuples +
+                ", expiredTuples=" + expiredTuples +
+                '}';
+    }
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/Window.java b/storm-core/src/jvm/backtype/storm/windowing/Window.java
new file mode 100644
index 0000000..9505e33
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/Window.java
@@ -0,0 +1,48 @@
+/**
+ * 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.windowing;
+
+import java.util.List;
+
+/**
+ * A view of events in a sliding window.
+ *
+ * @param <T> the type of event that this window contains. E.g. {@link backtype.storm.tuple.Tuple}
+ */
+public interface Window<T> {
+    /**
+     * Gets the list of events in the window.
+     *
+     * @return the list of events in the window.
+     */
+    List<T> get();
+
+    /**
+     * Get the list of newly added events in the window since the last time the window was generated.
+     *
+     * @return the list of newly added events in the window.
+     */
+    List<T> getNew();
+
+    /**
+     * Get the list of events expired from the window since the last time the window was generated.
+     *
+     * @return the list of events expired from the window.
+     */
+    List<T> getExpired();
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/WindowLifecycleListener.java b/storm-core/src/jvm/backtype/storm/windowing/WindowLifecycleListener.java
new file mode 100644
index 0000000..5954003
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/WindowLifecycleListener.java
@@ -0,0 +1,42 @@
+/**
+ * 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.windowing;
+
+import java.util.List;
+
+/**
+ * A callback for expiry, activation of events tracked by the {@link WindowManager}
+ *
+ * @param <T> The type of Event in the window (e.g. Tuple).
+ */
+public interface WindowLifecycleListener<T> {
+    /**
+     * Called on expiry of events from the window due to {@link EvictionPolicy}
+     *
+     * @param events the expired events
+     */
+    void onExpiry(List<T> events);
+
+    /**
+     * Called on activation of the window due to the {@link TriggerPolicy}
+     * @param events the list of current events in the window.
+     * @param newEvents the newly added events since last activation.
+     * @param expired the expired events since last activation.
+     */
+    void onActivation(List<T> events, List<T> newEvents, List<T> expired);
+}
diff --git a/storm-core/src/jvm/backtype/storm/windowing/WindowManager.java b/storm-core/src/jvm/backtype/storm/windowing/WindowManager.java
new file mode 100644
index 0000000..6603abf
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/windowing/WindowManager.java
@@ -0,0 +1,212 @@
+/**
+ * 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.windowing;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static backtype.storm.topology.base.BaseWindowedBolt.Count;
+import static backtype.storm.topology.base.BaseWindowedBolt.Duration;
+
+/**
+ * Tracks a window of events and fires {@link WindowLifecycleListener} callbacks
+ * on expiry of events or activation of the window due to {@link TriggerPolicy}.
+ *
+ * @param <T> the type of event in the window.
+ */
+public class WindowManager<T> implements TriggerHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(WindowManager.class);
+
+    /**
+     * Expire old events every EXPIRE_EVENTS_THRESHOLD to
+     * keep the window size in check.
+     */
+    public static final int EXPIRE_EVENTS_THRESHOLD = 100;
+
+    private final WindowLifecycleListener<T> windowLifecycleListener;
+    private final ConcurrentLinkedQueue<Event<T>> window;
+    private final List<T> expiredEvents;
+    private final Set<Event<T>> prevWindowEvents;
+    private final AtomicInteger eventsSinceLastExpiry;
+    private final ReentrantLock lock;
+    private EvictionPolicy<T> evictionPolicy;
+    private TriggerPolicy<T> triggerPolicy;
+
+    public WindowManager(WindowLifecycleListener<T> lifecycleListener) {
+        windowLifecycleListener = lifecycleListener;
+        window = new ConcurrentLinkedQueue<>();
+        expiredEvents = new ArrayList<>();
+        prevWindowEvents = new HashSet<>();
+        eventsSinceLastExpiry = new AtomicInteger();
+        lock = new ReentrantLock(true);
+    }
+
+    public void setWindowLength(Count count) {
+        this.evictionPolicy = new CountEvictionPolicy<>(count.value);
+    }
+
+    public void setWindowLength(Duration duration) {
+        this.evictionPolicy = new TimeEvictionPolicy<>(duration.value);
+    }
+
+    public void setSlidingInterval(Count count) {
+        this.triggerPolicy = new CountTriggerPolicy<>(count.value, this);
+    }
+
+    public void setSlidingInterval(Duration duration) {
+        this.triggerPolicy = new TimeTriggerPolicy<>(duration.value, this);
+    }
+
+    /**
+     * Add an event into the window, with {@link System#currentTimeMillis()} as
+     * the tracking ts.
+     *
+     * @param event the event to add
+     */
+    public void add(T event) {
+        add(event, System.currentTimeMillis());
+    }
+
+    /**
+     * Add an event into the window, with the given ts as the tracking ts.
+     *
+     * @param event the event to track
+     * @param ts    the timestamp
+     */
+    public void add(T event, long ts) {
+        Event<T> windowEvent = new EventImpl<T>(event, ts);
+        window.add(windowEvent);
+        track(windowEvent);
+        compactWindow();
+    }
+
+    /**
+     * The callback invoked by the trigger policy.
+     */
+    @Override
+    public void onTrigger() {
+        List<Event<T>> windowEvents = null;
+        List<T> expired = null;
+        try {
+            lock.lock();
+            /*
+             * scan the entire window to handle out of order events in
+             * the case of time based windows.
+             */
+            windowEvents = expireEvents(true);
+            expired = new ArrayList<>(expiredEvents);
+            expiredEvents.clear();
+        } finally {
+            lock.unlock();
+        }
+        List<T> events = new ArrayList<>();
+        List<T> newEvents = new ArrayList<>();
+        for (Event<T> event : windowEvents) {
+            events.add(event.get());
+            if (!prevWindowEvents.contains(event)) {
+                newEvents.add(event.get());
+            }
+        }
+        prevWindowEvents.clear();
+        prevWindowEvents.addAll(windowEvents);
+        LOG.debug("invoking windowLifecycleListener onActivation, [{}] events in window.", windowEvents.size());
+        windowLifecycleListener.onActivation(events, newEvents, expired);
+        triggerPolicy.reset();
+    }
+
+    public void shutdown() {
+        LOG.debug("Shutting down WindowManager");
+        if (triggerPolicy != null) {
+            triggerPolicy.shutdown();
+        }
+    }
+
+    /**
+     * expires events that fall out of the window every
+     * EXPIRE_EVENTS_THRESHOLD so that the window does not grow
+     * too big.
+     */
+    private void compactWindow() {
+        if (eventsSinceLastExpiry.incrementAndGet() >= EXPIRE_EVENTS_THRESHOLD) {
+            expireEvents(false);
+        }
+    }
+
+    /**
+     * feed the event to the eviction and trigger policies
+     * for bookkeeping and optionally firing the trigger.
+     */
+    private void track(Event<T> windowEvent) {
+        evictionPolicy.track(windowEvent);
+        triggerPolicy.track(windowEvent);
+    }
+
+    /**
+     * Expire events from the window, using the expiration policy to check
+     * if the event should be evicted or not.
+     *
+     * @param fullScan if set, will scan the entire window; if not set, will stop
+     *                 as soon as an event not satisfying the expiration policy is found
+     * @return the list of remaining events in the window after expiry
+     */
+    private List<Event<T>> expireEvents(boolean fullScan) {
+        LOG.debug("Expire events, eviction policy {}", evictionPolicy);
+        List<T> eventsToExpire = new ArrayList<>();
+        List<Event<T>> remaining = new ArrayList<>();
+        try {
+            lock.lock();
+            Iterator<Event<T>> it = window.iterator();
+            while (it.hasNext()) {
+                Event<T> windowEvent = it.next();
+                if (evictionPolicy.evict(windowEvent)) {
+                    eventsToExpire.add(windowEvent.get());
+                    it.remove();
+                } else if (!fullScan) {
+                    break;
+                } else {
+                    remaining.add(windowEvent);
+                }
+            }
+            expiredEvents.addAll(eventsToExpire);
+        } finally {
+            lock.unlock();
+        }
+        eventsSinceLastExpiry.set(0);
+        LOG.debug("[{}] events expired from window.", eventsToExpire.size());
+        windowLifecycleListener.onExpiry(eventsToExpire);
+        return remaining;
+    }
+
+    @Override
+    public String toString() {
+        return "WindowManager{" +
+                "evictionPolicy=" + evictionPolicy +
+                ", triggerPolicy=" + triggerPolicy +
+                '}';
+    }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/IServerMessageHandler.java b/storm-core/src/jvm/org/apache/storm/pacemaker/IServerMessageHandler.java
new file mode 100644
index 0000000..14dd4ad
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/IServerMessageHandler.java
@@ -0,0 +1,25 @@
+/**
+ * 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 org.apache.storm.pacemaker;
+
+import backtype.storm.generated.HBMessage;
+
+public interface IServerMessageHandler {
+
+    public HBMessage handleMessage(HBMessage m, boolean authenticated);
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
new file mode 100644
index 0000000..4f4a737
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@ -0,0 +1,255 @@
+/**
+ * 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 org.apache.storm.pacemaker;
+
+import backtype.storm.Config;
+import backtype.storm.generated.HBMessage;
+import backtype.storm.messaging.netty.ISaslClient;
+import backtype.storm.messaging.netty.NettyRenameThreadFactory;
+import backtype.storm.security.auth.AuthUtils;
+import backtype.storm.utils.StormBoundedExponentialBackoffRetry;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.security.auth.login.Configuration;
+import org.apache.storm.pacemaker.codec.ThriftNettyClientCodec;
+import org.jboss.netty.bootstrap.ClientBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class PacemakerClient implements ISaslClient {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PacemakerClient.class);
+
+    private String topo_name;
+    private String secret;
+    private boolean ready = false;
+    private final ClientBootstrap bootstrap;
+    private AtomicReference<Channel> channelRef;
+    private AtomicBoolean closing;
+    private InetSocketAddress remote_addr;
+    private int maxPending = 100;
+    private HBMessage messages[];
+    private LinkedBlockingQueue<Integer> availableMessageSlots;
+    private ThriftNettyClientCodec.AuthMethod authMethod;
+
+    private StormBoundedExponentialBackoffRetry backoff = new StormBoundedExponentialBackoffRetry(100, 5000, 20);
+    private int retryTimes = 0;
+
+    public PacemakerClient(Map config) {
+
+        String host = (String)config.get(Config.PACEMAKER_HOST);
+        int port = (int)config.get(Config.PACEMAKER_PORT);
+        topo_name = (String)config.get(Config.TOPOLOGY_NAME);
+        if(topo_name == null) {
+            topo_name = "pacemaker-client";
+        }
+
+        String auth = (String)config.get(Config.PACEMAKER_AUTH_METHOD);
+        ThriftNettyClientCodec.AuthMethod authMethod;
+
+        switch(auth) {
+
+        case "DIGEST":
+            Configuration login_conf = AuthUtils.GetConfiguration(config);
+            authMethod = ThriftNettyClientCodec.AuthMethod.DIGEST;
+            secret = AuthUtils.makeDigestPayload(login_conf, AuthUtils.LOGIN_CONTEXT_PACEMAKER_DIGEST);
+            if(secret == null) {
+                LOG.error("Can't start pacemaker server without digest secret.");
+                throw new RuntimeException("Can't start pacemaker server without digest secret.");
+            }
+            break;
+
+        case "KERBEROS":
+            authMethod = ThriftNettyClientCodec.AuthMethod.KERBEROS;
+            break;
+
+        case "NONE":
+            authMethod = ThriftNettyClientCodec.AuthMethod.NONE;
+            break;
+
+        default:
+            authMethod = ThriftNettyClientCodec.AuthMethod.NONE;
+            LOG.warn("Invalid auth scheme: '{}'. Falling back to 'NONE'", auth);
+            break;
+        }
+
+        closing = new AtomicBoolean(false);
+        channelRef = new AtomicReference<Channel>(null);
+        setupMessaging();
+
+        ThreadFactory bossFactory = new NettyRenameThreadFactory("client-boss");
+        ThreadFactory workerFactory = new NettyRenameThreadFactory("client-worker");
+        NioClientSocketChannelFactory factory =
+            new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
+                                              Executors.newCachedThreadPool(workerFactory));
+        bootstrap = new ClientBootstrap(factory);
+        bootstrap.setOption("tcpNoDelay", true);
+        bootstrap.setOption("sendBufferSize", 5242880);
+        bootstrap.setOption("keepAlive", true);
+
+        remote_addr = new InetSocketAddress(host, port);
+        ChannelPipelineFactory pipelineFactory = new ThriftNettyClientCodec(this, config, authMethod).pipelineFactory();
+        bootstrap.setPipelineFactory(pipelineFactory);
+        bootstrap.connect(remote_addr);
+    }
+
+    private void setupMessaging() {
+        messages = new HBMessage[maxPending];
+        availableMessageSlots = new LinkedBlockingQueue<Integer>();
+        for(int i = 0; i < maxPending; i++) {
+            availableMessageSlots.add(i);
+        }
+    }
+
+    public synchronized void channelConnected(Channel channel) {
+        LOG.debug("Channel is connected: {}", channel.toString());
+        channelRef.set(channel);
+
+        //If we're not going to authenticate, we can begin sending.
+        if(authMethod == ThriftNettyClientCodec.AuthMethod.NONE) {
+            ready = true;
+            this.notifyAll();
+        }
+        retryTimes = 0;
+    }
+
+    public synchronized void channelReady() {
+        LOG.debug("Channel is ready.");
+        ready = true;
+        this.notifyAll();
+    }
+
+    public String name() {
+        return topo_name;
+    }
+
+    public String secretKey() {
+        return secret;
+    }
+
+    public HBMessage send(HBMessage m) {
+        // Wait for 'ready' (channel connected and maybe authentication)
+        if(!ready) {
+            synchronized(this) {
+                if(!ready) {
+                    LOG.debug("Waiting for netty channel to be ready.");
+                    try {
+                        this.wait(1000);
+                        if(!ready) {
+                            throw new RuntimeException("Timed out waiting for channel ready.");
+                        }
+                    } catch (java.lang.InterruptedException e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+            }
+        }
+
+        LOG.debug("Sending message: {}", m.toString());
+        try {
+
+            int next = availableMessageSlots.take();
+            synchronized (m) {
+                m.set_message_id(next);
+                messages[next] = m;
+                LOG.debug("Put message in slot: {}", Integer.toString(next));
+                do {
+                    channelRef.get().write(m);
+                    m.wait(1000);
+                } while (messages[next] == m);
+            }
+
+            HBMessage ret = messages[next];
+            if(ret == null) {
+                // This can happen if we lost the connection and subsequently reconnected or timed out.
+                send(m);
+            }
+            messages[next] = null;
+            LOG.debug("Got Response: {}", ret);
+            return ret;
+        }
+        catch (InterruptedException e) {
+            LOG.error("PacemakerClient send interrupted: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void gotMessage(HBMessage m) {
+        int message_id = m.get_message_id();
+        if(message_id >=0 && message_id < maxPending) {
+
+            LOG.debug("Pacemaker client got message: {}", m.toString());
+            HBMessage request = messages[message_id];
+
+            if(request == null) {
+                LOG.debug("No message for slot: {}", Integer.toString(message_id));
+            }
+            else {
+                synchronized(request) {
+                    messages[message_id] = m;
+                    request.notifyAll();
+                    availableMessageSlots.add(message_id);
+                }
+            }
+        }
+        else {
+            LOG.error("Got Message with bad id: {}", m.toString());
+        }
+    }
+
+    public void reconnect() {
+        final PacemakerClient client = this;
+        Timer t = new Timer(true);
+        t.schedule(new TimerTask() {
+                public void run() {
+                    client.doReconnect();
+                }
+            },
+            backoff.getSleepTimeMs(retryTimes++, 0));
+        ready = false;
+        setupMessaging();
+    }
+
+    public synchronized void doReconnect() {
+        close_channel();
+        if(closing.get()) return;
+        bootstrap.connect(remote_addr);
+    }
+
+    synchronized void close_channel() {
+        if (channelRef.get() != null) {
+            channelRef.get().close();
+            LOG.debug("channel {} closed", remote_addr);
+            channelRef.set(null);
+        }
+    }
+
+    public void close() {
+        close_channel();
+    }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java
new file mode 100644
index 0000000..02cfde6
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClientHandler.java
@@ -0,0 +1,75 @@
+/**
+ * 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 org.apache.storm.pacemaker;
+
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelStateEvent;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.Channel;
+import backtype.storm.generated.HBMessage;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import backtype.storm.messaging.netty.ControlMessage;
+
+public class PacemakerClientHandler extends SimpleChannelUpstreamHandler {
+    private static final Logger LOG = LoggerFactory.getLogger(PacemakerClientHandler.class);
+
+    private PacemakerClient client;
+
+
+
+    public PacemakerClientHandler(PacemakerClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public void channelConnected(ChannelHandlerContext ctx,
+                                 ChannelStateEvent event) {
+        // register the newly established channel
+        Channel channel = ctx.getChannel();
+        client.channelConnected(channel);
+
+        LOG.info("Connection established from {} to {}",
+                 channel.getLocalAddress(), channel.getRemoteAddress());
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) {
+        LOG.debug("Got Message: {}", event.getMessage().toString());
+        Object evm = event.getMessage();
+
+        if(evm instanceof ControlMessage) {
+            LOG.debug("Got control message: {}", evm.toString());
+            return;
+        }
+        else if(evm instanceof HBMessage) {
+            client.gotMessage((HBMessage)evm);
+        }
+        else {
+            LOG.warn("Got unexpected message: {} from server.", evm);
+        }
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) {
+        LOG.error("Connection to pacemaker failed", event.getCause());
+        client.reconnect();
+    }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerServer.java b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerServer.java
new file mode 100644
index 0000000..0422dba
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerServer.java
@@ -0,0 +1,163 @@
+/**
+ * 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 org.apache.storm.pacemaker;
+
+import backtype.storm.Config;
+import backtype.storm.generated.HBMessage;
+import backtype.storm.messaging.netty.ISaslServer;
+import backtype.storm.messaging.netty.NettyRenameThreadFactory;
+import backtype.storm.security.auth.AuthUtils;
+import java.lang.InterruptedException;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import javax.security.auth.login.Configuration;
+import org.apache.storm.pacemaker.codec.ThriftNettyServerCodec;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class PacemakerServer implements ISaslServer {
+
+    private static final long FIVE_MB_IN_BYTES = 5 * 1024 * 1024;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PacemakerServer.class);
+
+    private final ServerBootstrap bootstrap;
+    private int port;
+    private IServerMessageHandler handler;
+    private String secret;
+    private String topo_name;
+    private volatile ChannelGroup allChannels = new DefaultChannelGroup("storm-server");
+    private ConcurrentSkipListSet<Channel> authenticated_channels = new ConcurrentSkipListSet<Channel>();
+    private ThriftNettyServerCodec.AuthMethod authMethod;
+
+    public PacemakerServer(IServerMessageHandler handler, Map config){
+        int maxWorkers = (int)config.get(Config.PACEMAKER_MAX_THREADS);
+        this.port = (int)config.get(Config.PACEMAKER_PORT);
+        this.handler = handler;
+        this.topo_name = "pacemaker_server";
+
+        String auth = (String)config.get(Config.PACEMAKER_AUTH_METHOD);
+        switch(auth) {
+
+        case "DIGEST":
+            Configuration login_conf = AuthUtils.GetConfiguration(config);
+            authMethod = ThriftNettyServerCodec.AuthMethod.DIGEST;
+            this.secret = AuthUtils.makeDigestPayload(login_conf, AuthUtils.LOGIN_CONTEXT_PACEMAKER_DIGEST);
+            if(this.secret == null) {
+                LOG.error("Can't start pacemaker server without digest secret.");
+                throw new RuntimeException("Can't start pacemaker server without digest secret.");
+            }
+            break;
+
+        case "KERBEROS":
+            authMethod = ThriftNettyServerCodec.AuthMethod.KERBEROS;
+            break;
+
+        case "NONE":
+            authMethod = ThriftNettyServerCodec.AuthMethod.NONE;
+            break;
+
+        default:
+            LOG.error("Can't start pacemaker server without proper PACEMAKER_AUTH_METHOD.");
+            throw new RuntimeException("Can't start pacemaker server without proper PACEMAKER_AUTH_METHOD.");
+        }
+
+        ThreadFactory bossFactory = new NettyRenameThreadFactory("server-boss");
+        ThreadFactory workerFactory = new NettyRenameThreadFactory("server-worker");
+        NioServerSocketChannelFactory factory;
+        if(maxWorkers > 0) {
+            factory =
+                new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
+                                                  Executors.newCachedThreadPool(workerFactory),
+                                                  maxWorkers);
+        }
+        else {
+            factory =
+                new NioServerSocketChannelFactory(Executors.newCachedThreadPool(bossFactory),
+                                                  Executors.newCachedThreadPool(workerFactory));
+        }
+
+        bootstrap = new ServerBootstrap(factory);
+        bootstrap.setOption("tcpNoDelay", true);
+        bootstrap.setOption("sendBufferSize", FIVE_MB_IN_BYTES);
+        bootstrap.setOption("keepAlive", true);
+
+        ChannelPipelineFactory pipelineFactory = new ThriftNettyServerCodec(this, config, authMethod).pipelineFactory();
+        bootstrap.setPipelineFactory(pipelineFactory);
+        Channel channel = bootstrap.bind(new InetSocketAddress(port));
+        allChannels.add(channel);
+        LOG.info("Bound server to port: {}", Integer.toString(port));
+    }
+
+    /** Implementing IServer. **/
+    public void channelConnected(Channel c) {
+        allChannels.add(c);
+    }
+
+    public void cleanPipeline(Channel channel) {
+        boolean authenticated = authenticated_channels.contains(channel);
+        if(!authenticated) {
+            if(channel.getPipeline().get(ThriftNettyServerCodec.SASL_HANDLER) != null) {
+                channel.getPipeline().remove(ThriftNettyServerCodec.SASL_HANDLER);
+            }
+            else if(channel.getPipeline().get(ThriftNettyServerCodec.KERBEROS_HANDLER) != null) {
+                channel.getPipeline().remove(ThriftNettyServerCodec.KERBEROS_HANDLER);
+            }
+        }
+    }
+
+    public void received(Object mesg, String remote, Channel channel) throws InterruptedException {
+        cleanPipeline(channel);
+
+        boolean authenticated = (authMethod == ThriftNettyServerCodec.AuthMethod.NONE) || authenticated_channels.contains(channel);
+        HBMessage m = (HBMessage)mesg;
+        LOG.debug("received message. Passing to handler. {} : {} : {}",
+                  handler.toString(), m.toString(), channel.toString());
+        HBMessage response = handler.handleMessage(m, authenticated);
+        LOG.debug("Got Response from handler: {}", response.toString());
+        channel.write(response);
+    }
+
+    public void closeChannel(Channel c) {
+        c.close().awaitUninterruptibly();
+        allChannels.remove(c);
+        authenticated_channels.remove(c);
+    }
+
+    public String name() {
+        return topo_name;
+    }
+
+    public String secretKey() {
+        return secret;
+    }
+
+    public void authenticated(Channel c) {
+        LOG.debug("Pacemaker server authenticated channel: {}", c.toString());
+        authenticated_channels.add(c);
+    }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java
new file mode 100644
index 0000000..113594a
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftDecoder.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.pacemaker.codec;
+
+import org.jboss.netty.handler.codec.frame.FrameDecoder;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.Channel;
+import backtype.storm.generated.HBMessage;
+import backtype.storm.generated.HBServerMessageType;
+import org.jboss.netty.buffer.ChannelBuffer;
+import backtype.storm.utils.Utils;
+import backtype.storm.messaging.netty.ControlMessage;
+import backtype.storm.messaging.netty.SaslMessageToken;
+
+public class ThriftDecoder extends FrameDecoder {
+
+    @Override
+    protected Object decode(ChannelHandlerContext ctx, Channel channel, ChannelBuffer buf) throws Exception {
+
+        long available = buf.readableBytes();
+        if(available < 2) {
+            return null;
+        }
+
+        buf.markReaderIndex();
+
+        int thriftLen = buf.readInt();
+        available -= 4;
+
+        if(available < thriftLen) {
+            // We haven't received the entire object yet, return and wait for more bytes.
+            buf.resetReaderIndex();
+            return null;
+        }
+
+        buf.discardReadBytes();
+
+        HBMessage m;
+        if(buf.hasArray()) {
+            m = Utils.thriftDeserialize(HBMessage.class, buf.array(), 0, thriftLen);
+            buf.readerIndex(buf.readerIndex() + thriftLen);
+        }
+        else {
+            byte serialized[] = new byte[thriftLen];
+            buf.readBytes(serialized, 0, thriftLen);
+            m = Utils.thriftDeserialize(HBMessage.class, serialized);
+        }
+
+        if(m.get_type() == HBServerMessageType.CONTROL_MESSAGE) {
+            ControlMessage cm = ControlMessage.read(m.get_data().get_message_blob());
+            return cm;
+        }
+        else if(m.get_type() == HBServerMessageType.SASL_MESSAGE_TOKEN) {
+            SaslMessageToken sm = SaslMessageToken.read(m.get_data().get_message_blob());
+            return sm;
+        }
+        else {
+            return m;
+        }
+    }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftEncoder.java b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftEncoder.java
new file mode 100644
index 0000000..a6912f2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftEncoder.java
@@ -0,0 +1,110 @@
+/**
+ * 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 org.apache.storm.pacemaker.codec;
+
+import org.jboss.netty.handler.codec.oneone.OneToOneEncoder;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.Channel;
+import backtype.storm.generated.HBMessage;
+import backtype.storm.generated.HBMessageData;
+import backtype.storm.generated.HBServerMessageType;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.buffer.ChannelBuffer;
+import backtype.storm.utils.Utils;
+import backtype.storm.messaging.netty.ControlMessage;
+import backtype.storm.messaging.netty.SaslMessageToken;
+import backtype.storm.messaging.netty.INettySerializable;
+import java.io.IOException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.thrift.TBase;
+
+public class ThriftEncoder extends OneToOneEncoder {
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(ThriftEncoder.class);
+
+    private HBMessage encodeNettySerializable(INettySerializable netty_message,
+                                              HBServerMessageType mType) {
+        
+        HBMessageData message_data = new HBMessageData();
+        HBMessage m = new HBMessage();
+        try {
+            ChannelBuffer cbuffer = netty_message.buffer();
+            if(cbuffer.hasArray()) {
+                message_data.set_message_blob(cbuffer.array());
+            }
+            else {
+                byte buff[] = new byte[netty_message.encodeLength()];
+                cbuffer.readBytes(buff, 0, netty_message.encodeLength());
+                message_data.set_message_blob(buff);
+            }
+            m.set_type(mType);
+            m.set_data(message_data);
+            return m;
+        }
+        catch( IOException e) {
+            LOG.error("Failed to encode NettySerializable: ", e);
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    protected Object encode(ChannelHandlerContext ctx, Channel channel, Object msg) {
+        if(msg == null) {
+            return null;
+        }
+
+        LOG.debug("Trying to encode: " + msg.getClass().toString() + " : " + msg.toString());
+
+        HBMessage m;
+        if(msg instanceof INettySerializable) {
+            INettySerializable nettyMsg = (INettySerializable)msg;
+
+            HBServerMessageType type;
+            if(msg instanceof ControlMessage) {
+                type = HBServerMessageType.CONTROL_MESSAGE;
+            }
+            else if(msg instanceof SaslMessageToken) {
+                type = HBServerMessageType.SASL_MESSAGE_TOKEN;
+            }
+            else {
+                LOG.error("Didn't recognise INettySerializable: " + nettyMsg.toString());
+                throw new RuntimeException("Unrecognized INettySerializable.");
+            }
+            m = encodeNettySerializable(nettyMsg, type);
+        }
+        else {
+            m = (HBMessage)msg;
+        }
+
+        try {
+            byte serialized[] = Utils.thriftSerialize(m);
+            ChannelBuffer ret = ChannelBuffers.directBuffer(serialized.length + 4);
+
+            ret.writeInt(serialized.length);
+            ret.writeBytes(serialized);
+
+            return ret;
+        }
+        catch (RuntimeException e) {
+            LOG.error("Failed to serialize.", e);
+            throw e;
+        }
+    }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
new file mode 100644
index 0000000..5ba90fd
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyClientCodec.java
@@ -0,0 +1,94 @@
+/**
+ * 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 org.apache.storm.pacemaker.codec;
+
+import backtype.storm.messaging.netty.KerberosSaslClientHandler;
+import backtype.storm.messaging.netty.SaslStormClientHandler;
+import backtype.storm.security.auth.AuthUtils;
+import java.io.IOException;
+import java.util.Map;
+import org.apache.storm.pacemaker.PacemakerClient;
+import org.apache.storm.pacemaker.PacemakerClientHandler;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ThriftNettyClientCodec {
+
+    public static final String SASL_HANDLER = "sasl-handler";
+    public static final String KERBEROS_HANDLER = "kerberos-handler";
+    
+    public enum AuthMethod {
+        DIGEST,
+        KERBEROS,
+        NONE
+    };
+
+    private static final Logger LOG = LoggerFactory
+        .getLogger(ThriftNettyClientCodec.class);
+
+    private PacemakerClient client;
+    private AuthMethod authMethod;
+    private Map storm_conf;
+
+    public ThriftNettyClientCodec(PacemakerClient pacemaker_client, Map storm_conf, AuthMethod authMethod) {
+        client = pacemaker_client;
+        this.authMethod = authMethod;
+        this.storm_conf = storm_conf;
+    }
+
+    public ChannelPipelineFactory pipelineFactory() {
+        return new ChannelPipelineFactory() {
+            public ChannelPipeline getPipeline() {
+                ChannelPipeline pipeline = Channels.pipeline();
+                pipeline.addLast("encoder", new ThriftEncoder());
+                pipeline.addLast("decoder", new ThriftDecoder());
+
+                if (authMethod == AuthMethod.KERBEROS) {
+                    try {
+                        LOG.debug("Adding KerberosSaslClientHandler to pacemaker client pipeline.");
+                        pipeline.addLast(KERBEROS_HANDLER,
+                                         new KerberosSaslClientHandler(client,
+                                                                       storm_conf,
+                                                                       AuthUtils.LOGIN_CONTEXT_PACEMAKER_CLIENT));
+                    }
+                    catch (IOException e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+                else if(authMethod == AuthMethod.DIGEST) {
+                    try {
+                        LOG.debug("Adding SaslStormClientHandler to pacemaker client pipeline.");
+                        pipeline.addLast(SASL_HANDLER, new SaslStormClientHandler(client));
+                    }
+                    catch (IOException e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+                else {
+                    client.channelReady();
+                }
+
+                pipeline.addLast("PacemakerClientHandler", new PacemakerClientHandler(client));
+                return pipeline;
+            }
+        };
+    }
+}
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyServerCodec.java b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyServerCodec.java
new file mode 100644
index 0000000..c1ca00e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/codec/ThriftNettyServerCodec.java
@@ -0,0 +1,99 @@
+/**
+ * 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 org.apache.storm.pacemaker.codec;
+
+import backtype.storm.Config;
+import backtype.storm.messaging.netty.ISaslServer;
+import backtype.storm.messaging.netty.IServer;
+import backtype.storm.messaging.netty.KerberosSaslServerHandler;
+import backtype.storm.messaging.netty.SaslStormServerHandler;
+import backtype.storm.messaging.netty.StormServerHandler;
+import backtype.storm.security.auth.AuthUtils;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Map;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ThriftNettyServerCodec {
+
+    public static final String SASL_HANDLER = "sasl-handler";
+    public static final String KERBEROS_HANDLER = "kerberos-handler";
+    
+    public enum AuthMethod {
+        DIGEST,
+        KERBEROS,
+        NONE
+    };
+    
+    private IServer server;
+    private AuthMethod authMethod;
+    private Map storm_conf;
+    
+    private static final Logger LOG = LoggerFactory
+        .getLogger(ThriftNettyServerCodec.class);
+
+    public ThriftNettyServerCodec(IServer server, Map storm_conf, AuthMethod authMethod) {
+        this.server = server;
+        this.authMethod = authMethod;
+        this.storm_conf = storm_conf;
+    }
+
+    public ChannelPipelineFactory pipelineFactory() {
+        return new ChannelPipelineFactory() {
+            public ChannelPipeline getPipeline() {
+
+                ChannelPipeline pipeline = Channels.pipeline();
+                pipeline.addLast("encoder", new ThriftEncoder());
+                pipeline.addLast("decoder", new ThriftDecoder());
+                if(authMethod == AuthMethod.DIGEST) {
+                    try {
+                        LOG.debug("Adding SaslStormServerHandler to pacemaker server pipeline.");
+                        pipeline.addLast(SASL_HANDLER, new SaslStormServerHandler((ISaslServer)server));
+                    }
+                    catch (IOException e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+                else if(authMethod == AuthMethod.KERBEROS) {
+                    try {
+                        LOG.debug("Adding KerberosSaslServerHandler to pacemaker server pipeline.");
+                        ArrayList<String> authorizedUsers = new ArrayList(1);
+                        authorizedUsers.add((String)storm_conf.get(Config.NIMBUS_DAEMON_USER));
+                        pipeline.addLast(KERBEROS_HANDLER, new KerberosSaslServerHandler((ISaslServer)server,
+                                                                                         storm_conf,
+                                                                                         AuthUtils.LOGIN_CONTEXT_PACEMAKER_SERVER,
+                                                                                         authorizedUsers));
+                    }
+                    catch (IOException e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+                else if(authMethod == AuthMethod.NONE) {
+                    LOG.debug("Not authenticating any clients. AuthMethod is NONE");
+                }
+
+                pipeline.addLast("handler", new StormServerHandler(server));
+                return pipeline;
+            }
+        };
+    }
+}
diff --git a/storm-core/src/jvm/storm/trident/TridentTopology.java b/storm-core/src/jvm/storm/trident/TridentTopology.java
index af24664..5dfac1c 100644
--- a/storm-core/src/jvm/storm/trident/TridentTopology.java
+++ b/storm-core/src/jvm/storm/trident/TridentTopology.java
@@ -30,15 +30,8 @@
 import backtype.storm.utils.Utils;
 
 import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-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.TreeMap;
+import java.util.*;
+
 import org.jgrapht.DirectedGraph;
 import org.jgrapht.UndirectedGraph;
 import org.jgrapht.alg.ConnectivityInspector;
@@ -88,16 +81,17 @@
 // all operations have finishBatch and can optionally be committers
 public class TridentTopology {
     
-    //TODO: add a method for drpc stream, needs to know how to automatically do returnresults, etc
+    //TODO: add a method for drpc stream, needs to know how to automatically do return results, etc
     // is it too expensive to do a batch per drpc request?
     
-    DefaultDirectedGraph<Node, IndexedEdge> _graph;
-    Map<String, List<Node>> _colocate = new HashMap();
-    UniqueIdGen _gen;
-    
+    final DefaultDirectedGraph<Node, IndexedEdge> _graph;
+    final Map<String, List<Node>> _colocate;
+    final UniqueIdGen _gen;
+
     public TridentTopology() {
-        _graph = new DefaultDirectedGraph(new ErrorEdgeFactory());
-        _gen = new UniqueIdGen();
+        this(new DefaultDirectedGraph<Node, IndexedEdge>(new ErrorEdgeFactory()),
+                new LinkedHashMap<String, List<Node>>(),
+                new UniqueIdGen());
     }
     
     private TridentTopology(DefaultDirectedGraph<Node, IndexedEdge> graph, Map<String, List<Node>> colocate, UniqueIdGen gen) {
@@ -210,7 +204,7 @@
     }    
     
     public Stream multiReduce(List<Fields> inputFields, List<Stream> streams, MultiReducer function, Fields outputFields) {
-        List<String> names = new ArrayList<String>();
+        List<String> names = new ArrayList<>();
         for(Stream s: streams) {
             if(s._name!=null) {
                 names.add(s._name);
@@ -221,9 +215,9 @@
     }
     
     public Stream multiReduce(List<Fields> inputFields, List<GroupedStream> groupedStreams, GroupedMultiReducer function, Fields outputFields) {
-        List<Fields> fullInputFields = new ArrayList<Fields>();
-        List<Stream> streams = new ArrayList<Stream>();
-        List<Fields> fullGroupFields = new ArrayList<Fields>();
+        List<Fields> fullInputFields = new ArrayList<>();
+        List<Stream> streams = new ArrayList<>();
+        List<Fields> fullGroupFields = new ArrayList<>();
         for(int i=0; i<groupedStreams.size(); i++) {
             GroupedStream gs = groupedStreams.get(i);
             Fields groupFields = gs.getGroupFields();
@@ -285,10 +279,10 @@
         
         completeDRPC(graph, _colocate, _gen);
         
-        List<SpoutNode> spoutNodes = new ArrayList<SpoutNode>();
+        List<SpoutNode> spoutNodes = new ArrayList<>();
         
         // can be regular nodes (static state) or processor nodes
-        Set<Node> boltNodes = new HashSet<Node>();
+        Set<Node> boltNodes = new LinkedHashSet<>();
         for(Node n: graph.vertexSet()) {
             if(n instanceof SpoutNode) {
                 spoutNodes.add((SpoutNode) n);
@@ -298,7 +292,7 @@
         }
         
         
-        Set<Group> initialGroups = new HashSet<Group>();
+        Set<Group> initialGroups = new LinkedHashSet<>();
         for(List<Node> colocate: _colocate.values()) {
             Group g = new Group(graph, colocate);
             boltNodes.removeAll(colocate);
@@ -316,7 +310,7 @@
         
         
         // add identity partitions between groups
-        for(IndexedEdge<Node> e: new HashSet<IndexedEdge>(graph.edgeSet())) {
+        for(IndexedEdge<Node> e: new HashSet<>(graph.edgeSet())) {
             if(!(e.source instanceof PartitionNode) && !(e.target instanceof PartitionNode)) {                
                 Group g1 = grouper.nodeGroup(e.source);
                 Group g2 = grouper.nodeGroup(e.target);
@@ -340,7 +334,7 @@
         // this is because can't currently merge splitting logic into a spout
         // not the most kosher algorithm here, since the grouper indexes are being trounced via the adding of nodes to random groups, but it 
         // works out
-        List<Node> forNewGroups = new ArrayList<Node>();
+        List<Node> forNewGroups = new ArrayList<>();
         for(Group g: mergedGroups) {
             for(PartitionNode n: extraPartitionInputs(g)) {
                 Node idNode = makeIdentityNode(n.allOutputFields);
@@ -365,7 +359,7 @@
             }
         }
         // TODO: in the future, want a way to include this logic in the spout itself,
-        // or make it unecessary by having storm include metadata about which grouping a tuple
+        // or make it unnecessary by having storm include metadata about which grouping a tuple
         // came from
         
         for(Node n: forNewGroups) {
@@ -381,8 +375,8 @@
         mergedGroups = grouper.getAllGroups();
                 
         
-        Map<Node, String> batchGroupMap = new HashMap();
-        List<Set<Node>> connectedComponents = new ConnectivityInspector<Node, IndexedEdge>(graph).connectedSets();
+        Map<Node, String> batchGroupMap = new HashMap<>();
+        List<Set<Node>> connectedComponents = new ConnectivityInspector<>(graph).connectedSets();
         for(int i=0; i<connectedComponents.size(); i++) {
             String groupId = "bg" + i;
             for(Node n: connectedComponents.get(i)) {
@@ -428,14 +422,10 @@
                 Collection<PartitionNode> inputs = uniquedSubscriptions(externalGroupInputs(g));
                 for(PartitionNode n: inputs) {
                     Node parent = TridentUtils.getParent(graph, n);
-                    String componentId;
-                    if(parent instanceof SpoutNode) {
-                        componentId = spoutIds.get(parent);
-                    } else {
-                        componentId = boltIds.get(grouper.nodeGroup(parent));
-                    }
+                    String componentId = parent instanceof SpoutNode ?
+                            spoutIds.get(parent) : boltIds.get(grouper.nodeGroup(parent));
                     d.grouping(new GlobalStreamId(componentId, n.streamId), n.thriftGrouping);
-                } 
+                }
             }
         }
         
@@ -443,7 +433,7 @@
     }
     
     private static void completeDRPC(DefaultDirectedGraph<Node, IndexedEdge> graph, Map<String, List<Node>> colocate, UniqueIdGen gen) {
-        List<Set<Node>> connectedComponents = new ConnectivityInspector<Node, IndexedEdge>(graph).connectedSets();
+        List<Set<Node>> connectedComponents = new ConnectivityInspector<>(graph).connectedSets();
         
         for(Set<Node> g: connectedComponents) {
             checkValidJoins(g);
@@ -511,7 +501,7 @@
     }
     
     private static Collection<PartitionNode> uniquedSubscriptions(Set<PartitionNode> subscriptions) {
-        Map<String, PartitionNode> ret = new HashMap();
+        Map<String, PartitionNode> ret = new HashMap<>();
         for(PartitionNode n: subscriptions) {
             PartitionNode curr = ret.get(n.streamId);
             if(curr!=null && !curr.thriftGrouping.equals(n.thriftGrouping)) {
@@ -523,7 +513,7 @@
     }
 
     private static Map<Node, String> genSpoutIds(Collection<SpoutNode> spoutNodes) {
-        Map<Node, String> ret = new HashMap();
+        Map<Node, String> ret = new HashMap<>();
         int ctr = 0;
         for(SpoutNode n: spoutNodes) {
             if (n.type == SpoutNode.SpoutType.BATCH) { // if Batch spout then id contains txId
@@ -540,11 +530,11 @@
     }
 
     private static Map<Group, String> genBoltIds(Collection<Group> groups) {
-        Map<Group, String> ret = new HashMap();
+        Map<Group, String> ret = new HashMap<>();
         int ctr = 0;
         for(Group g: groups) {
             if(!isSpoutGroup(g)) {
-                List<String> name = new ArrayList();
+                List<String> name = new ArrayList<>();
                 name.add("b");
                 name.add("" + ctr);
                 String groupName = getGroupName(g);
@@ -559,13 +549,13 @@
     }
     
     private static String getGroupName(Group g) {
-        TreeMap<Integer, String> sortedNames = new TreeMap();
+        TreeMap<Integer, String> sortedNames = new TreeMap<>();
         for(Node n: g.nodes) {
             if(n.name!=null) {
                 sortedNames.put(n.creationIndex, n.name);
             }
         }
-        List<String> names = new ArrayList<String>();
+        List<String> names = new ArrayList<>();
         String prevName = null;
         for(String n: sortedNames.values()) {
             if(prevName==null || !n.equals(prevName)) {
@@ -577,7 +567,7 @@
     }
     
     private static Map<String, String> getOutputStreamBatchGroups(Group g, Map<Node, String> batchGroupMap) {
-        Map<String, String> ret = new HashMap();
+        Map<String, String> ret = new HashMap<>();
         Set<PartitionNode> externalGroupOutputs = externalGroupOutputs(g);
         for(PartitionNode n: externalGroupOutputs) {
             ret.put(n.streamId, batchGroupMap.get(n));
@@ -586,7 +576,7 @@
     }
     
     private static Set<String> committerBatches(Group g, Map<Node, String> batchGroupMap) {
-        Set<String> ret = new HashSet();
+        Set<String> ret = new HashSet<>();
         for(Node n: g.nodes) {
            if(n instanceof ProcessorNode) {
                if(((ProcessorNode) n).committer) {
@@ -598,7 +588,7 @@
     }
     
     private static Map<Group, Integer> getGroupParallelisms(DirectedGraph<Node, IndexedEdge> graph, GraphGrouper grouper, Collection<Group> groups) {
-        UndirectedGraph<Group, Object> equivs = new Pseudograph<Group, Object>(Object.class);
+        UndirectedGraph<Group, Object> equivs = new Pseudograph<>(Object.class);
         for(Group g: groups) {
             equivs.addVertex(g);
         }
@@ -614,8 +604,8 @@
             }            
         }
         
-        Map<Group, Integer> ret = new HashMap();
-        List<Set<Group>> equivGroups = new ConnectivityInspector<Group, Object>(equivs).connectedSets();
+        Map<Group, Integer> ret = new HashMap<>();
+        List<Set<Group>> equivGroups = new ConnectivityInspector<>(equivs).connectedSets();
         for(Set<Group> equivGroup: equivGroups) {
             Integer fixedP = getFixedParallelism(equivGroup);
             Integer maxP = getMaxParallelism(equivGroup);
@@ -704,9 +694,9 @@
     }
     
     private static List<PartitionNode> extraPartitionInputs(Group g) {
-        List<PartitionNode> ret = new ArrayList();
+        List<PartitionNode> ret = new ArrayList<>();
         Set<PartitionNode> inputs = externalGroupInputs(g);
-        Map<String, List<PartitionNode>> grouped = new HashMap();
+        Map<String, List<PartitionNode>> grouped = new HashMap<>();
         for(PartitionNode n: inputs) {
             if(!grouped.containsKey(n.streamId)) {
                 grouped.put(n.streamId, new ArrayList());
@@ -726,7 +716,7 @@
     }
     
     private static Set<PartitionNode> externalGroupInputs(Group g) {
-        Set<PartitionNode> ret = new HashSet();
+        Set<PartitionNode> ret = new HashSet<>();
         for(Node n: g.incomingNodes()) {
             if(n instanceof PartitionNode) {
                 ret.add((PartitionNode) n);
@@ -736,7 +726,7 @@
     }
     
     private static Set<PartitionNode> externalGroupOutputs(Group g) {
-        Set<PartitionNode> ret = new HashSet();
+        Set<PartitionNode> ret = new HashSet<>();
         for(Node n: g.outgoingNodes()) {
             if(n instanceof PartitionNode) {
                 ret.add((PartitionNode) n);
@@ -803,7 +793,7 @@
     }       
     
     private static List<Fields> getAllOutputFields(List streams) {
-        List<Fields> ret = new ArrayList<Fields>();
+        List<Fields> ret = new ArrayList<>();
         for(Object o: streams) {
             ret.add(((IAggregatableStream) o).getOutputFields());
         }
@@ -812,7 +802,7 @@
     
     
     private static List<GroupedStream> groupedStreams(List<Stream> streams, List<Fields> joinFields) {
-        List<GroupedStream> ret = new ArrayList<GroupedStream>();
+        List<GroupedStream> ret = new ArrayList<>();
         for(int i=0; i<streams.size(); i++) {
             ret.add(streams.get(i).groupBy(joinFields.get(i)));
         }
@@ -820,7 +810,7 @@
     }
     
     private static List<Fields> strippedInputFields(List<Stream> streams, List<Fields> joinFields) {
-        List<Fields> ret = new ArrayList<Fields>();
+        List<Fields> ret = new ArrayList<>();
         for(int i=0; i<streams.size(); i++) {
             ret.add(TridentUtils.fieldsSubtract(streams.get(i).getOutputFields(), joinFields.get(i)));
         }
@@ -828,7 +818,7 @@
     }
     
     private static List<JoinType> repeat(int n, JoinType type) {
-        List<JoinType> ret = new ArrayList<JoinType>();
+        List<JoinType> ret = new ArrayList<>();
         for(int i=0; i<n; i++) {
             ret.add(type);
         }
diff --git a/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java b/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java
index 849fb10..515d3a2 100644
--- a/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java
+++ b/storm-core/src/jvm/storm/trident/drpc/ReturnResultsReducer.java
@@ -40,7 +40,7 @@
 
 public class ReturnResultsReducer implements MultiReducer<ReturnResultsState> {
     public static class ReturnResultsState {
-        List<TridentTuple> results = new ArrayList<TridentTuple>();
+        List<TridentTuple> results = new ArrayList<>();
         String returnInfo;
 
         @Override
@@ -50,7 +50,7 @@
     }
     boolean local;
     Map conf;
-    Map<List, DRPCInvocationsClient> _clients = new HashMap<List, DRPCInvocationsClient>();
+    Map<List, DRPCInvocationsClient> _clients = new HashMap<>();
     
     
     @Override
diff --git a/storm-core/src/jvm/storm/trident/fluent/ChainedAggregatorDeclarer.java b/storm-core/src/jvm/storm/trident/fluent/ChainedAggregatorDeclarer.java
index 8040e8b..976f983 100644
--- a/storm-core/src/jvm/storm/trident/fluent/ChainedAggregatorDeclarer.java
+++ b/storm-core/src/jvm/storm/trident/fluent/ChainedAggregatorDeclarer.java
@@ -59,7 +59,7 @@
         }
     }
     
-    List<AggSpec> _aggs = new ArrayList<AggSpec>();
+    List<AggSpec> _aggs = new ArrayList<>();
     IAggregatableStream _stream;
     AggType _type = null;
     GlobalAggregationScheme _globalScheme;
@@ -73,8 +73,8 @@
         Fields[] inputFields = new Fields[_aggs.size()];
         Aggregator[] aggs = new Aggregator[_aggs.size()];
         int[] outSizes = new int[_aggs.size()];
-        List<String> allOutFields = new ArrayList<String>();
-        Set<String> allInFields = new HashSet<String>();
+        List<String> allOutFields = new ArrayList<>();
+        Set<String> allInFields = new HashSet<>();
         for(int i=0; i<_aggs.size(); i++) {
             AggSpec spec = _aggs.get(i);
             Fields infields = spec.inFields;
@@ -92,7 +92,7 @@
             throw new IllegalArgumentException("Output fields for chained aggregators must be distinct: " + allOutFields.toString());
         }
         
-        Fields inFields = new Fields(new ArrayList<String>(allInFields));
+        Fields inFields = new Fields(new ArrayList<>(allInFields));
         Fields outFields = new Fields(allOutFields);
         Aggregator combined = new ChainedAggregatorImpl(aggs, inputFields, new ComboList.Factory(outSizes));
         
diff --git a/storm-core/src/jvm/storm/trident/graph/GraphGrouper.java b/storm-core/src/jvm/storm/trident/graph/GraphGrouper.java
index 94db077..671498e 100644
--- a/storm-core/src/jvm/storm/trident/graph/GraphGrouper.java
+++ b/storm-core/src/jvm/storm/trident/graph/GraphGrouper.java
@@ -17,26 +17,22 @@
  */
 package storm.trident.graph;
 
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
+
 import org.jgrapht.DirectedGraph;
 import storm.trident.planner.Node;
 import storm.trident.util.IndexedEdge;
 
 
 public class GraphGrouper {
-    
-    DirectedGraph<Node, IndexedEdge> graph;
-    Set<Group> currGroups;
-    Map<Node, Group> groupIndex = new HashMap();
+    final DirectedGraph<Node, IndexedEdge> graph;
+    final Set<Group> currGroups;
+    final Map<Node, Group> groupIndex = new HashMap<>();
     
     public GraphGrouper(DirectedGraph<Node, IndexedEdge> graph, Collection<Group> initialGroups) {
         this.graph = graph;
-        this.currGroups = new HashSet(initialGroups);
-        reindex();      
+        this.currGroups = new LinkedHashSet<>(initialGroups);
+        reindex();
     }
     
     public Collection<Group> getAllGroups() {
@@ -95,7 +91,7 @@
     }
     
     public Collection<Group> outgoingGroups(Group g) {
-        Set<Group> ret = new HashSet();
+        Set<Group> ret = new HashSet<>();
         for(Node n: g.outgoingNodes()) {
             Group other = nodeGroup(n);
             if(other==null || !other.equals(g)) {
@@ -106,7 +102,7 @@
     }
     
     public Collection<Group> incomingGroups(Group g) {
-        Set<Group> ret = new HashSet();
+        Set<Group> ret = new HashSet<>();
         for(Node n: g.incomingNodes()) {
             Group other = nodeGroup(n);
             if(other==null || !other.equals(g)) {
diff --git a/storm-core/src/jvm/storm/trident/graph/Group.java b/storm-core/src/jvm/storm/trident/graph/Group.java
index 8ed0023..64bdec6 100644
--- a/storm-core/src/jvm/storm/trident/graph/Group.java
+++ b/storm-core/src/jvm/storm/trident/graph/Group.java
@@ -29,14 +29,13 @@
 
 
 public class Group {
-    public Set<Node> nodes = new HashSet<Node>();
-    private DirectedGraph<Node, IndexedEdge> graph;
-    private String id;
+    public final Set<Node> nodes = new HashSet<>();
+    private final DirectedGraph<Node, IndexedEdge> graph;
+    private final String id = UUID.randomUUID().toString();
     
     public Group(DirectedGraph graph, List<Node> nodes) {
-        init(graph);
-        this.nodes.addAll(nodes);
         this.graph = graph;
+        this.nodes.addAll(nodes);
     }
     
     public Group(DirectedGraph graph, Node n) {
@@ -44,18 +43,13 @@
     }
     
     public Group(Group g1, Group g2) {
-        init(g1.graph);
+        this.graph = g1.graph;
         nodes.addAll(g1.nodes);
         nodes.addAll(g2.nodes);
     }
     
-    private void init(DirectedGraph graph) {
-        this.graph = graph;
-        this.id = UUID.randomUUID().toString();
-    }
-    
     public Set<Node> outgoingNodes() {
-        Set<Node> ret = new HashSet<Node>();
+        Set<Node> ret = new HashSet<>();
         for(Node n: nodes) {
             ret.addAll(TridentUtils.getChildren(graph, n));
         }
@@ -63,7 +57,7 @@
     }
     
     public Set<Node> incomingNodes() {
-        Set<Node> ret = new HashSet<Node>();
+        Set<Node> ret = new HashSet<>();
         for(Node n: nodes) {
             ret.addAll(TridentUtils.getParents(graph, n));
         }        
@@ -77,6 +71,9 @@
 
     @Override
     public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
         return id.equals(((Group) o).id);
     }
 
diff --git a/storm-core/src/jvm/storm/trident/operation/builtin/SnapshotGet.java b/storm-core/src/jvm/storm/trident/operation/builtin/SnapshotGet.java
index 6d24ae6..2a44eab 100644
--- a/storm-core/src/jvm/storm/trident/operation/builtin/SnapshotGet.java
+++ b/storm-core/src/jvm/storm/trident/operation/builtin/SnapshotGet.java
@@ -29,9 +29,9 @@
 
     @Override
     public List<Object> batchRetrieve(ReadOnlySnapshottable state, List<TridentTuple> args) {
-        List<Object> ret = new ArrayList<Object>(args.size());
+        List<Object> ret = new ArrayList<>(args.size());
         Object snapshot = state.get();
-        for(int i=0; i<args.size(); i++) {
+        for (TridentTuple arg : args) {
             ret.add(snapshot);
         }
         return ret;
diff --git a/storm-core/src/jvm/storm/trident/operation/builtin/TupleCollectionGet.java b/storm-core/src/jvm/storm/trident/operation/builtin/TupleCollectionGet.java
index 52dd633..426b56f 100644
--- a/storm-core/src/jvm/storm/trident/operation/builtin/TupleCollectionGet.java
+++ b/storm-core/src/jvm/storm/trident/operation/builtin/TupleCollectionGet.java
@@ -30,9 +30,9 @@
 
     @Override
     public List<Iterator<List<Object>>> batchRetrieve(State state, List<TridentTuple> args) {
-        List<Iterator<List<Object>>> ret = new ArrayList(args.size());
-        for(int i=0; i<args.size(); i++) {
-            ret.add(((ITupleCollection)state).getTuples());
+        List<Iterator<List<Object>>> ret = new ArrayList<>(args.size());
+        for (TridentTuple arg : args) {
+            ret.add(((ITupleCollection) state).getTuples());
         }
         return ret;
     }
diff --git a/storm-core/src/jvm/storm/trident/partition/GlobalGrouping.java b/storm-core/src/jvm/storm/trident/partition/GlobalGrouping.java
index 54fa844..a535eb2 100644
--- a/storm-core/src/jvm/storm/trident/partition/GlobalGrouping.java
+++ b/storm-core/src/jvm/storm/trident/partition/GlobalGrouping.java
@@ -26,13 +26,11 @@
 import java.util.List;
 
 public class GlobalGrouping implements CustomStreamGrouping {
-
     List<Integer> target;
     
-    
     @Override
     public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> targets) {
-        List<Integer> sorted = new ArrayList<Integer>(targets);
+        List<Integer> sorted = new ArrayList<>(targets);
         Collections.sort(sorted);
         target = Arrays.asList(sorted.get(0));
     }
@@ -41,5 +39,4 @@
     public List<Integer> chooseTasks(int i, List<Object> list) {
         return target;
     }
-    
 }
diff --git a/storm-core/src/jvm/storm/trident/partition/IdentityGrouping.java b/storm-core/src/jvm/storm/trident/partition/IdentityGrouping.java
index 30f48ad..88c2922 100644
--- a/storm-core/src/jvm/storm/trident/partition/IdentityGrouping.java
+++ b/storm-core/src/jvm/storm/trident/partition/IdentityGrouping.java
@@ -29,18 +29,16 @@
 
 
 public class IdentityGrouping implements CustomStreamGrouping {
-
-    List<Integer> ret = new ArrayList<Integer>();
-    Map<Integer, List<Integer>> _precomputed = new HashMap();
+    final Map<Integer, List<Integer>> _precomputed = new HashMap<>();
     
     @Override
     public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List<Integer> tasks) {
-        List<Integer> sourceTasks = new ArrayList<Integer>(context.getComponentTasks(stream.get_componentId()));
+        List<Integer> sourceTasks = new ArrayList<>(context.getComponentTasks(stream.get_componentId()));
         Collections.sort(sourceTasks);
         if(sourceTasks.size()!=tasks.size()) {
             throw new RuntimeException("Can only do an identity grouping when source and target have same number of tasks");
         }
-        tasks = new ArrayList<Integer>(tasks);
+        tasks = new ArrayList<>(tasks);
         Collections.sort(tasks);
         for(int i=0; i<sourceTasks.size(); i++) {
             int s = sourceTasks.get(i);
diff --git a/storm-core/src/jvm/storm/trident/planner/Node.java b/storm-core/src/jvm/storm/trident/planner/Node.java
index 6284cb9..d35e7da 100644
--- a/storm-core/src/jvm/storm/trident/planner/Node.java
+++ b/storm-core/src/jvm/storm/trident/planner/Node.java
@@ -26,7 +26,7 @@
 
 
 public class Node implements Serializable {
-    private static AtomicInteger INDEX = new AtomicInteger(0);
+    private static final AtomicInteger INDEX = new AtomicInteger(0);
     
     private String nodeId;
     
@@ -47,6 +47,9 @@
 
     @Override
     public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
         return nodeId.equals(((Node) o).nodeId);
     }
 
diff --git a/storm-core/src/jvm/storm/trident/planner/PartitionNode.java b/storm-core/src/jvm/storm/trident/planner/PartitionNode.java
index 4f10c25..4485765 100644
--- a/storm-core/src/jvm/storm/trident/planner/PartitionNode.java
+++ b/storm-core/src/jvm/storm/trident/planner/PartitionNode.java
@@ -22,8 +22,6 @@
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
-import java.util.ArrayList;
-import java.util.List;
 import storm.trident.util.TridentUtils;
 
 
diff --git a/storm-core/src/jvm/storm/trident/planner/SubtopologyBolt.java b/storm-core/src/jvm/storm/trident/planner/SubtopologyBolt.java
index cdceaf9..6320a21 100644
--- a/storm-core/src/jvm/storm/trident/planner/SubtopologyBolt.java
+++ b/storm-core/src/jvm/storm/trident/planner/SubtopologyBolt.java
@@ -23,7 +23,6 @@
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Tuple;
-import backtype.storm.utils.Utils;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -49,9 +48,9 @@
 public class SubtopologyBolt implements ITridentBatchBolt {
     DirectedGraph _graph;
     Set<Node> _nodes;
-    Map<String, InitialReceiver> _roots = new HashMap();
-    Map<Node, Factory> _outputFactories = new HashMap();
-    Map<String, List<TridentProcessor>> _myTopologicallyOrdered = new HashMap();
+    Map<String, InitialReceiver> _roots = new HashMap<>();
+    Map<Node, Factory> _outputFactories = new HashMap<>();
+    Map<String, List<TridentProcessor>> _myTopologicallyOrdered = new HashMap<>();
     Map<Node, String> _batchGroups;
     
     //given processornodes and static state nodes
@@ -71,7 +70,7 @@
             }
         }
         DirectedSubgraph<Node, Object> subgraph = new DirectedSubgraph(_graph, _nodes, null);
-        TopologicalOrderIterator it = new TopologicalOrderIterator<Node, Object>(subgraph);
+        TopologicalOrderIterator it = new TopologicalOrderIterator<>(subgraph);
         int stateIndex = 0;
         while(it.hasNext()) {
             Node n = (Node) it.next();
@@ -82,8 +81,8 @@
                     _myTopologicallyOrdered.put(batchGroup, new ArrayList());
                 }
                 _myTopologicallyOrdered.get(batchGroup).add(pn.processor);
-                List<String> parentStreams = new ArrayList();
-                List<Factory> parentFactories = new ArrayList();
+                List<String> parentStreams = new ArrayList<>();
+                List<Factory> parentFactories = new ArrayList<>();
                 for(Node p: TridentUtils.getParents(_graph, n)) {
                     parentStreams.add(p.streamId);
                     if(_nodes.contains(p)) {
@@ -96,7 +95,7 @@
                         parentFactories.add(_roots.get(p.streamId).getOutputFactory());
                     }
                 }
-                List<TupleReceiver> targets = new ArrayList();
+                List<TupleReceiver> targets = new ArrayList<>();
                 boolean outgoingNode = false;
                 for(Node cn: TridentUtils.getChildren(_graph, n)) {
                     if(_nodes.contains(cn)) {
@@ -185,7 +184,7 @@
     
     
     protected static class InitialReceiver {
-        List<TridentProcessor> _receivers = new ArrayList();
+        List<TridentProcessor> _receivers = new ArrayList<>();
         RootFactory _factory;
         ProjectionFactory _project;
         String _stream;
@@ -195,7 +194,7 @@
             // how to distinguish "batch" streams from non-batch streams?
             _stream = stream;
             _factory = new RootFactory(allFields);
-            List<String> projected = new ArrayList(allFields.toList());
+            List<String> projected = new ArrayList<>(allFields.toList());
             projected.remove(0);
             _project = new ProjectionFactory(_factory, new Fields(projected));
         }
diff --git a/storm-core/src/jvm/storm/trident/planner/processor/MultiReducerProcessor.java b/storm-core/src/jvm/storm/trident/planner/processor/MultiReducerProcessor.java
index 6777d2f..4086996 100644
--- a/storm-core/src/jvm/storm/trident/planner/processor/MultiReducerProcessor.java
+++ b/storm-core/src/jvm/storm/trident/planner/processor/MultiReducerProcessor.java
@@ -49,7 +49,7 @@
     public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) {
         List<Factory> parents = tridentContext.getParentTupleFactories();
         _context = tridentContext;
-        _streamToIndex = new HashMap<String, Integer>();
+        _streamToIndex = new HashMap<>();
         List<String> parentStreams = tridentContext.getParentStreams();
         for(int i=0; i<parentStreams.size(); i++) {
             _streamToIndex.put(parentStreams.get(i), i);
diff --git a/storm-core/src/jvm/storm/trident/spout/ITridentSpout.java b/storm-core/src/jvm/storm/trident/spout/ITridentSpout.java
index e43d144..3037298 100644
--- a/storm-core/src/jvm/storm/trident/spout/ITridentSpout.java
+++ b/storm-core/src/jvm/storm/trident/spout/ITridentSpout.java
@@ -34,8 +34,8 @@
          * 
          * The metadata is stored in Zookeeper.
          * 
-         * Storm uses the Kryo serializations configured in the component configuration 
-         * for this spout to serialize and deserialize the metadata.
+         * Storm uses JSON encoding to store the metadata.  Only simple types
+         * such as numbers, booleans, strings, lists, and maps should be used.
          * 
          * @param txid The id of the transaction.
          * @param prevMetadata The metadata of the previous transaction
@@ -44,9 +44,20 @@
          * @return the metadata for this new transaction
          */
         X initializeTransaction(long txid, X prevMetadata, X currMetadata);
-        
+
+        /**
+         * This attempt committed successfully, so all state for this commit and before can be safely cleaned up.
+         *
+         * @param txid transaction id that completed
+         */
         void success(long txid);
-        
+
+        /**
+         * hint to Storm if the spout is ready for the transaction id
+         *
+         * @param txid the id of the transaction
+         * @return true, if the spout is ready for the given transaction id
+         */
         boolean isReady(long txid);
         
         /**
@@ -58,14 +69,17 @@
     interface Emitter<X> {
         /**
          * Emit a batch for the specified transaction attempt and metadata for the transaction. The metadata
-         * was created by the Coordinator in the initializeTranaction method. This method must always emit
+         * was created by the Coordinator in the initializeTransaction method. This method must always emit
          * the same batch of tuples across all tasks for the same transaction id.
-         * 
+         * @param tx transaction id
+         * @param coordinatorMeta metadata from the coordinator defining this transaction
+         * @param collector output tuple collector
          */
         void emitBatch(TransactionAttempt tx, X coordinatorMeta, TridentCollector collector);
         
         /**
          * This attempt committed successfully, so all state for this commit and before can be safely cleaned up.
+         * @param tx attempt object containing transaction id and attempt number
          */
         void success(TransactionAttempt tx);
         
@@ -76,9 +90,18 @@
     }
     
     /**
-     * The coordinator for a TransactionalSpout runs in a single thread and indicates when batches
-     * of tuples should be emitted and when transactions should commit. The Coordinator that you provide 
-     * in a TransactionalSpout provides metadata for each transaction so that the transactions can be replayed.
+     * The coordinator for a TransactionalSpout runs in a single thread and indicates when batches of tuples
+     * should be emitted. The Coordinator that you provide in a TransactionalSpout provides metadata for each
+     * transaction so that the transactions can be replayed in case of failure.
+     *
+     * Two instances are requested, one on the master batch coordinator where isReady() is called, and an instance
+     * in the coordinator bolt which is used for all other operations. The two instances do not necessarily share a
+     * worker JVM.
+     *
+     * @param txStateId stream id
+     * @param conf Storm config map
+     * @param context topology context
+     * @return spout coordinator instance
      */
     BatchCoordinator<T> getCoordinator(String txStateId, Map conf, TopologyContext context);
 
@@ -86,7 +109,15 @@
      * The emitter for a TransactionalSpout runs as many tasks across the cluster. Emitters are responsible for
      * emitting batches of tuples for a transaction and must ensure that the same batch of tuples is always
      * emitted for the same transaction id.
-     */    
+     *
+     * All emitter tasks get the same transaction metadata. The topology context parameter contains the instance
+     * task id that can be used to distribute the work across the tasks.
+     *
+     * @param txStateId stream id
+     * @param conf Storm config map
+     * @param context topology context
+     * @return spout emitter
+     */
     Emitter<T> getEmitter(String txStateId, Map conf, TopologyContext context); 
     
     Map<String, Object> getComponentConfiguration();
diff --git a/storm-core/src/jvm/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java b/storm-core/src/jvm/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java
index 9cd98ee..cb19b7c 100644
--- a/storm-core/src/jvm/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java
+++ b/storm-core/src/jvm/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java
@@ -119,8 +119,9 @@
                 prevCached = new HashMap<>();
             }
             
-            for(String id: _partitionStates.keySet()) {
-                EmitterPartitionState s = _partitionStates.get(id);
+            for(Entry<String, EmitterPartitionState> e: _partitionStates.entrySet()) {
+                String id = e.getKey();
+                EmitterPartitionState s = e.getValue();
                 s.rotatingState.removeState(tx.getTransactionId());
                 Object lastMeta = prevCached.get(id);
                 if(lastMeta==null) lastMeta = s.rotatingState.getLastState();
@@ -162,9 +163,8 @@
             
             Long txid = attempt.getTransactionId();
             Map<String, Object> metas = _cachedMetas.remove(txid);
-            for(String partitionId: metas.keySet()) {
-                Object meta = metas.get(partitionId);
-                _partitionStates.get(partitionId).rotatingState.overrideState(txid, meta);
+            for(Entry<String, Object> entry: metas.entrySet()) {
+                _partitionStates.get(entry.getKey()).rotatingState.overrideState(txid, entry.getValue());
             }
         }
 
diff --git a/storm-core/src/jvm/storm/trident/spout/TridentSpoutExecutor.java b/storm-core/src/jvm/storm/trident/spout/TridentSpoutExecutor.java
index 6431426..d43d4e4 100644
--- a/storm-core/src/jvm/storm/trident/spout/TridentSpoutExecutor.java
+++ b/storm-core/src/jvm/storm/trident/spout/TridentSpoutExecutor.java
@@ -37,9 +37,9 @@
 import storm.trident.tuple.ConsList;
 
 public class TridentSpoutExecutor implements ITridentBatchBolt {
-    public static String ID_FIELD = "$tx";
+    public static final String ID_FIELD = "$tx";
     
-    public static Logger LOG = LoggerFactory.getLogger(TridentSpoutExecutor.class);    
+    public static final Logger LOG = LoggerFactory.getLogger(TridentSpoutExecutor.class);
 
     AddIdCollector _collector;
     ITridentSpout<Object> _spout;
diff --git a/storm-core/src/jvm/storm/trident/topology/TridentBoltExecutor.java b/storm-core/src/jvm/storm/trident/topology/TridentBoltExecutor.java
index fa68ae2..7210541 100644
--- a/storm-core/src/jvm/storm/trident/topology/TridentBoltExecutor.java
+++ b/storm-core/src/jvm/storm/trident/topology/TridentBoltExecutor.java
@@ -47,7 +47,7 @@
 import storm.trident.spout.IBatchID;
 
 public class TridentBoltExecutor implements IRichBolt {
-    public static String COORD_STREAM_PREFIX = "$coord-";
+    public static final String COORD_STREAM_PREFIX = "$coord-";
     
     public static String COORD_STREAM(String batch) {
         return COORD_STREAM_PREFIX + batch;
@@ -81,7 +81,7 @@
     
     public static class CoordSpec implements Serializable {
         public GlobalStreamId commitStream = null;
-        public Map<String, CoordType> coords = new HashMap<String, CoordType>();
+        public Map<String, CoordType> coords = new HashMap<>();
         
         public CoordSpec() {
         }
@@ -307,7 +307,7 @@
             }
             return;
         }
-        String batchGroup = _batchGroupIds.get(tuple.getSourceGlobalStreamid());
+        String batchGroup = _batchGroupIds.get(tuple.getSourceGlobalStreamId());
         if(batchGroup==null) {
             // this is so we can do things like have simple DRPC that doesn't need to use batch processing
             _coordCollector.setCurrBatch(null);
@@ -317,7 +317,7 @@
         }
         IBatchID id = (IBatchID) tuple.getValue(0);
         //get transaction id
-        //if it already exissts and attempt id is greater than the attempt there
+        //if it already exists and attempt id is greater than the attempt there
         
         
         TrackedBatch tracked = (TrackedBatch) _batches.get(id.getId());
@@ -407,7 +407,7 @@
     private TupleType getTupleType(Tuple tuple, TrackedBatch batch) {
         CoordCondition cond = batch.condition;
         if(cond.commitStream!=null
-                && tuple.getSourceGlobalStreamid().equals(cond.commitStream)) {
+                && tuple.getSourceGlobalStreamId().equals(cond.commitStream)) {
             return TupleType.COMMIT;
         } else if(cond.expectedTaskReports > 0
                 && tuple.getSourceStreamId().startsWith(COORD_STREAM_PREFIX)) {
diff --git a/storm-core/src/jvm/storm/trident/topology/TridentTopologyBuilder.java b/storm-core/src/jvm/storm/trident/topology/TridentTopologyBuilder.java
index 226a11a..15b5296 100644
--- a/storm-core/src/jvm/storm/trident/topology/TridentTopologyBuilder.java
+++ b/storm-core/src/jvm/storm/trident/topology/TridentTopologyBuilder.java
@@ -102,7 +102,7 @@
     }
     
     Map<GlobalStreamId, String> fleshOutStreamBatchIds(boolean includeCommitStream) {
-        Map<GlobalStreamId, String> ret = new HashMap<GlobalStreamId, String>(_batchIds);
+        Map<GlobalStreamId, String> ret = new HashMap<>(_batchIds);
         Set<String> allBatches = new HashSet(_batchIds.values());
         for(String b: allBatches) {
             ret.put(new GlobalStreamId(masterCoordinator(b), MasterBatchCoordinator.BATCH_STREAM_ID), b);
@@ -134,8 +134,8 @@
         Map<GlobalStreamId, String> batchIdsForSpouts = fleshOutStreamBatchIds(false);
         Map<GlobalStreamId, String> batchIdsForBolts = fleshOutStreamBatchIds(true);
 
-        Map<String, List<String>> batchesToCommitIds = new HashMap<String, List<String>>();
-        Map<String, List<ITridentSpout>> batchesToSpouts = new HashMap<String, List<ITridentSpout>>();
+        Map<String, List<String>> batchesToCommitIds = new HashMap<>();
+        Map<String, List<ITridentSpout>> batchesToSpouts = new HashMap<>();
         
         for(String id: _spouts.keySet()) {
             TransactionalSpoutComponent c = _spouts.get(id);
@@ -196,8 +196,9 @@
             }
         }
         
-        for(String batch: batchesToCommitIds.keySet()) {
-            List<String> commitIds = batchesToCommitIds.get(batch);
+        for(Map.Entry<String, List<String>> entry: batchesToCommitIds.entrySet()) {
+            String batch = entry.getKey();
+            List<String> commitIds = entry.getValue();
             builder.setSpout(masterCoordinator(batch), new MasterBatchCoordinator(commitIds, batchesToSpouts.get(batch)));
         }
                 
@@ -233,9 +234,9 @@
             }
             
             Map<String, Set<String>> batchToComponents = getBoltBatchToComponentSubscriptions(id);
-            for(String b: batchToComponents.keySet()) {
-                for(String comp: batchToComponents.get(b)) {
-                    d.directGrouping(comp, TridentBoltExecutor.COORD_STREAM(b));
+            for(Map.Entry<String, Set<String>> entry: batchToComponents.entrySet()) {
+                for(String comp: entry.getValue()) {
+                    d.directGrouping(comp, TridentBoltExecutor.COORD_STREAM(entry.getKey()));
                 }
             }
             
@@ -248,8 +249,8 @@
     }
     
     private void markBatchGroups(String component, Map<String, String> batchGroups) {
-        for(String stream: batchGroups.keySet()) {
-            _batchIds.put(new GlobalStreamId(component, stream), batchGroups.get(stream));
+        for(Map.Entry<String, String> entry: batchGroups.entrySet()) {
+            _batchIds.put(new GlobalStreamId(component, entry.getKey()), entry.getValue());
         }
     }
     
@@ -291,7 +292,7 @@
     private static class Component {
         public ITridentBatchBolt bolt;
         public Integer parallelism;
-        public List<InputDeclaration> declarations = new ArrayList<InputDeclaration>();
+        public List<InputDeclaration> declarations = new ArrayList<>();
         public List<Map<String, Object>> componentConfs = new ArrayList<>();
         public Set<String> committerBatches;
         
diff --git a/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java b/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java
index 0be0c3f..761b221 100644
--- a/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java
+++ b/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java
@@ -31,7 +31,9 @@
 import java.util.Set;
 import java.util.Arrays;
 
-//extends abstractlist so that it can be emitted directly as Storm tuples
+/**
+ * Extends AbstractList so that it can be emitted directly as Storm tuples
+ */
 public class TridentTupleView extends AbstractList<Object> implements TridentTuple {
     ValuePointer[] _index;
     Map<String, ValuePointer> _fieldIndex;
@@ -46,7 +48,7 @@
             _parent = parent;
             if(projectFields==null) projectFields = new Fields();
             Map<String, ValuePointer> parentFieldIndex = parent.getFieldIndex();
-            _fieldIndex = new HashMap<String, ValuePointer>();
+            _fieldIndex = new HashMap<>();
             for(String f: projectFields) {
                 _fieldIndex.put(f, parentFieldIndex.get(f));
             }            
@@ -79,7 +81,7 @@
         ValuePointer[] _index;
 
         public FreshOutputFactory(Fields selfFields) {
-            _fieldIndex = new HashMap<String, ValuePointer>();
+            _fieldIndex = new HashMap<>();
             for(int i=0; i<selfFields.size(); i++) {
                 String field = selfFields.get(i);
                 _fieldIndex.put(field, new ValuePointer(0, i, field));
@@ -114,15 +116,15 @@
 
         public OperationOutputFactory(Factory parent, Fields selfFields) {
             _parent = parent;
-            _fieldIndex = new HashMap(parent.getFieldIndex());
+            _fieldIndex = new HashMap<>(parent.getFieldIndex());
             int myIndex = parent.numDelegates();
             for(int i=0; i<selfFields.size(); i++) {
                 String field = selfFields.get(i);
                 _fieldIndex.put(field, new ValuePointer(myIndex, i, field));
             }
-            List<String> myOrder = new ArrayList<String>(parent.getOutputFields());
+            List<String> myOrder = new ArrayList<>(parent.getOutputFields());
             
-            Set<String> parentFieldsSet = new HashSet<String>(myOrder);
+            Set<String> parentFieldsSet = new HashSet<>(myOrder);
             for(String f: selfFields) {
                 if(parentFieldsSet.contains(f)) {
                     throw new IllegalArgumentException(
@@ -192,14 +194,14 @@
     }
     
     private static List<String> indexToFieldsList(ValuePointer[] index) {
-        List<String> ret = new ArrayList<String>();
+        List<String> ret = new ArrayList<>();
         for(ValuePointer p: index) {
             ret.add(p.field);
         }
         return ret;
     }
     
-    public static TridentTupleView EMPTY_TUPLE = new TridentTupleView(null, new ValuePointer[0], new HashMap());
+    public static final TridentTupleView EMPTY_TUPLE = new TridentTupleView(null, new ValuePointer[0], new HashMap());
 
     // index and fieldIndex are precomputed, delegates built up over many operations using persistent data structures
     public TridentTupleView(IPersistentVector delegates, ValuePointer[] index, Map<String, ValuePointer> fieldIndex) {
diff --git a/storm-core/src/jvm/storm/trident/util/TridentUtils.java b/storm-core/src/jvm/storm/trident/util/TridentUtils.java
index 0cdec0a..214bcf2 100644
--- a/storm-core/src/jvm/storm/trident/util/TridentUtils.java
+++ b/storm-core/src/jvm/storm/trident/util/TridentUtils.java
@@ -30,11 +30,10 @@
 import java.util.logging.Level;
 import java.util.logging.Logger;
 import org.apache.thrift.TBase;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
 import org.jgrapht.DirectedGraph;
 
+import backtype.storm.utils.Utils;
+
 public class TridentUtils {
     public static Fields fieldsUnion(Fields... fields) {
         Set<String> ret = new HashSet<String>();
@@ -108,35 +107,11 @@
         return parents.get(0);
     }
     
-    private static ThreadLocal<TSerializer> threadSer = new ThreadLocal<TSerializer>();
-    private static ThreadLocal<TDeserializer> threadDes = new ThreadLocal<TDeserializer>();
-    
     public static byte[] thriftSerialize(TBase t) {
-        try {
-            TSerializer ser = threadSer.get();
-            if (ser == null) {
-                ser = new TSerializer();
-                threadSer.set(ser);
-            } 
-            return ser.serialize(t);
-        } catch (TException e) {
-            throw new RuntimeException(e);
-        }
+        return Utils.thriftSerialize(t);
     }
 
     public static <T> T thriftDeserialize(Class c, byte[] b) {
-        try {
-            T ret = (T) c.newInstance();
-            TDeserializer des = threadDes.get();
-            if (des == null) {
-                des = new TDeserializer();
-                threadDes.set(des);
-            }
-            des.deserialize((TBase) ret, b);
-            return ret;
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-        
+        return Utils.thriftDeserialize(c,b);
     }
 }
diff --git a/storm-core/src/native/worker-launcher/impl/main.c b/storm-core/src/native/worker-launcher/impl/main.c
index 7067cf9..a51f9f9 100644
--- a/storm-core/src/native/worker-launcher/impl/main.c
+++ b/storm-core/src/native/worker-launcher/impl/main.c
@@ -47,6 +47,7 @@
   fprintf(stream, "   initialize stormdist dir: code-dir <code-directory>\n");
   fprintf(stream, "   remove a file/directory: rmr <directory>\n");
   fprintf(stream, "   launch a worker: worker <working-directory> <script-to-run>\n");
+  fprintf(stream, "   launch a profiler: profiler <working-directory> <script-to-run>\n");
   fprintf(stream, "   signal a worker: signal <pid> <signal>\n");
 }
 
@@ -176,6 +177,15 @@
     if (exit_code == 0) {
       exit_code = exec_as_user(working_dir, argv[optind]);
     }
+   } else if (strcasecmp("profiler", command) == 0) {
+    if (argc != 5) {
+      fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 5) for profiler\n",
+	      argc);
+      fflush(ERRORFILE);
+      return INVALID_ARGUMENT_NUMBER;
+    }
+    working_dir = argv[optind++];
+    exit_code = exec_as_user(working_dir, argv[optind]);
   } else if (strcasecmp("signal", command) == 0) {
     if (argc != 5) {
       fprintf(ERRORFILE, "Incorrect number of arguments (%d vs 5) for signal\n",
diff --git a/storm-core/src/native/worker-launcher/impl/worker-launcher.c b/storm-core/src/native/worker-launcher/impl/worker-launcher.c
index 3a6c4b8..5dc90d3 100644
--- a/storm-core/src/native/worker-launcher/impl/worker-launcher.c
+++ b/storm-core/src/native/worker-launcher/impl/worker-launcher.c
@@ -491,8 +491,6 @@
       case FTS_F:         // A regular file
       case FTS_SL:        // A symbolic link
       case FTS_SLNONE:    // A broken symbolic link
-        //TODO it would be good to validate that the file is owned by the correct user first.
-        fprintf(LOGFILE, "visiting: %s\n", entry->fts_path);
         if (setup_stormdist(entry, euser) != 0) {
           exit_code = -1;
         }
@@ -743,6 +741,53 @@
   return -1;
 }
 
+int fork_as_user(const char * working_dir, const char * script_file) {
+  char *script_file_dest = NULL;
+  script_file_dest = get_container_launcher_file(working_dir);
+  if (script_file_dest == NULL) {
+    return OUT_OF_MEMORY;
+  }
+
+  // open launch script
+  int script_file_source = open_file_as_nm(script_file);
+  if (script_file_source == -1) {
+    return -1;
+  }
+
+  setsid();
+
+  // give up root privs
+  if (change_user(user_detail->pw_uid, user_detail->pw_gid) != 0) {
+    return SETUID_OPER_FAILED;
+  }
+
+  if (copy_file(script_file_source, script_file, script_file_dest, S_IRWXU) != 0) {
+    return -1;
+  }
+
+  fcloseall();
+  umask(0027);
+  if (chdir(working_dir) != 0) {
+    fprintf(LOGFILE, "Can't change directory to %s -%s\n", working_dir,
+	    strerror(errno));
+    return -1;
+  }
+
+  int pid = fork();
+  if (pid == 0 && execlp(script_file_dest, script_file_dest, NULL) != 0) {
+    fprintf(LOGFILE, "Couldn't execute the container launch file %s - %s",
+            script_file_dest, strerror(errno));
+    return UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
+  } else {
+    fprintf(LOGFILE, "Launched the process from the container launch file %s - with pid %d",
+            script_file_dest, pid);
+    return 0;
+  }
+
+  //Unreachable
+  return -1;
+}
+
 /**
  * Delete the given directory as the user from each of the directories
  * user: the user doing the delete
diff --git a/storm-core/src/native/worker-launcher/impl/worker-launcher.h b/storm-core/src/native/worker-launcher/impl/worker-launcher.h
index 59ab998..3b1ec24 100644
--- a/storm-core/src/native/worker-launcher/impl/worker-launcher.h
+++ b/storm-core/src/native/worker-launcher/impl/worker-launcher.h
@@ -70,6 +70,8 @@
 
 int exec_as_user(const char * working_dir, const char * args);
 
+int fork_as_user(const char * working_dir, const char * args);
+
 // delete a directory (or file) recursively as the user. The directory
 // could optionally be relative to the baseDir set of directories (if the same
 // directory appears on multiple disk volumes, the disk volumes should be passed
diff --git a/storm-core/src/py/storm/DistributedRPC-remote b/storm-core/src/py/storm/DistributedRPC-remote
index 3d06d07..90f894a 100644
--- a/storm-core/src/py/storm/DistributedRPC-remote
+++ b/storm-core/src/py/storm/DistributedRPC-remote
@@ -18,7 +18,7 @@
 
 #!/usr/bin/env python
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
diff --git a/storm-core/src/py/storm/DistributedRPC.py b/storm-core/src/py/storm/DistributedRPC.py
index 330499c..225c57e 100644
--- a/storm-core/src/py/storm/DistributedRPC.py
+++ b/storm-core/src/py/storm/DistributedRPC.py
@@ -17,7 +17,7 @@
 # limitations under the License.
 
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
@@ -25,6 +25,7 @@
 #
 
 from thrift.Thrift import TType, TMessageType, TException, TApplicationException
+import logging
 from ttypes import *
 from thrift.Thrift import TProcessor
 from thrift.transport import TTransport
@@ -87,7 +88,7 @@
       raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "execute failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "execute failed: unknown result")
 
 
 class Processor(Iface, TProcessor):
@@ -118,11 +119,20 @@
     result = execute_result()
     try:
       result.success = self._handler.execute(args.functionName, args.funcArgs)
-    except DRPCExecutionException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except DRPCExecutionException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("execute", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("execute", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
diff --git a/storm-core/src/py/storm/DistributedRPCInvocations-remote b/storm-core/src/py/storm/DistributedRPCInvocations-remote
index 9dd50cd..01435b6 100644
--- a/storm-core/src/py/storm/DistributedRPCInvocations-remote
+++ b/storm-core/src/py/storm/DistributedRPCInvocations-remote
@@ -18,7 +18,7 @@
 
 #!/usr/bin/env python
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
diff --git a/storm-core/src/py/storm/DistributedRPCInvocations.py b/storm-core/src/py/storm/DistributedRPCInvocations.py
index 493fcc7..207fa9d 100644
--- a/storm-core/src/py/storm/DistributedRPCInvocations.py
+++ b/storm-core/src/py/storm/DistributedRPCInvocations.py
@@ -17,7 +17,7 @@
 # limitations under the License.
 
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
@@ -25,6 +25,7 @@
 #
 
 from thrift.Thrift import TType, TMessageType, TException, TApplicationException
+import logging
 from ttypes import *
 from thrift.Thrift import TProcessor
 from thrift.transport import TTransport
@@ -130,7 +131,7 @@
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result")
 
   def failRequest(self, id):
     """
@@ -194,9 +195,17 @@
     result = result_result()
     try:
       self._handler.result(args.id, args.result)
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("result", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("result", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -208,9 +217,17 @@
     result = fetchRequest_result()
     try:
       result.success = self._handler.fetchRequest(args.functionName)
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("fetchRequest", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("fetchRequest", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -222,9 +239,17 @@
     result = failRequest_result()
     try:
       self._handler.failRequest(args.id)
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("failRequest", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("failRequest", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
diff --git a/storm-core/src/py/storm/Nimbus-remote b/storm-core/src/py/storm/Nimbus-remote
index fff63f9..63e7dce 100644
--- a/storm-core/src/py/storm/Nimbus-remote
+++ b/storm-core/src/py/storm/Nimbus-remote
@@ -18,7 +18,7 @@
 
 #!/usr/bin/env python
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
@@ -49,7 +49,11 @@
   print('  void activate(string name)')
   print('  void deactivate(string name)')
   print('  void rebalance(string name, RebalanceOptions options)')
+  print('  void setLogConfig(string name, LogConfig config)')
+  print('  LogConfig getLogConfig(string name)')
   print('  void debug(string name, string component, bool enable, double samplingPercentage)')
+  print('  void setWorkerProfiler(string id, ProfileRequest profileRequest)')
+  print('   getComponentPendingProfileActions(string id, string component_id, ProfileAction action)')
   print('  void uploadNewCredentials(string name, Credentials creds)')
   print('  string beginFileUpload()')
   print('  void uploadChunk(string location, string chunk)')
@@ -60,9 +64,12 @@
   print('  ClusterSummary getClusterInfo()')
   print('  TopologyInfo getTopologyInfo(string id)')
   print('  TopologyInfo getTopologyInfoWithOpts(string id, GetInfoOptions options)')
+  print('  TopologyPageInfo getTopologyPageInfo(string id, string window, bool is_include_sys)')
+  print('  ComponentPageInfo getComponentPageInfo(string topology_id, string component_id, string window, bool is_include_sys)')
   print('  string getTopologyConf(string id)')
   print('  StormTopology getTopology(string id)')
   print('  StormTopology getUserTopology(string id)')
+  print('  TopologyHistoryInfo getTopologyHistory(string user)')
   print('')
   sys.exit(0)
 
@@ -161,12 +168,36 @@
     sys.exit(1)
   pp.pprint(client.rebalance(args[0],eval(args[1]),))
 
+elif cmd == 'setLogConfig':
+  if len(args) != 2:
+    print('setLogConfig requires 2 args')
+    sys.exit(1)
+  pp.pprint(client.setLogConfig(args[0],eval(args[1]),))
+
+elif cmd == 'getLogConfig':
+  if len(args) != 1:
+    print('getLogConfig requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.getLogConfig(args[0],))
+
 elif cmd == 'debug':
   if len(args) != 4:
     print('debug requires 4 args')
     sys.exit(1)
   pp.pprint(client.debug(args[0],args[1],eval(args[2]),eval(args[3]),))
 
+elif cmd == 'setWorkerProfiler':
+  if len(args) != 2:
+    print('setWorkerProfiler requires 2 args')
+    sys.exit(1)
+  pp.pprint(client.setWorkerProfiler(args[0],eval(args[1]),))
+
+elif cmd == 'getComponentPendingProfileActions':
+  if len(args) != 3:
+    print('getComponentPendingProfileActions requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.getComponentPendingProfileActions(args[0],args[1],eval(args[2]),))
+
 elif cmd == 'uploadNewCredentials':
   if len(args) != 2:
     print('uploadNewCredentials requires 2 args')
@@ -227,6 +258,18 @@
     sys.exit(1)
   pp.pprint(client.getTopologyInfoWithOpts(args[0],eval(args[1]),))
 
+elif cmd == 'getTopologyPageInfo':
+  if len(args) != 3:
+    print('getTopologyPageInfo requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.getTopologyPageInfo(args[0],args[1],eval(args[2]),))
+
+elif cmd == 'getComponentPageInfo':
+  if len(args) != 4:
+    print('getComponentPageInfo requires 4 args')
+    sys.exit(1)
+  pp.pprint(client.getComponentPageInfo(args[0],args[1],args[2],eval(args[3]),))
+
 elif cmd == 'getTopologyConf':
   if len(args) != 1:
     print('getTopologyConf requires 1 args')
@@ -245,6 +288,12 @@
     sys.exit(1)
   pp.pprint(client.getUserTopology(args[0],))
 
+elif cmd == 'getTopologyHistory':
+  if len(args) != 1:
+    print('getTopologyHistory requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.getTopologyHistory(args[0],))
+
 else:
   print('Unrecognized method %s' % cmd)
   sys.exit(1)
diff --git a/storm-core/src/py/storm/Nimbus.py b/storm-core/src/py/storm/Nimbus.py
index 0caeed0..c2bb9ac 100644
--- a/storm-core/src/py/storm/Nimbus.py
+++ b/storm-core/src/py/storm/Nimbus.py
@@ -17,7 +17,7 @@
 # limitations under the License.
 
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
@@ -25,6 +25,7 @@
 #
 
 from thrift.Thrift import TType, TMessageType, TException, TApplicationException
+import logging
 from ttypes import *
 from thrift.Thrift import TProcessor
 from thrift.transport import TTransport
@@ -94,6 +95,21 @@
     """
     pass
 
+  def setLogConfig(self, name, config):
+    """
+    Parameters:
+     - name
+     - config
+    """
+    pass
+
+  def getLogConfig(self, name):
+    """
+    Parameters:
+     - name
+    """
+    pass
+
   def debug(self, name, component, enable, samplingPercentage):
     """
     Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
@@ -102,7 +118,6 @@
     The 'samplingPercentage' will limit loggging to a percentage of generated tuples.
 
 
-
     Parameters:
      - name
      - component
@@ -111,6 +126,23 @@
     """
     pass
 
+  def setWorkerProfiler(self, id, profileRequest):
+    """
+    Parameters:
+     - id
+     - profileRequest
+    """
+    pass
+
+  def getComponentPendingProfileActions(self, id, component_id, action):
+    """
+    Parameters:
+     - id
+     - component_id
+     - action
+    """
+    pass
+
   def uploadNewCredentials(self, name, creds):
     """
     Parameters:
@@ -172,6 +204,25 @@
     """
     pass
 
+  def getTopologyPageInfo(self, id, window, is_include_sys):
+    """
+    Parameters:
+     - id
+     - window
+     - is_include_sys
+    """
+    pass
+
+  def getComponentPageInfo(self, topology_id, component_id, window, is_include_sys):
+    """
+    Parameters:
+     - topology_id
+     - component_id
+     - window
+     - is_include_sys
+    """
+    pass
+
   def getTopologyConf(self, id):
     """
     Parameters:
@@ -197,6 +248,13 @@
     """
     pass
 
+  def getTopologyHistory(self, user):
+    """
+    Parameters:
+     - user
+    """
+    pass
+
 
 class Client(Iface):
   def __init__(self, iprot, oprot=None):
@@ -460,12 +518,74 @@
       raise result.aze
     return
 
+  def setLogConfig(self, name, config):
+    """
+    Parameters:
+     - name
+     - config
+    """
+    self.send_setLogConfig(name, config)
+    self.recv_setLogConfig()
+
+  def send_setLogConfig(self, name, config):
+    self._oprot.writeMessageBegin('setLogConfig', TMessageType.CALL, self._seqid)
+    args = setLogConfig_args()
+    args.name = name
+    args.config = config
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_setLogConfig(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = setLogConfig_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    return
+
+  def getLogConfig(self, name):
+    """
+    Parameters:
+     - name
+    """
+    self.send_getLogConfig(name)
+    return self.recv_getLogConfig()
+
+  def send_getLogConfig(self, name):
+    self._oprot.writeMessageBegin('getLogConfig', TMessageType.CALL, self._seqid)
+    args = getLogConfig_args()
+    args.name = name
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getLogConfig(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getLogConfig_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getLogConfig failed: unknown result")
+
   def debug(self, name, component, enable, samplingPercentage):
     """
     Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
     and if null or empty, the debug flag will apply to the entire topology.
 
-    If 'samplingPercentage' is specified, it will limit loggging to a percentage of generated tuples. The default is to log all (100 pct).
+    The 'samplingPercentage' will limit loggging to a percentage of generated tuples.
 
 
     Parameters:
@@ -505,6 +625,72 @@
       raise result.aze
     return
 
+  def setWorkerProfiler(self, id, profileRequest):
+    """
+    Parameters:
+     - id
+     - profileRequest
+    """
+    self.send_setWorkerProfiler(id, profileRequest)
+    self.recv_setWorkerProfiler()
+
+  def send_setWorkerProfiler(self, id, profileRequest):
+    self._oprot.writeMessageBegin('setWorkerProfiler', TMessageType.CALL, self._seqid)
+    args = setWorkerProfiler_args()
+    args.id = id
+    args.profileRequest = profileRequest
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_setWorkerProfiler(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = setWorkerProfiler_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    return
+
+  def getComponentPendingProfileActions(self, id, component_id, action):
+    """
+    Parameters:
+     - id
+     - component_id
+     - action
+    """
+    self.send_getComponentPendingProfileActions(id, component_id, action)
+    return self.recv_getComponentPendingProfileActions()
+
+  def send_getComponentPendingProfileActions(self, id, component_id, action):
+    self._oprot.writeMessageBegin('getComponentPendingProfileActions', TMessageType.CALL, self._seqid)
+    args = getComponentPendingProfileActions_args()
+    args.id = id
+    args.component_id = component_id
+    args.action = action
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getComponentPendingProfileActions(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getComponentPendingProfileActions_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getComponentPendingProfileActions failed: unknown result")
+
   def uploadNewCredentials(self, name, creds):
     """
     Parameters:
@@ -568,7 +754,7 @@
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result")
 
   def uploadChunk(self, location, chunk):
     """
@@ -665,7 +851,7 @@
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result")
 
   def downloadChunk(self, id):
     """
@@ -698,7 +884,7 @@
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result")
 
   def getNimbusConf(self):
     self.send_getNimbusConf()
@@ -726,7 +912,7 @@
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result")
 
   def getClusterInfo(self):
     self.send_getClusterInfo()
@@ -754,7 +940,7 @@
       return result.success
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result")
 
   def getTopologyInfo(self, id):
     """
@@ -789,7 +975,7 @@
       raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result")
 
   def getTopologyInfoWithOpts(self, id, options):
     """
@@ -826,7 +1012,87 @@
       raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfoWithOpts failed: unknown result")
+
+  def getTopologyPageInfo(self, id, window, is_include_sys):
+    """
+    Parameters:
+     - id
+     - window
+     - is_include_sys
+    """
+    self.send_getTopologyPageInfo(id, window, is_include_sys)
+    return self.recv_getTopologyPageInfo()
+
+  def send_getTopologyPageInfo(self, id, window, is_include_sys):
+    self._oprot.writeMessageBegin('getTopologyPageInfo', TMessageType.CALL, self._seqid)
+    args = getTopologyPageInfo_args()
+    args.id = id
+    args.window = window
+    args.is_include_sys = is_include_sys
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopologyPageInfo(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getTopologyPageInfo_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyPageInfo failed: unknown result")
+
+  def getComponentPageInfo(self, topology_id, component_id, window, is_include_sys):
+    """
+    Parameters:
+     - topology_id
+     - component_id
+     - window
+     - is_include_sys
+    """
+    self.send_getComponentPageInfo(topology_id, component_id, window, is_include_sys)
+    return self.recv_getComponentPageInfo()
+
+  def send_getComponentPageInfo(self, topology_id, component_id, window, is_include_sys):
+    self._oprot.writeMessageBegin('getComponentPageInfo', TMessageType.CALL, self._seqid)
+    args = getComponentPageInfo_args()
+    args.topology_id = topology_id
+    args.component_id = component_id
+    args.window = window
+    args.is_include_sys = is_include_sys
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getComponentPageInfo(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getComponentPageInfo_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.e is not None:
+      raise result.e
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getComponentPageInfo failed: unknown result")
 
   def getTopologyConf(self, id):
     """
@@ -861,7 +1127,7 @@
       raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result")
 
   def getTopology(self, id):
     """
@@ -898,7 +1164,7 @@
       raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result")
 
   def getUserTopology(self, id):
     """
@@ -935,7 +1201,40 @@
       raise result.e
     if result.aze is not None:
       raise result.aze
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result");
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result")
+
+  def getTopologyHistory(self, user):
+    """
+    Parameters:
+     - user
+    """
+    self.send_getTopologyHistory(user)
+    return self.recv_getTopologyHistory()
+
+  def send_getTopologyHistory(self, user):
+    self._oprot.writeMessageBegin('getTopologyHistory', TMessageType.CALL, self._seqid)
+    args = getTopologyHistory_args()
+    args.user = user
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopologyHistory(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = getTopologyHistory_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.aze is not None:
+      raise result.aze
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyHistory failed: unknown result")
 
 
 class Processor(Iface, TProcessor):
@@ -949,7 +1248,11 @@
     self._processMap["activate"] = Processor.process_activate
     self._processMap["deactivate"] = Processor.process_deactivate
     self._processMap["rebalance"] = Processor.process_rebalance
+    self._processMap["setLogConfig"] = Processor.process_setLogConfig
+    self._processMap["getLogConfig"] = Processor.process_getLogConfig
     self._processMap["debug"] = Processor.process_debug
+    self._processMap["setWorkerProfiler"] = Processor.process_setWorkerProfiler
+    self._processMap["getComponentPendingProfileActions"] = Processor.process_getComponentPendingProfileActions
     self._processMap["uploadNewCredentials"] = Processor.process_uploadNewCredentials
     self._processMap["beginFileUpload"] = Processor.process_beginFileUpload
     self._processMap["uploadChunk"] = Processor.process_uploadChunk
@@ -960,9 +1263,12 @@
     self._processMap["getClusterInfo"] = Processor.process_getClusterInfo
     self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo
     self._processMap["getTopologyInfoWithOpts"] = Processor.process_getTopologyInfoWithOpts
+    self._processMap["getTopologyPageInfo"] = Processor.process_getTopologyPageInfo
+    self._processMap["getComponentPageInfo"] = Processor.process_getComponentPageInfo
     self._processMap["getTopologyConf"] = Processor.process_getTopologyConf
     self._processMap["getTopology"] = Processor.process_getTopology
     self._processMap["getUserTopology"] = Processor.process_getUserTopology
+    self._processMap["getTopologyHistory"] = Processor.process_getTopologyHistory
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -986,13 +1292,23 @@
     result = submitTopology_result()
     try:
       self._handler.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology)
-    except AlreadyAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except InvalidTopologyException, ite:
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
       result.ite = ite
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("submitTopology", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("submitTopology", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1004,13 +1320,23 @@
     result = submitTopologyWithOpts_result()
     try:
       self._handler.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options)
-    except AlreadyAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except InvalidTopologyException, ite:
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
       result.ite = ite
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("submitTopologyWithOpts", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("submitTopologyWithOpts", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1022,11 +1348,20 @@
     result = killTopology_result()
     try:
       self._handler.killTopology(args.name)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("killTopology", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("killTopology", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1038,11 +1373,20 @@
     result = killTopologyWithOpts_result()
     try:
       self._handler.killTopologyWithOpts(args.name, args.options)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("killTopologyWithOpts", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("killTopologyWithOpts", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1054,11 +1398,20 @@
     result = activate_result()
     try:
       self._handler.activate(args.name)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("activate", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("activate", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1070,11 +1423,20 @@
     result = deactivate_result()
     try:
       self._handler.deactivate(args.name)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("deactivate", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("deactivate", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1086,13 +1448,61 @@
     result = rebalance_result()
     try:
       self._handler.rebalance(args.name, args.options)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except InvalidTopologyException, ite:
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
       result.ite = ite
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("rebalance", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("rebalance", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_setLogConfig(self, seqid, iprot, oprot):
+    args = setLogConfig_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = setLogConfig_result()
+    try:
+      self._handler.setLogConfig(args.name, args.config)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("setLogConfig", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getLogConfig(self, seqid, iprot, oprot):
+    args = getLogConfig_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getLogConfig_result()
+    try:
+      result.success = self._handler.getLogConfig(args.name)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getLogConfig", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1104,11 +1514,58 @@
     result = debug_result()
     try:
       self._handler.debug(args.name, args.component, args.enable, args.samplingPercentage)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("debug", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("debug", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_setWorkerProfiler(self, seqid, iprot, oprot):
+    args = setWorkerProfiler_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = setWorkerProfiler_result()
+    try:
+      self._handler.setWorkerProfiler(args.id, args.profileRequest)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("setWorkerProfiler", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getComponentPendingProfileActions(self, seqid, iprot, oprot):
+    args = getComponentPendingProfileActions_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getComponentPendingProfileActions_result()
+    try:
+      result.success = self._handler.getComponentPendingProfileActions(args.id, args.component_id, args.action)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getComponentPendingProfileActions", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1120,13 +1577,23 @@
     result = uploadNewCredentials_result()
     try:
       self._handler.uploadNewCredentials(args.name, args.creds)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except InvalidTopologyException, ite:
+    except InvalidTopologyException as ite:
+      msg_type = TMessageType.REPLY
       result.ite = ite
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("uploadNewCredentials", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("uploadNewCredentials", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1138,9 +1605,17 @@
     result = beginFileUpload_result()
     try:
       result.success = self._handler.beginFileUpload()
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("beginFileUpload", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("beginFileUpload", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1152,9 +1627,17 @@
     result = uploadChunk_result()
     try:
       self._handler.uploadChunk(args.location, args.chunk)
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("uploadChunk", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("uploadChunk", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1166,9 +1649,17 @@
     result = finishFileUpload_result()
     try:
       self._handler.finishFileUpload(args.location)
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("finishFileUpload", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("finishFileUpload", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1180,9 +1671,17 @@
     result = beginFileDownload_result()
     try:
       result.success = self._handler.beginFileDownload(args.file)
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("beginFileDownload", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("beginFileDownload", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1194,9 +1693,17 @@
     result = downloadChunk_result()
     try:
       result.success = self._handler.downloadChunk(args.id)
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("downloadChunk", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("downloadChunk", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1208,9 +1715,17 @@
     result = getNimbusConf_result()
     try:
       result.success = self._handler.getNimbusConf()
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("getNimbusConf", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getNimbusConf", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1222,9 +1737,17 @@
     result = getClusterInfo_result()
     try:
       result.success = self._handler.getClusterInfo()
-    except AuthorizationException, aze:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("getClusterInfo", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getClusterInfo", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1236,11 +1759,20 @@
     result = getTopologyInfo_result()
     try:
       result.success = self._handler.getTopologyInfo(args.id)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("getTopologyInfo", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getTopologyInfo", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1252,11 +1784,70 @@
     result = getTopologyInfoWithOpts_result()
     try:
       result.success = self._handler.getTopologyInfoWithOpts(args.id, args.options)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("getTopologyInfoWithOpts", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getTopologyInfoWithOpts", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getTopologyPageInfo(self, seqid, iprot, oprot):
+    args = getTopologyPageInfo_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getTopologyPageInfo_result()
+    try:
+      result.success = self._handler.getTopologyPageInfo(args.id, args.window, args.is_include_sys)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getTopologyPageInfo", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getComponentPageInfo(self, seqid, iprot, oprot):
+    args = getComponentPageInfo_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getComponentPageInfo_result()
+    try:
+      result.success = self._handler.getComponentPageInfo(args.topology_id, args.component_id, args.window, args.is_include_sys)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
+      result.e = e
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getComponentPageInfo", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1268,11 +1859,20 @@
     result = getTopologyConf_result()
     try:
       result.success = self._handler.getTopologyConf(args.id)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("getTopologyConf", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getTopologyConf", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1284,11 +1884,20 @@
     result = getTopology_result()
     try:
       result.success = self._handler.getTopology(args.id)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("getTopology", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getTopology", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -1300,11 +1909,42 @@
     result = getUserTopology_result()
     try:
       result.success = self._handler.getUserTopology(args.id)
-    except NotAliveException, e:
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NotAliveException as e:
+      msg_type = TMessageType.REPLY
       result.e = e
-    except AuthorizationException, aze:
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
       result.aze = aze
-    oprot.writeMessageBegin("getUserTopology", TMessageType.REPLY, seqid)
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getUserTopology", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getTopologyHistory(self, seqid, iprot, oprot):
+    args = getTopologyHistory_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getTopologyHistory_result()
+    try:
+      result.success = self._handler.getTopologyHistory(args.user)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AuthorizationException as aze:
+      msg_type = TMessageType.REPLY
+      result.aze = aze
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("getTopologyHistory", msg_type, seqid)
     result.write(oprot)
     oprot.writeMessageEnd()
     oprot.trans.flush()
@@ -2491,6 +3131,261 @@
   def __ne__(self, other):
     return not (self == other)
 
+class setLogConfig_args:
+  """
+  Attributes:
+   - name
+   - config
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.STRUCT, 'config', (LogConfig, LogConfig.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, name=None, config=None,):
+    self.name = name
+    self.config = config
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.config = LogConfig()
+          self.config.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('setLogConfig_args')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.config is not None:
+      oprot.writeFieldBegin('config', TType.STRUCT, 2)
+      self.config.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.config)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class setLogConfig_result:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('setLogConfig_result')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getLogConfig_args:
+  """
+  Attributes:
+   - name
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+  )
+
+  def __init__(self, name=None,):
+    self.name = name
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getLogConfig_args')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getLogConfig_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (LogConfig, LogConfig.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = LogConfig()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getLogConfig_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class debug_args:
   """
   Attributes:
@@ -2535,12 +3430,12 @@
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.BOOL:
-          self.enable = iprot.readBool();
+          self.enable = iprot.readBool()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.DOUBLE:
-          self.samplingPercentage = iprot.readDouble();
+          self.samplingPercentage = iprot.readDouble()
         else:
           iprot.skip(ftype)
       else:
@@ -2675,6 +3570,295 @@
   def __ne__(self, other):
     return not (self == other)
 
+class setWorkerProfiler_args:
+  """
+  Attributes:
+   - id
+   - profileRequest
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRUCT, 'profileRequest', (ProfileRequest, ProfileRequest.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, id=None, profileRequest=None,):
+    self.id = id
+    self.profileRequest = profileRequest
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.profileRequest = ProfileRequest()
+          self.profileRequest.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('setWorkerProfiler_args')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.profileRequest is not None:
+      oprot.writeFieldBegin('profileRequest', TType.STRUCT, 2)
+      self.profileRequest.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.id)
+    value = (value * 31) ^ hash(self.profileRequest)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class setWorkerProfiler_result:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('setWorkerProfiler_result')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getComponentPendingProfileActions_args:
+  """
+  Attributes:
+   - id
+   - component_id
+   - action
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRING, 'component_id', None, None, ), # 2
+    (3, TType.I32, 'action', None, None, ), # 3
+  )
+
+  def __init__(self, id=None, component_id=None, action=None,):
+    self.id = id
+    self.component_id = component_id
+    self.action = action
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.component_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.action = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getComponentPendingProfileActions_args')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.component_id is not None:
+      oprot.writeFieldBegin('component_id', TType.STRING, 2)
+      oprot.writeString(self.component_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.action is not None:
+      oprot.writeFieldBegin('action', TType.I32, 3)
+      oprot.writeI32(self.action)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.id)
+    value = (value * 31) ^ hash(self.component_id)
+    value = (value * 31) ^ hash(self.action)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getComponentPendingProfileActions_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.LIST, 'success', (TType.STRUCT,(ProfileRequest, ProfileRequest.thrift_spec)), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.LIST:
+          self.success = []
+          (_etype648, _size645) = iprot.readListBegin()
+          for _i649 in xrange(_size645):
+            _elem650 = ProfileRequest()
+            _elem650.read(iprot)
+            self.success.append(_elem650)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getComponentPendingProfileActions_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.LIST, 0)
+      oprot.writeListBegin(TType.STRUCT, len(self.success))
+      for iter651 in self.success:
+        iter651.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class uploadNewCredentials_args:
   """
   Attributes:
@@ -3005,7 +4189,7 @@
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.chunk = iprot.readString();
+          self.chunk = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -3482,7 +4666,7 @@
         break
       if fid == 0:
         if ftype == TType.STRING:
-          self.success = iprot.readString();
+          self.success = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 1:
@@ -4112,6 +5296,387 @@
   def __ne__(self, other):
     return not (self == other)
 
+class getTopologyPageInfo_args:
+  """
+  Attributes:
+   - id
+   - window
+   - is_include_sys
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRING, 'window', None, None, ), # 2
+    (3, TType.BOOL, 'is_include_sys', None, None, ), # 3
+  )
+
+  def __init__(self, id=None, window=None, is_include_sys=None,):
+    self.id = id
+    self.window = window
+    self.is_include_sys = is_include_sys
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.window = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.BOOL:
+          self.is_include_sys = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyPageInfo_args')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.window is not None:
+      oprot.writeFieldBegin('window', TType.STRING, 2)
+      oprot.writeString(self.window.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.is_include_sys is not None:
+      oprot.writeFieldBegin('is_include_sys', TType.BOOL, 3)
+      oprot.writeBool(self.is_include_sys)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.id)
+    value = (value * 31) ^ hash(self.window)
+    value = (value * 31) ^ hash(self.is_include_sys)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getTopologyPageInfo_result:
+  """
+  Attributes:
+   - success
+   - e
+   - aze
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (TopologyPageInfo, TopologyPageInfo.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, success=None, e=None, aze=None,):
+    self.success = success
+    self.e = e
+    self.aze = aze
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = TopologyPageInfo()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = NotAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyPageInfo_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.e is not None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.e)
+    value = (value * 31) ^ hash(self.aze)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getComponentPageInfo_args:
+  """
+  Attributes:
+   - topology_id
+   - component_id
+   - window
+   - is_include_sys
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'topology_id', None, None, ), # 1
+    (2, TType.STRING, 'component_id', None, None, ), # 2
+    (3, TType.STRING, 'window', None, None, ), # 3
+    (4, TType.BOOL, 'is_include_sys', None, None, ), # 4
+  )
+
+  def __init__(self, topology_id=None, component_id=None, window=None, is_include_sys=None,):
+    self.topology_id = topology_id
+    self.component_id = component_id
+    self.window = window
+    self.is_include_sys = is_include_sys
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.topology_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.component_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.window = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.BOOL:
+          self.is_include_sys = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getComponentPageInfo_args')
+    if self.topology_id is not None:
+      oprot.writeFieldBegin('topology_id', TType.STRING, 1)
+      oprot.writeString(self.topology_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.component_id is not None:
+      oprot.writeFieldBegin('component_id', TType.STRING, 2)
+      oprot.writeString(self.component_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.window is not None:
+      oprot.writeFieldBegin('window', TType.STRING, 3)
+      oprot.writeString(self.window.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.is_include_sys is not None:
+      oprot.writeFieldBegin('is_include_sys', TType.BOOL, 4)
+      oprot.writeBool(self.is_include_sys)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.topology_id)
+    value = (value * 31) ^ hash(self.component_id)
+    value = (value * 31) ^ hash(self.window)
+    value = (value * 31) ^ hash(self.is_include_sys)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getComponentPageInfo_result:
+  """
+  Attributes:
+   - success
+   - e
+   - aze
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (ComponentPageInfo, ComponentPageInfo.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, success=None, e=None, aze=None,):
+    self.success = success
+    self.e = e
+    self.aze = aze
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = ComponentPageInfo()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = NotAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getComponentPageInfo_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.e is not None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 2)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.e)
+    value = (value * 31) ^ hash(self.aze)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class getTopologyConf_args:
   """
   Attributes:
@@ -4584,3 +6149,147 @@
 
   def __ne__(self, other):
     return not (self == other)
+
+class getTopologyHistory_args:
+  """
+  Attributes:
+   - user
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'user', None, None, ), # 1
+  )
+
+  def __init__(self, user=None,):
+    self.user = user
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.user = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyHistory_args')
+    if self.user is not None:
+      oprot.writeFieldBegin('user', TType.STRING, 1)
+      oprot.writeString(self.user.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.user)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class getTopologyHistory_result:
+  """
+  Attributes:
+   - success
+   - aze
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (TopologyHistoryInfo, TopologyHistoryInfo.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'aze', (AuthorizationException, AuthorizationException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, aze=None,):
+    self.success = success
+    self.aze = aze
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = TopologyHistoryInfo()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.aze = AuthorizationException()
+          self.aze.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyHistory_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.aze is not None:
+      oprot.writeFieldBegin('aze', TType.STRUCT, 1)
+      self.aze.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.aze)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
diff --git a/storm-core/src/py/storm/constants.py b/storm-core/src/py/storm/constants.py
index 3f0c64a..b403f97 100644
--- a/storm-core/src/py/storm/constants.py
+++ b/storm-core/src/py/storm/constants.py
@@ -17,7 +17,7 @@
 # limitations under the License.
 
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
diff --git a/storm-core/src/py/storm/ttypes.py b/storm-core/src/py/storm/ttypes.py
index 4f48449..a730c13 100644
--- a/storm-core/src/py/storm/ttypes.py
+++ b/storm-core/src/py/storm/ttypes.py
@@ -17,7 +17,7 @@
 # limitations under the License.
 
 #
-# Autogenerated by Thrift Compiler (0.9.2)
+# Autogenerated by Thrift Compiler (0.9.3)
 #
 # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 #
@@ -34,6 +34,20 @@
   fastbinary = None
 
 
+class ComponentType:
+  BOLT = 1
+  SPOUT = 2
+
+  _VALUES_TO_NAMES = {
+    1: "BOLT",
+    2: "SPOUT",
+  }
+
+  _NAMES_TO_VALUES = {
+    "BOLT": 1,
+    "SPOUT": 2,
+  }
+
 class TopologyInitialStatus:
   ACTIVE = 1
   INACTIVE = 2
@@ -85,6 +99,114 @@
     "ONE": 2,
   }
 
+class ProfileAction:
+  JPROFILE_STOP = 0
+  JPROFILE_START = 1
+  JPROFILE_DUMP = 2
+  JMAP_DUMP = 3
+  JSTACK_DUMP = 4
+  JVM_RESTART = 5
+
+  _VALUES_TO_NAMES = {
+    0: "JPROFILE_STOP",
+    1: "JPROFILE_START",
+    2: "JPROFILE_DUMP",
+    3: "JMAP_DUMP",
+    4: "JSTACK_DUMP",
+    5: "JVM_RESTART",
+  }
+
+  _NAMES_TO_VALUES = {
+    "JPROFILE_STOP": 0,
+    "JPROFILE_START": 1,
+    "JPROFILE_DUMP": 2,
+    "JMAP_DUMP": 3,
+    "JSTACK_DUMP": 4,
+    "JVM_RESTART": 5,
+  }
+
+class LogLevelAction:
+  UNCHANGED = 1
+  UPDATE = 2
+  REMOVE = 3
+
+  _VALUES_TO_NAMES = {
+    1: "UNCHANGED",
+    2: "UPDATE",
+    3: "REMOVE",
+  }
+
+  _NAMES_TO_VALUES = {
+    "UNCHANGED": 1,
+    "UPDATE": 2,
+    "REMOVE": 3,
+  }
+
+class HBServerMessageType:
+  CREATE_PATH = 0
+  CREATE_PATH_RESPONSE = 1
+  EXISTS = 2
+  EXISTS_RESPONSE = 3
+  SEND_PULSE = 4
+  SEND_PULSE_RESPONSE = 5
+  GET_ALL_PULSE_FOR_PATH = 6
+  GET_ALL_PULSE_FOR_PATH_RESPONSE = 7
+  GET_ALL_NODES_FOR_PATH = 8
+  GET_ALL_NODES_FOR_PATH_RESPONSE = 9
+  GET_PULSE = 10
+  GET_PULSE_RESPONSE = 11
+  DELETE_PATH = 12
+  DELETE_PATH_RESPONSE = 13
+  DELETE_PULSE_ID = 14
+  DELETE_PULSE_ID_RESPONSE = 15
+  CONTROL_MESSAGE = 16
+  SASL_MESSAGE_TOKEN = 17
+  NOT_AUTHORIZED = 18
+
+  _VALUES_TO_NAMES = {
+    0: "CREATE_PATH",
+    1: "CREATE_PATH_RESPONSE",
+    2: "EXISTS",
+    3: "EXISTS_RESPONSE",
+    4: "SEND_PULSE",
+    5: "SEND_PULSE_RESPONSE",
+    6: "GET_ALL_PULSE_FOR_PATH",
+    7: "GET_ALL_PULSE_FOR_PATH_RESPONSE",
+    8: "GET_ALL_NODES_FOR_PATH",
+    9: "GET_ALL_NODES_FOR_PATH_RESPONSE",
+    10: "GET_PULSE",
+    11: "GET_PULSE_RESPONSE",
+    12: "DELETE_PATH",
+    13: "DELETE_PATH_RESPONSE",
+    14: "DELETE_PULSE_ID",
+    15: "DELETE_PULSE_ID_RESPONSE",
+    16: "CONTROL_MESSAGE",
+    17: "SASL_MESSAGE_TOKEN",
+    18: "NOT_AUTHORIZED",
+  }
+
+  _NAMES_TO_VALUES = {
+    "CREATE_PATH": 0,
+    "CREATE_PATH_RESPONSE": 1,
+    "EXISTS": 2,
+    "EXISTS_RESPONSE": 3,
+    "SEND_PULSE": 4,
+    "SEND_PULSE_RESPONSE": 5,
+    "GET_ALL_PULSE_FOR_PATH": 6,
+    "GET_ALL_PULSE_FOR_PATH_RESPONSE": 7,
+    "GET_ALL_NODES_FOR_PATH": 8,
+    "GET_ALL_NODES_FOR_PATH_RESPONSE": 9,
+    "GET_PULSE": 10,
+    "GET_PULSE_RESPONSE": 11,
+    "DELETE_PATH": 12,
+    "DELETE_PATH_RESPONSE": 13,
+    "DELETE_PULSE_ID": 14,
+    "DELETE_PULSE_ID_RESPONSE": 15,
+    "CONTROL_MESSAGE": 16,
+    "SASL_MESSAGE_TOKEN": 17,
+    "NOT_AUTHORIZED": 18,
+  }
+
 
 class JavaObjectArg:
   """
@@ -126,12 +248,12 @@
         break
       if fid == 1:
         if ftype == TType.I32:
-          self.int_arg = iprot.readI32();
+          self.int_arg = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I64:
-          self.long_arg = iprot.readI64();
+          self.long_arg = iprot.readI64()
         else:
           iprot.skip(ftype)
       elif fid == 3:
@@ -141,17 +263,17 @@
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.BOOL:
-          self.bool_arg = iprot.readBool();
+          self.bool_arg = iprot.readBool()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.STRING:
-          self.binary_arg = iprot.readString();
+          self.binary_arg = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.DOUBLE:
-          self.double_arg = iprot.readDouble();
+          self.double_arg = iprot.readDouble()
         else:
           iprot.skip(ftype)
       else:
@@ -521,7 +643,7 @@
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.STRING:
-          self.custom_serialized = iprot.readString();
+          self.custom_serialized = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 8:
@@ -643,7 +765,7 @@
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.BOOL:
-          self.direct = iprot.readBool();
+          self.direct = iprot.readBool()
         else:
           iprot.skip(ftype)
       else:
@@ -804,7 +926,7 @@
         break
       if fid == 1:
         if ftype == TType.STRING:
-          self.serialized_java = iprot.readString();
+          self.serialized_java = iprot.readString()
         else:
           iprot.skip(ftype)
       elif fid == 2:
@@ -925,7 +1047,7 @@
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.I32:
-          self.parallelism_hint = iprot.readI32();
+          self.parallelism_hint = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 4:
@@ -1255,6 +1377,7 @@
    - spouts
    - bolts
    - state_spouts
+   - worker_hooks
   """
 
   thrift_spec = (
@@ -1262,12 +1385,14 @@
     (1, TType.MAP, 'spouts', (TType.STRING,None,TType.STRUCT,(SpoutSpec, SpoutSpec.thrift_spec)), None, ), # 1
     (2, TType.MAP, 'bolts', (TType.STRING,None,TType.STRUCT,(Bolt, Bolt.thrift_spec)), None, ), # 2
     (3, TType.MAP, 'state_spouts', (TType.STRING,None,TType.STRUCT,(StateSpoutSpec, StateSpoutSpec.thrift_spec)), None, ), # 3
+    (4, TType.LIST, 'worker_hooks', (TType.STRING,None), None, ), # 4
   )
 
-  def __init__(self, spouts=None, bolts=None, state_spouts=None,):
+  def __init__(self, spouts=None, bolts=None, state_spouts=None, worker_hooks=None,):
     self.spouts = spouts
     self.bolts = bolts
     self.state_spouts = state_spouts
+    self.worker_hooks = worker_hooks
 
   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:
@@ -1314,6 +1439,16 @@
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.worker_hooks = []
+          (_etype63, _size60) = iprot.readListBegin()
+          for _i64 in xrange(_size60):
+            _elem65 = iprot.readString()
+            self.worker_hooks.append(_elem65)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -1327,27 +1462,34 @@
     if self.spouts is not None:
       oprot.writeFieldBegin('spouts', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.spouts))
-      for kiter60,viter61 in self.spouts.items():
-        oprot.writeString(kiter60.encode('utf-8'))
-        viter61.write(oprot)
+      for kiter66,viter67 in self.spouts.items():
+        oprot.writeString(kiter66.encode('utf-8'))
+        viter67.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.bolts is not None:
       oprot.writeFieldBegin('bolts', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.bolts))
-      for kiter62,viter63 in self.bolts.items():
-        oprot.writeString(kiter62.encode('utf-8'))
-        viter63.write(oprot)
+      for kiter68,viter69 in self.bolts.items():
+        oprot.writeString(kiter68.encode('utf-8'))
+        viter69.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.state_spouts is not None:
       oprot.writeFieldBegin('state_spouts', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.state_spouts))
-      for kiter64,viter65 in self.state_spouts.items():
-        oprot.writeString(kiter64.encode('utf-8'))
-        viter65.write(oprot)
+      for kiter70,viter71 in self.state_spouts.items():
+        oprot.writeString(kiter70.encode('utf-8'))
+        viter71.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
+    if self.worker_hooks is not None:
+      oprot.writeFieldBegin('worker_hooks', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRING, len(self.worker_hooks))
+      for iter72 in self.worker_hooks:
+        oprot.writeString(iter72)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -1366,6 +1508,7 @@
     value = (value * 31) ^ hash(self.spouts)
     value = (value * 31) ^ hash(self.bolts)
     value = (value * 31) ^ hash(self.state_spouts)
+    value = (value * 31) ^ hash(self.worker_hooks)
     return value
 
   def __repr__(self):
@@ -1672,6 +1815,12 @@
    - sched_status
    - owner
    - replication_count
+   - requested_memonheap
+   - requested_memoffheap
+   - requested_cpu
+   - assigned_memonheap
+   - assigned_memoffheap
+   - assigned_cpu
   """
 
   thrift_spec = (
@@ -2191,9 +2340,20 @@
     (513, TType.STRING, 'sched_status', None, None, ), # 513
     (514, TType.STRING, 'owner', None, None, ), # 514
     (515, TType.I32, 'replication_count', None, None, ), # 515
+    None, # 516
+    None, # 517
+    None, # 518
+    None, # 519
+    None, # 520
+    (521, TType.DOUBLE, 'requested_memonheap', None, None, ), # 521
+    (522, TType.DOUBLE, 'requested_memoffheap', None, None, ), # 522
+    (523, TType.DOUBLE, 'requested_cpu', None, None, ), # 523
+    (524, TType.DOUBLE, 'assigned_memonheap', None, None, ), # 524
+    (525, TType.DOUBLE, 'assigned_memoffheap', None, None, ), # 525
+    (526, TType.DOUBLE, 'assigned_cpu', None, None, ), # 526
   )
 
-  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None, replication_count=None,):
+  def __init__(self, id=None, name=None, num_tasks=None, num_executors=None, num_workers=None, uptime_secs=None, status=None, sched_status=None, owner=None, replication_count=None, requested_memonheap=None, requested_memoffheap=None, requested_cpu=None, assigned_memonheap=None, assigned_memoffheap=None, assigned_cpu=None,):
     self.id = id
     self.name = name
     self.num_tasks = num_tasks
@@ -2204,6 +2364,12 @@
     self.sched_status = sched_status
     self.owner = owner
     self.replication_count = replication_count
+    self.requested_memonheap = requested_memonheap
+    self.requested_memoffheap = requested_memoffheap
+    self.requested_cpu = requested_cpu
+    self.assigned_memonheap = assigned_memonheap
+    self.assigned_memoffheap = assigned_memoffheap
+    self.assigned_cpu = assigned_cpu
 
   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:
@@ -2226,22 +2392,22 @@
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.I32:
-          self.num_tasks = iprot.readI32();
+          self.num_tasks = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.I32:
-          self.num_executors = iprot.readI32();
+          self.num_executors = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.I32:
-          self.num_workers = iprot.readI32();
+          self.num_workers = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.I32:
-          self.uptime_secs = iprot.readI32();
+          self.uptime_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 7:
@@ -2261,7 +2427,37 @@
           iprot.skip(ftype)
       elif fid == 515:
         if ftype == TType.I32:
-          self.replication_count = iprot.readI32();
+          self.replication_count = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 521:
+        if ftype == TType.DOUBLE:
+          self.requested_memonheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 522:
+        if ftype == TType.DOUBLE:
+          self.requested_memoffheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 523:
+        if ftype == TType.DOUBLE:
+          self.requested_cpu = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 524:
+        if ftype == TType.DOUBLE:
+          self.assigned_memonheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 525:
+        if ftype == TType.DOUBLE:
+          self.assigned_memoffheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 526:
+        if ftype == TType.DOUBLE:
+          self.assigned_cpu = iprot.readDouble()
         else:
           iprot.skip(ftype)
       else:
@@ -2314,6 +2510,30 @@
       oprot.writeFieldBegin('replication_count', TType.I32, 515)
       oprot.writeI32(self.replication_count)
       oprot.writeFieldEnd()
+    if self.requested_memonheap is not None:
+      oprot.writeFieldBegin('requested_memonheap', TType.DOUBLE, 521)
+      oprot.writeDouble(self.requested_memonheap)
+      oprot.writeFieldEnd()
+    if self.requested_memoffheap is not None:
+      oprot.writeFieldBegin('requested_memoffheap', TType.DOUBLE, 522)
+      oprot.writeDouble(self.requested_memoffheap)
+      oprot.writeFieldEnd()
+    if self.requested_cpu is not None:
+      oprot.writeFieldBegin('requested_cpu', TType.DOUBLE, 523)
+      oprot.writeDouble(self.requested_cpu)
+      oprot.writeFieldEnd()
+    if self.assigned_memonheap is not None:
+      oprot.writeFieldBegin('assigned_memonheap', TType.DOUBLE, 524)
+      oprot.writeDouble(self.assigned_memonheap)
+      oprot.writeFieldEnd()
+    if self.assigned_memoffheap is not None:
+      oprot.writeFieldBegin('assigned_memoffheap', TType.DOUBLE, 525)
+      oprot.writeDouble(self.assigned_memoffheap)
+      oprot.writeFieldEnd()
+    if self.assigned_cpu is not None:
+      oprot.writeFieldBegin('assigned_cpu', TType.DOUBLE, 526)
+      oprot.writeDouble(self.assigned_cpu)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2347,6 +2567,12 @@
     value = (value * 31) ^ hash(self.sched_status)
     value = (value * 31) ^ hash(self.owner)
     value = (value * 31) ^ hash(self.replication_count)
+    value = (value * 31) ^ hash(self.requested_memonheap)
+    value = (value * 31) ^ hash(self.requested_memoffheap)
+    value = (value * 31) ^ hash(self.requested_cpu)
+    value = (value * 31) ^ hash(self.assigned_memonheap)
+    value = (value * 31) ^ hash(self.assigned_memoffheap)
+    value = (value * 31) ^ hash(self.assigned_cpu)
     return value
 
   def __repr__(self):
@@ -2369,6 +2595,9 @@
    - num_used_workers
    - supervisor_id
    - version
+   - total_resources
+   - used_mem
+   - used_cpu
   """
 
   thrift_spec = (
@@ -2379,15 +2608,21 @@
     (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
+    (8, TType.DOUBLE, 'used_mem', None, None, ), # 8
+    (9, TType.DOUBLE, 'used_cpu', None, None, ), # 9
   )
 
-  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, used_mem=None, used_cpu=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
+    self.used_mem = used_mem
+    self.used_cpu = used_cpu
 
   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:
@@ -2405,17 +2640,17 @@
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I32:
-          self.uptime_secs = iprot.readI32();
+          self.uptime_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.I32:
-          self.num_workers = iprot.readI32();
+          self.num_workers = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.I32:
-          self.num_used_workers = iprot.readI32();
+          self.num_used_workers = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 5:
@@ -2428,6 +2663,27 @@
           self.version = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.MAP:
+          self.total_resources = {}
+          (_ktype74, _vtype75, _size73 ) = iprot.readMapBegin()
+          for _i77 in xrange(_size73):
+            _key78 = iprot.readString().decode('utf-8')
+            _val79 = iprot.readDouble()
+            self.total_resources[_key78] = _val79
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.DOUBLE:
+          self.used_mem = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.DOUBLE:
+          self.used_cpu = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2462,6 +2718,22 @@
       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 kiter80,viter81 in self.total_resources.items():
+        oprot.writeString(kiter80.encode('utf-8'))
+        oprot.writeDouble(viter81)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.used_mem is not None:
+      oprot.writeFieldBegin('used_mem', TType.DOUBLE, 8)
+      oprot.writeDouble(self.used_mem)
+      oprot.writeFieldEnd()
+    if self.used_cpu is not None:
+      oprot.writeFieldBegin('used_cpu', TType.DOUBLE, 9)
+      oprot.writeDouble(self.used_cpu)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2487,6 +2759,9 @@
     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)
+    value = (value * 31) ^ hash(self.used_mem)
+    value = (value * 31) ^ hash(self.used_cpu)
     return value
 
   def __repr__(self):
@@ -2542,17 +2817,17 @@
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I32:
-          self.port = iprot.readI32();
+          self.port = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.I32:
-          self.uptime_secs = iprot.readI32();
+          self.uptime_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.BOOL:
-          self.isLeader = iprot.readBool();
+          self.isLeader = iprot.readBool()
         else:
           iprot.skip(ftype)
       elif fid == 5:
@@ -2631,6 +2906,7 @@
   """
   Attributes:
    - supervisors
+   - nimbus_uptime_secs
    - topologies
    - nimbuses
   """
@@ -2638,13 +2914,14 @@
   thrift_spec = (
     None, # 0
     (1, TType.LIST, 'supervisors', (TType.STRUCT,(SupervisorSummary, SupervisorSummary.thrift_spec)), None, ), # 1
-    None, # 2
+    (2, TType.I32, 'nimbus_uptime_secs', None, 0, ), # 2
     (3, TType.LIST, 'topologies', (TType.STRUCT,(TopologySummary, TopologySummary.thrift_spec)), None, ), # 3
     (4, TType.LIST, 'nimbuses', (TType.STRUCT,(NimbusSummary, NimbusSummary.thrift_spec)), None, ), # 4
   )
 
-  def __init__(self, supervisors=None, topologies=None, nimbuses=None,):
+  def __init__(self, supervisors=None, nimbus_uptime_secs=thrift_spec[2][4], topologies=None, nimbuses=None,):
     self.supervisors = supervisors
+    self.nimbus_uptime_secs = nimbus_uptime_secs
     self.topologies = topologies
     self.nimbuses = nimbuses
 
@@ -2660,33 +2937,38 @@
       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)
+          (_etype85, _size82) = iprot.readListBegin()
+          for _i86 in xrange(_size82):
+            _elem87 = SupervisorSummary()
+            _elem87.read(iprot)
+            self.supervisors.append(_elem87)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.nimbus_uptime_secs = iprot.readI32()
+        else:
+          iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.topologies = []
-          (_etype75, _size72) = iprot.readListBegin()
-          for _i76 in xrange(_size72):
-            _elem77 = TopologySummary()
-            _elem77.read(iprot)
-            self.topologies.append(_elem77)
+          (_etype91, _size88) = iprot.readListBegin()
+          for _i92 in xrange(_size88):
+            _elem93 = TopologySummary()
+            _elem93.read(iprot)
+            self.topologies.append(_elem93)
           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)
+          (_etype97, _size94) = iprot.readListBegin()
+          for _i98 in xrange(_size94):
+            _elem99 = NimbusSummary()
+            _elem99.read(iprot)
+            self.nimbuses.append(_elem99)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -2703,22 +2985,26 @@
     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 iter100 in self.supervisors:
+        iter100.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
+    if self.nimbus_uptime_secs is not None:
+      oprot.writeFieldBegin('nimbus_uptime_secs', TType.I32, 2)
+      oprot.writeI32(self.nimbus_uptime_secs)
+      oprot.writeFieldEnd()
     if self.topologies is not None:
       oprot.writeFieldBegin('topologies', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.topologies))
-      for iter85 in self.topologies:
-        iter85.write(oprot)
+      for iter101 in self.topologies:
+        iter101.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 iter102 in self.nimbuses:
+        iter102.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -2737,6 +3023,7 @@
   def __hash__(self):
     value = 17
     value = (value * 31) ^ hash(self.supervisors)
+    value = (value * 31) ^ hash(self.nimbus_uptime_secs)
     value = (value * 31) ^ hash(self.topologies)
     value = (value * 31) ^ hash(self.nimbuses)
     return value
@@ -2791,7 +3078,7 @@
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I32:
-          self.error_time_secs = iprot.readI32();
+          self.error_time_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 3:
@@ -2801,7 +3088,7 @@
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.I32:
-          self.port = iprot.readI32();
+          self.port = iprot.readI32()
         else:
           iprot.skip(ftype)
       else:
@@ -2898,90 +3185,90 @@
       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
+          (_ktype104, _vtype105, _size103 ) = iprot.readMapBegin()
+          for _i107 in xrange(_size103):
+            _key108 = iprot.readString().decode('utf-8')
+            _val109 = {}
+            (_ktype111, _vtype112, _size110 ) = iprot.readMapBegin()
+            for _i114 in xrange(_size110):
+              _key115 = GlobalStreamId()
+              _key115.read(iprot)
+              _val116 = iprot.readI64()
+              _val109[_key115] = _val116
             iprot.readMapEnd()
-            self.acked[_key92] = _val93
+            self.acked[_key108] = _val109
           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
+          (_ktype118, _vtype119, _size117 ) = iprot.readMapBegin()
+          for _i121 in xrange(_size117):
+            _key122 = iprot.readString().decode('utf-8')
+            _val123 = {}
+            (_ktype125, _vtype126, _size124 ) = iprot.readMapBegin()
+            for _i128 in xrange(_size124):
+              _key129 = GlobalStreamId()
+              _key129.read(iprot)
+              _val130 = iprot.readI64()
+              _val123[_key129] = _val130
             iprot.readMapEnd()
-            self.failed[_key106] = _val107
+            self.failed[_key122] = _val123
           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
+          (_ktype132, _vtype133, _size131 ) = iprot.readMapBegin()
+          for _i135 in xrange(_size131):
+            _key136 = iprot.readString().decode('utf-8')
+            _val137 = {}
+            (_ktype139, _vtype140, _size138 ) = iprot.readMapBegin()
+            for _i142 in xrange(_size138):
+              _key143 = GlobalStreamId()
+              _key143.read(iprot)
+              _val144 = iprot.readDouble()
+              _val137[_key143] = _val144
             iprot.readMapEnd()
-            self.process_ms_avg[_key120] = _val121
+            self.process_ms_avg[_key136] = _val137
           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
+          (_ktype146, _vtype147, _size145 ) = iprot.readMapBegin()
+          for _i149 in xrange(_size145):
+            _key150 = iprot.readString().decode('utf-8')
+            _val151 = {}
+            (_ktype153, _vtype154, _size152 ) = iprot.readMapBegin()
+            for _i156 in xrange(_size152):
+              _key157 = GlobalStreamId()
+              _key157.read(iprot)
+              _val158 = iprot.readI64()
+              _val151[_key157] = _val158
             iprot.readMapEnd()
-            self.executed[_key134] = _val135
+            self.executed[_key150] = _val151
           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
+          (_ktype160, _vtype161, _size159 ) = iprot.readMapBegin()
+          for _i163 in xrange(_size159):
+            _key164 = iprot.readString().decode('utf-8')
+            _val165 = {}
+            (_ktype167, _vtype168, _size166 ) = iprot.readMapBegin()
+            for _i170 in xrange(_size166):
+              _key171 = GlobalStreamId()
+              _key171.read(iprot)
+              _val172 = iprot.readDouble()
+              _val165[_key171] = _val172
             iprot.readMapEnd()
-            self.execute_ms_avg[_key148] = _val149
+            self.execute_ms_avg[_key164] = _val165
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -2998,60 +3285,60 @@
     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 kiter173,viter174 in self.acked.items():
+        oprot.writeString(kiter173.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter174))
+        for kiter175,viter176 in viter174.items():
+          kiter175.write(oprot)
+          oprot.writeI64(viter176)
         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 kiter177,viter178 in self.failed.items():
+        oprot.writeString(kiter177.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter178))
+        for kiter179,viter180 in viter178.items():
+          kiter179.write(oprot)
+          oprot.writeI64(viter180)
         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 kiter181,viter182 in self.process_ms_avg.items():
+        oprot.writeString(kiter181.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter182))
+        for kiter183,viter184 in viter182.items():
+          kiter183.write(oprot)
+          oprot.writeDouble(viter184)
         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 kiter185,viter186 in self.executed.items():
+        oprot.writeString(kiter185.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter186))
+        for kiter187,viter188 in viter186.items():
+          kiter187.write(oprot)
+          oprot.writeI64(viter188)
         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 kiter189,viter190 in self.execute_ms_avg.items():
+        oprot.writeString(kiter189.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter190))
+        for kiter191,viter192 in viter190.items():
+          kiter191.write(oprot)
+          oprot.writeDouble(viter192)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -3124,51 +3411,51 @@
       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
+          (_ktype194, _vtype195, _size193 ) = iprot.readMapBegin()
+          for _i197 in xrange(_size193):
+            _key198 = iprot.readString().decode('utf-8')
+            _val199 = {}
+            (_ktype201, _vtype202, _size200 ) = iprot.readMapBegin()
+            for _i204 in xrange(_size200):
+              _key205 = iprot.readString().decode('utf-8')
+              _val206 = iprot.readI64()
+              _val199[_key205] = _val206
             iprot.readMapEnd()
-            self.acked[_key182] = _val183
+            self.acked[_key198] = _val199
           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
+          (_ktype208, _vtype209, _size207 ) = iprot.readMapBegin()
+          for _i211 in xrange(_size207):
+            _key212 = iprot.readString().decode('utf-8')
+            _val213 = {}
+            (_ktype215, _vtype216, _size214 ) = iprot.readMapBegin()
+            for _i218 in xrange(_size214):
+              _key219 = iprot.readString().decode('utf-8')
+              _val220 = iprot.readI64()
+              _val213[_key219] = _val220
             iprot.readMapEnd()
-            self.failed[_key196] = _val197
+            self.failed[_key212] = _val213
           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
+          (_ktype222, _vtype223, _size221 ) = iprot.readMapBegin()
+          for _i225 in xrange(_size221):
+            _key226 = iprot.readString().decode('utf-8')
+            _val227 = {}
+            (_ktype229, _vtype230, _size228 ) = iprot.readMapBegin()
+            for _i232 in xrange(_size228):
+              _key233 = iprot.readString().decode('utf-8')
+              _val234 = iprot.readDouble()
+              _val227[_key233] = _val234
             iprot.readMapEnd()
-            self.complete_ms_avg[_key210] = _val211
+            self.complete_ms_avg[_key226] = _val227
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3185,36 +3472,36 @@
     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 kiter235,viter236 in self.acked.items():
+        oprot.writeString(kiter235.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter236))
+        for kiter237,viter238 in viter236.items():
+          oprot.writeString(kiter237.encode('utf-8'))
+          oprot.writeI64(viter238)
         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 kiter239,viter240 in self.failed.items():
+        oprot.writeString(kiter239.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter240))
+        for kiter241,viter242 in viter240.items():
+          oprot.writeString(kiter241.encode('utf-8'))
+          oprot.writeI64(viter242)
         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 kiter243,viter244 in self.complete_ms_avg.items():
+        oprot.writeString(kiter243.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter244))
+        for kiter245,viter246 in viter244.items():
+          oprot.writeString(kiter245.encode('utf-8'))
+          oprot.writeDouble(viter246)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -3364,34 +3651,34 @@
       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
+          (_ktype248, _vtype249, _size247 ) = iprot.readMapBegin()
+          for _i251 in xrange(_size247):
+            _key252 = iprot.readString().decode('utf-8')
+            _val253 = {}
+            (_ktype255, _vtype256, _size254 ) = iprot.readMapBegin()
+            for _i258 in xrange(_size254):
+              _key259 = iprot.readString().decode('utf-8')
+              _val260 = iprot.readI64()
+              _val253[_key259] = _val260
             iprot.readMapEnd()
-            self.emitted[_key236] = _val237
+            self.emitted[_key252] = _val253
           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
+          (_ktype262, _vtype263, _size261 ) = iprot.readMapBegin()
+          for _i265 in xrange(_size261):
+            _key266 = iprot.readString().decode('utf-8')
+            _val267 = {}
+            (_ktype269, _vtype270, _size268 ) = iprot.readMapBegin()
+            for _i272 in xrange(_size268):
+              _key273 = iprot.readString().decode('utf-8')
+              _val274 = iprot.readI64()
+              _val267[_key273] = _val274
             iprot.readMapEnd()
-            self.transferred[_key250] = _val251
+            self.transferred[_key266] = _val267
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -3403,7 +3690,7 @@
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.DOUBLE:
-          self.rate = iprot.readDouble();
+          self.rate = iprot.readDouble()
         else:
           iprot.skip(ftype)
       else:
@@ -3419,24 +3706,24 @@
     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 kiter275,viter276 in self.emitted.items():
+        oprot.writeString(kiter275.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter276))
+        for kiter277,viter278 in viter276.items():
+          oprot.writeString(kiter277.encode('utf-8'))
+          oprot.writeI64(viter278)
         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 kiter279,viter280 in self.transferred.items():
+        oprot.writeString(kiter279.encode('utf-8'))
+        oprot.writeMapBegin(TType.STRING, TType.I64, len(viter280))
+        for kiter281,viter282 in viter280.items():
+          oprot.writeString(kiter281.encode('utf-8'))
+          oprot.writeI64(viter282)
         oprot.writeMapEnd()
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
@@ -3510,12 +3797,12 @@
         break
       if fid == 1:
         if ftype == TType.I32:
-          self.task_start = iprot.readI32();
+          self.task_start = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I32:
-          self.task_end = iprot.readI32();
+          self.task_end = iprot.readI32()
         else:
           iprot.skip(ftype)
       else:
@@ -3621,12 +3908,12 @@
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.I32:
-          self.port = iprot.readI32();
+          self.port = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.I32:
-          self.uptime_secs = iprot.readI32();
+          self.uptime_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 7:
@@ -3707,6 +3994,84 @@
   def __ne__(self, other):
     return not (self == other)
 
+class DebugOptions:
+  """
+  Attributes:
+   - enable
+   - samplingpct
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.BOOL, 'enable', None, None, ), # 1
+    (2, TType.DOUBLE, 'samplingpct', None, None, ), # 2
+  )
+
+  def __init__(self, enable=None, samplingpct=None,):
+    self.enable = enable
+    self.samplingpct = samplingpct
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.BOOL:
+          self.enable = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.DOUBLE:
+          self.samplingpct = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('DebugOptions')
+    if self.enable is not None:
+      oprot.writeFieldBegin('enable', TType.BOOL, 1)
+      oprot.writeBool(self.enable)
+      oprot.writeFieldEnd()
+    if self.samplingpct is not None:
+      oprot.writeFieldBegin('samplingpct', TType.DOUBLE, 2)
+      oprot.writeDouble(self.samplingpct)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.enable)
+    value = (value * 31) ^ hash(self.samplingpct)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class TopologyInfo:
   """
   Attributes:
@@ -3720,6 +4085,12 @@
    - sched_status
    - owner
    - replication_count
+   - requested_memonheap
+   - requested_memoffheap
+   - requested_cpu
+   - assigned_memonheap
+   - assigned_memoffheap
+   - assigned_cpu
   """
 
   thrift_spec = (
@@ -4239,9 +4610,20 @@
     (513, TType.STRING, 'sched_status', None, None, ), # 513
     (514, TType.STRING, 'owner', None, None, ), # 514
     (515, TType.I32, 'replication_count', None, None, ), # 515
+    None, # 516
+    None, # 517
+    None, # 518
+    None, # 519
+    None, # 520
+    (521, TType.DOUBLE, 'requested_memonheap', None, None, ), # 521
+    (522, TType.DOUBLE, 'requested_memoffheap', None, None, ), # 522
+    (523, TType.DOUBLE, 'requested_cpu', None, None, ), # 523
+    (524, TType.DOUBLE, 'assigned_memonheap', None, None, ), # 524
+    (525, TType.DOUBLE, 'assigned_memoffheap', None, None, ), # 525
+    (526, TType.DOUBLE, 'assigned_cpu', None, None, ), # 526
   )
 
-  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, component_debug=None, sched_status=None, owner=None, replication_count=None,):
+  def __init__(self, id=None, name=None, uptime_secs=None, executors=None, status=None, errors=None, component_debug=None, sched_status=None, owner=None, replication_count=None, requested_memonheap=None, requested_memoffheap=None, requested_cpu=None, assigned_memonheap=None, assigned_memoffheap=None, assigned_cpu=None,):
     self.id = id
     self.name = name
     self.uptime_secs = uptime_secs
@@ -4252,6 +4634,12 @@
     self.sched_status = sched_status
     self.owner = owner
     self.replication_count = replication_count
+    self.requested_memonheap = requested_memonheap
+    self.requested_memoffheap = requested_memoffheap
+    self.requested_cpu = requested_cpu
+    self.assigned_memonheap = assigned_memonheap
+    self.assigned_memoffheap = assigned_memoffheap
+    self.assigned_cpu = assigned_cpu
 
   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:
@@ -4274,17 +4662,17 @@
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.I32:
-          self.uptime_secs = iprot.readI32();
+          self.uptime_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       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)
+          (_etype286, _size283) = iprot.readListBegin()
+          for _i287 in xrange(_size283):
+            _elem288 = ExecutorSummary()
+            _elem288.read(iprot)
+            self.executors.append(_elem288)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -4296,29 +4684,29 @@
       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)
+          (_ktype290, _vtype291, _size289 ) = iprot.readMapBegin()
+          for _i293 in xrange(_size289):
+            _key294 = iprot.readString().decode('utf-8')
+            _val295 = []
+            (_etype299, _size296) = iprot.readListBegin()
+            for _i300 in xrange(_size296):
+              _elem301 = ErrorInfo()
+              _elem301.read(iprot)
+              _val295.append(_elem301)
             iprot.readListEnd()
-            self.errors[_key278] = _val279
+            self.errors[_key294] = _val295
           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
+          (_ktype303, _vtype304, _size302 ) = iprot.readMapBegin()
+          for _i306 in xrange(_size302):
+            _key307 = iprot.readString().decode('utf-8')
+            _val308 = DebugOptions()
+            _val308.read(iprot)
+            self.component_debug[_key307] = _val308
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4334,7 +4722,37 @@
           iprot.skip(ftype)
       elif fid == 515:
         if ftype == TType.I32:
-          self.replication_count = iprot.readI32();
+          self.replication_count = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 521:
+        if ftype == TType.DOUBLE:
+          self.requested_memonheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 522:
+        if ftype == TType.DOUBLE:
+          self.requested_memoffheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 523:
+        if ftype == TType.DOUBLE:
+          self.requested_cpu = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 524:
+        if ftype == TType.DOUBLE:
+          self.assigned_memonheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 525:
+        if ftype == TType.DOUBLE:
+          self.assigned_memoffheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 526:
+        if ftype == TType.DOUBLE:
+          self.assigned_cpu = iprot.readDouble()
         else:
           iprot.skip(ftype)
       else:
@@ -4362,8 +4780,8 @@
     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 iter309 in self.executors:
+        iter309.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.status is not None:
@@ -4373,20 +4791,20 @@
     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 kiter310,viter311 in self.errors.items():
+        oprot.writeString(kiter310.encode('utf-8'))
+        oprot.writeListBegin(TType.STRUCT, len(viter311))
+        for iter312 in viter311:
+          iter312.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 kiter313,viter314 in self.component_debug.items():
+        oprot.writeString(kiter313.encode('utf-8'))
+        viter314.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.sched_status is not None:
@@ -4401,6 +4819,30 @@
       oprot.writeFieldBegin('replication_count', TType.I32, 515)
       oprot.writeI32(self.replication_count)
       oprot.writeFieldEnd()
+    if self.requested_memonheap is not None:
+      oprot.writeFieldBegin('requested_memonheap', TType.DOUBLE, 521)
+      oprot.writeDouble(self.requested_memonheap)
+      oprot.writeFieldEnd()
+    if self.requested_memoffheap is not None:
+      oprot.writeFieldBegin('requested_memoffheap', TType.DOUBLE, 522)
+      oprot.writeDouble(self.requested_memoffheap)
+      oprot.writeFieldEnd()
+    if self.requested_cpu is not None:
+      oprot.writeFieldBegin('requested_cpu', TType.DOUBLE, 523)
+      oprot.writeDouble(self.requested_cpu)
+      oprot.writeFieldEnd()
+    if self.assigned_memonheap is not None:
+      oprot.writeFieldBegin('assigned_memonheap', TType.DOUBLE, 524)
+      oprot.writeDouble(self.assigned_memonheap)
+      oprot.writeFieldEnd()
+    if self.assigned_memoffheap is not None:
+      oprot.writeFieldBegin('assigned_memoffheap', TType.DOUBLE, 525)
+      oprot.writeDouble(self.assigned_memoffheap)
+      oprot.writeFieldEnd()
+    if self.assigned_cpu is not None:
+      oprot.writeFieldBegin('assigned_cpu', TType.DOUBLE, 526)
+      oprot.writeDouble(self.assigned_cpu)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -4432,6 +4874,12 @@
     value = (value * 31) ^ hash(self.sched_status)
     value = (value * 31) ^ hash(self.owner)
     value = (value * 31) ^ hash(self.replication_count)
+    value = (value * 31) ^ hash(self.requested_memonheap)
+    value = (value * 31) ^ hash(self.requested_memoffheap)
+    value = (value * 31) ^ hash(self.requested_cpu)
+    value = (value * 31) ^ hash(self.assigned_memonheap)
+    value = (value * 31) ^ hash(self.assigned_memoffheap)
+    value = (value * 31) ^ hash(self.assigned_cpu)
     return value
 
   def __repr__(self):
@@ -4445,22 +4893,34 @@
   def __ne__(self, other):
     return not (self == other)
 
-class DebugOptions:
+class CommonAggregateStats:
   """
   Attributes:
-   - enable
-   - samplingpct
+   - num_executors
+   - num_tasks
+   - emitted
+   - transferred
+   - acked
+   - failed
   """
 
   thrift_spec = (
     None, # 0
-    (1, TType.BOOL, 'enable', None, None, ), # 1
-    (2, TType.DOUBLE, 'samplingpct', None, None, ), # 2
+    (1, TType.I32, 'num_executors', None, None, ), # 1
+    (2, TType.I32, 'num_tasks', None, None, ), # 2
+    (3, TType.I64, 'emitted', None, None, ), # 3
+    (4, TType.I64, 'transferred', None, None, ), # 4
+    (5, TType.I64, 'acked', None, None, ), # 5
+    (6, TType.I64, 'failed', None, None, ), # 6
   )
 
-  def __init__(self, enable=None, samplingpct=None,):
-    self.enable = enable
-    self.samplingpct = samplingpct
+  def __init__(self, num_executors=None, num_tasks=None, emitted=None, transferred=None, acked=None, failed=None,):
+    self.num_executors = num_executors
+    self.num_tasks = num_tasks
+    self.emitted = emitted
+    self.transferred = transferred
+    self.acked = acked
+    self.failed = failed
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -4472,13 +4932,33 @@
       if ftype == TType.STOP:
         break
       if fid == 1:
-        if ftype == TType.BOOL:
-          self.enable = iprot.readBool();
+        if ftype == TType.I32:
+          self.num_executors = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 2:
-        if ftype == TType.DOUBLE:
-          self.samplingpct = iprot.readDouble();
+        if ftype == TType.I32:
+          self.num_tasks = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I64:
+          self.emitted = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I64:
+          self.transferred = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I64:
+          self.acked = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.I64:
+          self.failed = iprot.readI64()
         else:
           iprot.skip(ftype)
       else:
@@ -4490,14 +4970,30 @@
     if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
       oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
       return
-    oprot.writeStructBegin('DebugOptions')
-    if self.enable is not None:
-      oprot.writeFieldBegin('enable', TType.BOOL, 1)
-      oprot.writeBool(self.enable)
+    oprot.writeStructBegin('CommonAggregateStats')
+    if self.num_executors is not None:
+      oprot.writeFieldBegin('num_executors', TType.I32, 1)
+      oprot.writeI32(self.num_executors)
       oprot.writeFieldEnd()
-    if self.samplingpct is not None:
-      oprot.writeFieldBegin('samplingpct', TType.DOUBLE, 2)
-      oprot.writeDouble(self.samplingpct)
+    if self.num_tasks is not None:
+      oprot.writeFieldBegin('num_tasks', TType.I32, 2)
+      oprot.writeI32(self.num_tasks)
+      oprot.writeFieldEnd()
+    if self.emitted is not None:
+      oprot.writeFieldBegin('emitted', TType.I64, 3)
+      oprot.writeI64(self.emitted)
+      oprot.writeFieldEnd()
+    if self.transferred is not None:
+      oprot.writeFieldBegin('transferred', TType.I64, 4)
+      oprot.writeI64(self.transferred)
+      oprot.writeFieldEnd()
+    if self.acked is not None:
+      oprot.writeFieldBegin('acked', TType.I64, 5)
+      oprot.writeI64(self.acked)
+      oprot.writeFieldEnd()
+    if self.failed is not None:
+      oprot.writeFieldBegin('failed', TType.I64, 6)
+      oprot.writeI64(self.failed)
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
@@ -4508,8 +5004,1775 @@
 
   def __hash__(self):
     value = 17
-    value = (value * 31) ^ hash(self.enable)
-    value = (value * 31) ^ hash(self.samplingpct)
+    value = (value * 31) ^ hash(self.num_executors)
+    value = (value * 31) ^ hash(self.num_tasks)
+    value = (value * 31) ^ hash(self.emitted)
+    value = (value * 31) ^ hash(self.transferred)
+    value = (value * 31) ^ hash(self.acked)
+    value = (value * 31) ^ hash(self.failed)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class SpoutAggregateStats:
+  """
+  Attributes:
+   - complete_latency_ms
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.DOUBLE, 'complete_latency_ms', None, None, ), # 1
+  )
+
+  def __init__(self, complete_latency_ms=None,):
+    self.complete_latency_ms = complete_latency_ms
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.DOUBLE:
+          self.complete_latency_ms = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SpoutAggregateStats')
+    if self.complete_latency_ms is not None:
+      oprot.writeFieldBegin('complete_latency_ms', TType.DOUBLE, 1)
+      oprot.writeDouble(self.complete_latency_ms)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.complete_latency_ms)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class BoltAggregateStats:
+  """
+  Attributes:
+   - execute_latency_ms
+   - process_latency_ms
+   - executed
+   - capacity
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.DOUBLE, 'execute_latency_ms', None, None, ), # 1
+    (2, TType.DOUBLE, 'process_latency_ms', None, None, ), # 2
+    (3, TType.I64, 'executed', None, None, ), # 3
+    (4, TType.DOUBLE, 'capacity', None, None, ), # 4
+  )
+
+  def __init__(self, execute_latency_ms=None, process_latency_ms=None, executed=None, capacity=None,):
+    self.execute_latency_ms = execute_latency_ms
+    self.process_latency_ms = process_latency_ms
+    self.executed = executed
+    self.capacity = capacity
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.DOUBLE:
+          self.execute_latency_ms = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.DOUBLE:
+          self.process_latency_ms = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I64:
+          self.executed = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.DOUBLE:
+          self.capacity = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('BoltAggregateStats')
+    if self.execute_latency_ms is not None:
+      oprot.writeFieldBegin('execute_latency_ms', TType.DOUBLE, 1)
+      oprot.writeDouble(self.execute_latency_ms)
+      oprot.writeFieldEnd()
+    if self.process_latency_ms is not None:
+      oprot.writeFieldBegin('process_latency_ms', TType.DOUBLE, 2)
+      oprot.writeDouble(self.process_latency_ms)
+      oprot.writeFieldEnd()
+    if self.executed is not None:
+      oprot.writeFieldBegin('executed', TType.I64, 3)
+      oprot.writeI64(self.executed)
+      oprot.writeFieldEnd()
+    if self.capacity is not None:
+      oprot.writeFieldBegin('capacity', TType.DOUBLE, 4)
+      oprot.writeDouble(self.capacity)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.execute_latency_ms)
+    value = (value * 31) ^ hash(self.process_latency_ms)
+    value = (value * 31) ^ hash(self.executed)
+    value = (value * 31) ^ hash(self.capacity)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class SpecificAggregateStats:
+  """
+  Attributes:
+   - bolt
+   - spout
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'bolt', (BoltAggregateStats, BoltAggregateStats.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'spout', (SpoutAggregateStats, SpoutAggregateStats.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, bolt=None, spout=None,):
+    self.bolt = bolt
+    self.spout = spout
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.bolt = BoltAggregateStats()
+          self.bolt.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.spout = SpoutAggregateStats()
+          self.spout.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SpecificAggregateStats')
+    if self.bolt is not None:
+      oprot.writeFieldBegin('bolt', TType.STRUCT, 1)
+      self.bolt.write(oprot)
+      oprot.writeFieldEnd()
+    if self.spout is not None:
+      oprot.writeFieldBegin('spout', TType.STRUCT, 2)
+      self.spout.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.bolt)
+    value = (value * 31) ^ hash(self.spout)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ComponentAggregateStats:
+  """
+  Attributes:
+   - type
+   - common_stats
+   - specific_stats
+   - last_error
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'type', None, None, ), # 1
+    (2, TType.STRUCT, 'common_stats', (CommonAggregateStats, CommonAggregateStats.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'specific_stats', (SpecificAggregateStats, SpecificAggregateStats.thrift_spec), None, ), # 3
+    (4, TType.STRUCT, 'last_error', (ErrorInfo, ErrorInfo.thrift_spec), None, ), # 4
+  )
+
+  def __init__(self, type=None, common_stats=None, specific_stats=None, last_error=None,):
+    self.type = type
+    self.common_stats = common_stats
+    self.specific_stats = specific_stats
+    self.last_error = last_error
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I32:
+          self.type = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.common_stats = CommonAggregateStats()
+          self.common_stats.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.specific_stats = SpecificAggregateStats()
+          self.specific_stats.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRUCT:
+          self.last_error = ErrorInfo()
+          self.last_error.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ComponentAggregateStats')
+    if self.type is not None:
+      oprot.writeFieldBegin('type', TType.I32, 1)
+      oprot.writeI32(self.type)
+      oprot.writeFieldEnd()
+    if self.common_stats is not None:
+      oprot.writeFieldBegin('common_stats', TType.STRUCT, 2)
+      self.common_stats.write(oprot)
+      oprot.writeFieldEnd()
+    if self.specific_stats is not None:
+      oprot.writeFieldBegin('specific_stats', TType.STRUCT, 3)
+      self.specific_stats.write(oprot)
+      oprot.writeFieldEnd()
+    if self.last_error is not None:
+      oprot.writeFieldBegin('last_error', TType.STRUCT, 4)
+      self.last_error.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.type)
+    value = (value * 31) ^ hash(self.common_stats)
+    value = (value * 31) ^ hash(self.specific_stats)
+    value = (value * 31) ^ hash(self.last_error)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class TopologyStats:
+  """
+  Attributes:
+   - window_to_emitted
+   - window_to_transferred
+   - window_to_complete_latencies_ms
+   - window_to_acked
+   - window_to_failed
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'window_to_emitted', (TType.STRING,None,TType.I64,None), None, ), # 1
+    (2, TType.MAP, 'window_to_transferred', (TType.STRING,None,TType.I64,None), None, ), # 2
+    (3, TType.MAP, 'window_to_complete_latencies_ms', (TType.STRING,None,TType.DOUBLE,None), None, ), # 3
+    (4, TType.MAP, 'window_to_acked', (TType.STRING,None,TType.I64,None), None, ), # 4
+    (5, TType.MAP, 'window_to_failed', (TType.STRING,None,TType.I64,None), None, ), # 5
+  )
+
+  def __init__(self, window_to_emitted=None, window_to_transferred=None, window_to_complete_latencies_ms=None, window_to_acked=None, window_to_failed=None,):
+    self.window_to_emitted = window_to_emitted
+    self.window_to_transferred = window_to_transferred
+    self.window_to_complete_latencies_ms = window_to_complete_latencies_ms
+    self.window_to_acked = window_to_acked
+    self.window_to_failed = window_to_failed
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.window_to_emitted = {}
+          (_ktype316, _vtype317, _size315 ) = iprot.readMapBegin()
+          for _i319 in xrange(_size315):
+            _key320 = iprot.readString().decode('utf-8')
+            _val321 = iprot.readI64()
+            self.window_to_emitted[_key320] = _val321
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.window_to_transferred = {}
+          (_ktype323, _vtype324, _size322 ) = iprot.readMapBegin()
+          for _i326 in xrange(_size322):
+            _key327 = iprot.readString().decode('utf-8')
+            _val328 = iprot.readI64()
+            self.window_to_transferred[_key327] = _val328
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.MAP:
+          self.window_to_complete_latencies_ms = {}
+          (_ktype330, _vtype331, _size329 ) = iprot.readMapBegin()
+          for _i333 in xrange(_size329):
+            _key334 = iprot.readString().decode('utf-8')
+            _val335 = iprot.readDouble()
+            self.window_to_complete_latencies_ms[_key334] = _val335
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.MAP:
+          self.window_to_acked = {}
+          (_ktype337, _vtype338, _size336 ) = iprot.readMapBegin()
+          for _i340 in xrange(_size336):
+            _key341 = iprot.readString().decode('utf-8')
+            _val342 = iprot.readI64()
+            self.window_to_acked[_key341] = _val342
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.MAP:
+          self.window_to_failed = {}
+          (_ktype344, _vtype345, _size343 ) = iprot.readMapBegin()
+          for _i347 in xrange(_size343):
+            _key348 = iprot.readString().decode('utf-8')
+            _val349 = iprot.readI64()
+            self.window_to_failed[_key348] = _val349
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TopologyStats')
+    if self.window_to_emitted is not None:
+      oprot.writeFieldBegin('window_to_emitted', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRING, TType.I64, len(self.window_to_emitted))
+      for kiter350,viter351 in self.window_to_emitted.items():
+        oprot.writeString(kiter350.encode('utf-8'))
+        oprot.writeI64(viter351)
+      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 kiter352,viter353 in self.window_to_transferred.items():
+        oprot.writeString(kiter352.encode('utf-8'))
+        oprot.writeI64(viter353)
+      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 kiter354,viter355 in self.window_to_complete_latencies_ms.items():
+        oprot.writeString(kiter354.encode('utf-8'))
+        oprot.writeDouble(viter355)
+      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 kiter356,viter357 in self.window_to_acked.items():
+        oprot.writeString(kiter356.encode('utf-8'))
+        oprot.writeI64(viter357)
+      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 kiter358,viter359 in self.window_to_failed.items():
+        oprot.writeString(kiter358.encode('utf-8'))
+        oprot.writeI64(viter359)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.window_to_emitted)
+    value = (value * 31) ^ hash(self.window_to_transferred)
+    value = (value * 31) ^ hash(self.window_to_complete_latencies_ms)
+    value = (value * 31) ^ hash(self.window_to_acked)
+    value = (value * 31) ^ hash(self.window_to_failed)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class TopologyPageInfo:
+  """
+  Attributes:
+   - id
+   - name
+   - uptime_secs
+   - status
+   - num_tasks
+   - num_workers
+   - num_executors
+   - topology_conf
+   - id_to_spout_agg_stats
+   - id_to_bolt_agg_stats
+   - sched_status
+   - topology_stats
+   - owner
+   - debug_options
+   - replication_count
+   - requested_memonheap
+   - requested_memoffheap
+   - requested_cpu
+   - assigned_memonheap
+   - assigned_memoffheap
+   - assigned_cpu
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRING, 'name', None, None, ), # 2
+    (3, TType.I32, 'uptime_secs', None, None, ), # 3
+    (4, TType.STRING, 'status', None, None, ), # 4
+    (5, TType.I32, 'num_tasks', None, None, ), # 5
+    (6, TType.I32, 'num_workers', None, None, ), # 6
+    (7, TType.I32, 'num_executors', None, None, ), # 7
+    (8, TType.STRING, 'topology_conf', None, None, ), # 8
+    (9, TType.MAP, 'id_to_spout_agg_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 9
+    (10, TType.MAP, 'id_to_bolt_agg_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 10
+    (11, TType.STRING, 'sched_status', None, None, ), # 11
+    (12, TType.STRUCT, 'topology_stats', (TopologyStats, TopologyStats.thrift_spec), None, ), # 12
+    (13, TType.STRING, 'owner', None, None, ), # 13
+    (14, TType.STRUCT, 'debug_options', (DebugOptions, DebugOptions.thrift_spec), None, ), # 14
+    (15, TType.I32, 'replication_count', None, None, ), # 15
+    None, # 16
+    None, # 17
+    None, # 18
+    None, # 19
+    None, # 20
+    None, # 21
+    None, # 22
+    None, # 23
+    None, # 24
+    None, # 25
+    None, # 26
+    None, # 27
+    None, # 28
+    None, # 29
+    None, # 30
+    None, # 31
+    None, # 32
+    None, # 33
+    None, # 34
+    None, # 35
+    None, # 36
+    None, # 37
+    None, # 38
+    None, # 39
+    None, # 40
+    None, # 41
+    None, # 42
+    None, # 43
+    None, # 44
+    None, # 45
+    None, # 46
+    None, # 47
+    None, # 48
+    None, # 49
+    None, # 50
+    None, # 51
+    None, # 52
+    None, # 53
+    None, # 54
+    None, # 55
+    None, # 56
+    None, # 57
+    None, # 58
+    None, # 59
+    None, # 60
+    None, # 61
+    None, # 62
+    None, # 63
+    None, # 64
+    None, # 65
+    None, # 66
+    None, # 67
+    None, # 68
+    None, # 69
+    None, # 70
+    None, # 71
+    None, # 72
+    None, # 73
+    None, # 74
+    None, # 75
+    None, # 76
+    None, # 77
+    None, # 78
+    None, # 79
+    None, # 80
+    None, # 81
+    None, # 82
+    None, # 83
+    None, # 84
+    None, # 85
+    None, # 86
+    None, # 87
+    None, # 88
+    None, # 89
+    None, # 90
+    None, # 91
+    None, # 92
+    None, # 93
+    None, # 94
+    None, # 95
+    None, # 96
+    None, # 97
+    None, # 98
+    None, # 99
+    None, # 100
+    None, # 101
+    None, # 102
+    None, # 103
+    None, # 104
+    None, # 105
+    None, # 106
+    None, # 107
+    None, # 108
+    None, # 109
+    None, # 110
+    None, # 111
+    None, # 112
+    None, # 113
+    None, # 114
+    None, # 115
+    None, # 116
+    None, # 117
+    None, # 118
+    None, # 119
+    None, # 120
+    None, # 121
+    None, # 122
+    None, # 123
+    None, # 124
+    None, # 125
+    None, # 126
+    None, # 127
+    None, # 128
+    None, # 129
+    None, # 130
+    None, # 131
+    None, # 132
+    None, # 133
+    None, # 134
+    None, # 135
+    None, # 136
+    None, # 137
+    None, # 138
+    None, # 139
+    None, # 140
+    None, # 141
+    None, # 142
+    None, # 143
+    None, # 144
+    None, # 145
+    None, # 146
+    None, # 147
+    None, # 148
+    None, # 149
+    None, # 150
+    None, # 151
+    None, # 152
+    None, # 153
+    None, # 154
+    None, # 155
+    None, # 156
+    None, # 157
+    None, # 158
+    None, # 159
+    None, # 160
+    None, # 161
+    None, # 162
+    None, # 163
+    None, # 164
+    None, # 165
+    None, # 166
+    None, # 167
+    None, # 168
+    None, # 169
+    None, # 170
+    None, # 171
+    None, # 172
+    None, # 173
+    None, # 174
+    None, # 175
+    None, # 176
+    None, # 177
+    None, # 178
+    None, # 179
+    None, # 180
+    None, # 181
+    None, # 182
+    None, # 183
+    None, # 184
+    None, # 185
+    None, # 186
+    None, # 187
+    None, # 188
+    None, # 189
+    None, # 190
+    None, # 191
+    None, # 192
+    None, # 193
+    None, # 194
+    None, # 195
+    None, # 196
+    None, # 197
+    None, # 198
+    None, # 199
+    None, # 200
+    None, # 201
+    None, # 202
+    None, # 203
+    None, # 204
+    None, # 205
+    None, # 206
+    None, # 207
+    None, # 208
+    None, # 209
+    None, # 210
+    None, # 211
+    None, # 212
+    None, # 213
+    None, # 214
+    None, # 215
+    None, # 216
+    None, # 217
+    None, # 218
+    None, # 219
+    None, # 220
+    None, # 221
+    None, # 222
+    None, # 223
+    None, # 224
+    None, # 225
+    None, # 226
+    None, # 227
+    None, # 228
+    None, # 229
+    None, # 230
+    None, # 231
+    None, # 232
+    None, # 233
+    None, # 234
+    None, # 235
+    None, # 236
+    None, # 237
+    None, # 238
+    None, # 239
+    None, # 240
+    None, # 241
+    None, # 242
+    None, # 243
+    None, # 244
+    None, # 245
+    None, # 246
+    None, # 247
+    None, # 248
+    None, # 249
+    None, # 250
+    None, # 251
+    None, # 252
+    None, # 253
+    None, # 254
+    None, # 255
+    None, # 256
+    None, # 257
+    None, # 258
+    None, # 259
+    None, # 260
+    None, # 261
+    None, # 262
+    None, # 263
+    None, # 264
+    None, # 265
+    None, # 266
+    None, # 267
+    None, # 268
+    None, # 269
+    None, # 270
+    None, # 271
+    None, # 272
+    None, # 273
+    None, # 274
+    None, # 275
+    None, # 276
+    None, # 277
+    None, # 278
+    None, # 279
+    None, # 280
+    None, # 281
+    None, # 282
+    None, # 283
+    None, # 284
+    None, # 285
+    None, # 286
+    None, # 287
+    None, # 288
+    None, # 289
+    None, # 290
+    None, # 291
+    None, # 292
+    None, # 293
+    None, # 294
+    None, # 295
+    None, # 296
+    None, # 297
+    None, # 298
+    None, # 299
+    None, # 300
+    None, # 301
+    None, # 302
+    None, # 303
+    None, # 304
+    None, # 305
+    None, # 306
+    None, # 307
+    None, # 308
+    None, # 309
+    None, # 310
+    None, # 311
+    None, # 312
+    None, # 313
+    None, # 314
+    None, # 315
+    None, # 316
+    None, # 317
+    None, # 318
+    None, # 319
+    None, # 320
+    None, # 321
+    None, # 322
+    None, # 323
+    None, # 324
+    None, # 325
+    None, # 326
+    None, # 327
+    None, # 328
+    None, # 329
+    None, # 330
+    None, # 331
+    None, # 332
+    None, # 333
+    None, # 334
+    None, # 335
+    None, # 336
+    None, # 337
+    None, # 338
+    None, # 339
+    None, # 340
+    None, # 341
+    None, # 342
+    None, # 343
+    None, # 344
+    None, # 345
+    None, # 346
+    None, # 347
+    None, # 348
+    None, # 349
+    None, # 350
+    None, # 351
+    None, # 352
+    None, # 353
+    None, # 354
+    None, # 355
+    None, # 356
+    None, # 357
+    None, # 358
+    None, # 359
+    None, # 360
+    None, # 361
+    None, # 362
+    None, # 363
+    None, # 364
+    None, # 365
+    None, # 366
+    None, # 367
+    None, # 368
+    None, # 369
+    None, # 370
+    None, # 371
+    None, # 372
+    None, # 373
+    None, # 374
+    None, # 375
+    None, # 376
+    None, # 377
+    None, # 378
+    None, # 379
+    None, # 380
+    None, # 381
+    None, # 382
+    None, # 383
+    None, # 384
+    None, # 385
+    None, # 386
+    None, # 387
+    None, # 388
+    None, # 389
+    None, # 390
+    None, # 391
+    None, # 392
+    None, # 393
+    None, # 394
+    None, # 395
+    None, # 396
+    None, # 397
+    None, # 398
+    None, # 399
+    None, # 400
+    None, # 401
+    None, # 402
+    None, # 403
+    None, # 404
+    None, # 405
+    None, # 406
+    None, # 407
+    None, # 408
+    None, # 409
+    None, # 410
+    None, # 411
+    None, # 412
+    None, # 413
+    None, # 414
+    None, # 415
+    None, # 416
+    None, # 417
+    None, # 418
+    None, # 419
+    None, # 420
+    None, # 421
+    None, # 422
+    None, # 423
+    None, # 424
+    None, # 425
+    None, # 426
+    None, # 427
+    None, # 428
+    None, # 429
+    None, # 430
+    None, # 431
+    None, # 432
+    None, # 433
+    None, # 434
+    None, # 435
+    None, # 436
+    None, # 437
+    None, # 438
+    None, # 439
+    None, # 440
+    None, # 441
+    None, # 442
+    None, # 443
+    None, # 444
+    None, # 445
+    None, # 446
+    None, # 447
+    None, # 448
+    None, # 449
+    None, # 450
+    None, # 451
+    None, # 452
+    None, # 453
+    None, # 454
+    None, # 455
+    None, # 456
+    None, # 457
+    None, # 458
+    None, # 459
+    None, # 460
+    None, # 461
+    None, # 462
+    None, # 463
+    None, # 464
+    None, # 465
+    None, # 466
+    None, # 467
+    None, # 468
+    None, # 469
+    None, # 470
+    None, # 471
+    None, # 472
+    None, # 473
+    None, # 474
+    None, # 475
+    None, # 476
+    None, # 477
+    None, # 478
+    None, # 479
+    None, # 480
+    None, # 481
+    None, # 482
+    None, # 483
+    None, # 484
+    None, # 485
+    None, # 486
+    None, # 487
+    None, # 488
+    None, # 489
+    None, # 490
+    None, # 491
+    None, # 492
+    None, # 493
+    None, # 494
+    None, # 495
+    None, # 496
+    None, # 497
+    None, # 498
+    None, # 499
+    None, # 500
+    None, # 501
+    None, # 502
+    None, # 503
+    None, # 504
+    None, # 505
+    None, # 506
+    None, # 507
+    None, # 508
+    None, # 509
+    None, # 510
+    None, # 511
+    None, # 512
+    None, # 513
+    None, # 514
+    None, # 515
+    None, # 516
+    None, # 517
+    None, # 518
+    None, # 519
+    None, # 520
+    (521, TType.DOUBLE, 'requested_memonheap', None, None, ), # 521
+    (522, TType.DOUBLE, 'requested_memoffheap', None, None, ), # 522
+    (523, TType.DOUBLE, 'requested_cpu', None, None, ), # 523
+    (524, TType.DOUBLE, 'assigned_memonheap', None, None, ), # 524
+    (525, TType.DOUBLE, 'assigned_memoffheap', None, None, ), # 525
+    (526, TType.DOUBLE, 'assigned_cpu', None, None, ), # 526
+  )
+
+  def __init__(self, id=None, name=None, uptime_secs=None, status=None, num_tasks=None, num_workers=None, num_executors=None, topology_conf=None, id_to_spout_agg_stats=None, id_to_bolt_agg_stats=None, sched_status=None, topology_stats=None, owner=None, debug_options=None, replication_count=None, requested_memonheap=None, requested_memoffheap=None, requested_cpu=None, assigned_memonheap=None, assigned_memoffheap=None, assigned_cpu=None,):
+    self.id = id
+    self.name = name
+    self.uptime_secs = uptime_secs
+    self.status = status
+    self.num_tasks = num_tasks
+    self.num_workers = num_workers
+    self.num_executors = num_executors
+    self.topology_conf = topology_conf
+    self.id_to_spout_agg_stats = id_to_spout_agg_stats
+    self.id_to_bolt_agg_stats = id_to_bolt_agg_stats
+    self.sched_status = sched_status
+    self.topology_stats = topology_stats
+    self.owner = owner
+    self.debug_options = debug_options
+    self.replication_count = replication_count
+    self.requested_memonheap = requested_memonheap
+    self.requested_memoffheap = requested_memoffheap
+    self.requested_cpu = requested_cpu
+    self.assigned_memonheap = assigned_memonheap
+    self.assigned_memoffheap = assigned_memoffheap
+    self.assigned_cpu = assigned_cpu
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.uptime_secs = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.status = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.num_tasks = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.I32:
+          self.num_workers = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I32:
+          self.num_executors = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.STRING:
+          self.topology_conf = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.MAP:
+          self.id_to_spout_agg_stats = {}
+          (_ktype361, _vtype362, _size360 ) = iprot.readMapBegin()
+          for _i364 in xrange(_size360):
+            _key365 = iprot.readString().decode('utf-8')
+            _val366 = ComponentAggregateStats()
+            _val366.read(iprot)
+            self.id_to_spout_agg_stats[_key365] = _val366
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 10:
+        if ftype == TType.MAP:
+          self.id_to_bolt_agg_stats = {}
+          (_ktype368, _vtype369, _size367 ) = iprot.readMapBegin()
+          for _i371 in xrange(_size367):
+            _key372 = iprot.readString().decode('utf-8')
+            _val373 = ComponentAggregateStats()
+            _val373.read(iprot)
+            self.id_to_bolt_agg_stats[_key372] = _val373
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 11:
+        if ftype == TType.STRING:
+          self.sched_status = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 12:
+        if ftype == TType.STRUCT:
+          self.topology_stats = TopologyStats()
+          self.topology_stats.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 13:
+        if ftype == TType.STRING:
+          self.owner = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 14:
+        if ftype == TType.STRUCT:
+          self.debug_options = DebugOptions()
+          self.debug_options.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 15:
+        if ftype == TType.I32:
+          self.replication_count = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 521:
+        if ftype == TType.DOUBLE:
+          self.requested_memonheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 522:
+        if ftype == TType.DOUBLE:
+          self.requested_memoffheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 523:
+        if ftype == TType.DOUBLE:
+          self.requested_cpu = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 524:
+        if ftype == TType.DOUBLE:
+          self.assigned_memonheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 525:
+        if ftype == TType.DOUBLE:
+          self.assigned_memoffheap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 526:
+        if ftype == TType.DOUBLE:
+          self.assigned_cpu = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TopologyPageInfo')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 2)
+      oprot.writeString(self.name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.uptime_secs is not None:
+      oprot.writeFieldBegin('uptime_secs', TType.I32, 3)
+      oprot.writeI32(self.uptime_secs)
+      oprot.writeFieldEnd()
+    if self.status is not None:
+      oprot.writeFieldBegin('status', TType.STRING, 4)
+      oprot.writeString(self.status.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.num_tasks is not None:
+      oprot.writeFieldBegin('num_tasks', TType.I32, 5)
+      oprot.writeI32(self.num_tasks)
+      oprot.writeFieldEnd()
+    if self.num_workers is not None:
+      oprot.writeFieldBegin('num_workers', TType.I32, 6)
+      oprot.writeI32(self.num_workers)
+      oprot.writeFieldEnd()
+    if self.num_executors is not None:
+      oprot.writeFieldBegin('num_executors', TType.I32, 7)
+      oprot.writeI32(self.num_executors)
+      oprot.writeFieldEnd()
+    if self.topology_conf is not None:
+      oprot.writeFieldBegin('topology_conf', TType.STRING, 8)
+      oprot.writeString(self.topology_conf.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.id_to_spout_agg_stats is not None:
+      oprot.writeFieldBegin('id_to_spout_agg_stats', TType.MAP, 9)
+      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.id_to_spout_agg_stats))
+      for kiter374,viter375 in self.id_to_spout_agg_stats.items():
+        oprot.writeString(kiter374.encode('utf-8'))
+        viter375.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 kiter376,viter377 in self.id_to_bolt_agg_stats.items():
+        oprot.writeString(kiter376.encode('utf-8'))
+        viter377.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.sched_status is not None:
+      oprot.writeFieldBegin('sched_status', TType.STRING, 11)
+      oprot.writeString(self.sched_status.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.topology_stats is not None:
+      oprot.writeFieldBegin('topology_stats', TType.STRUCT, 12)
+      self.topology_stats.write(oprot)
+      oprot.writeFieldEnd()
+    if self.owner is not None:
+      oprot.writeFieldBegin('owner', TType.STRING, 13)
+      oprot.writeString(self.owner.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.debug_options is not None:
+      oprot.writeFieldBegin('debug_options', TType.STRUCT, 14)
+      self.debug_options.write(oprot)
+      oprot.writeFieldEnd()
+    if self.replication_count is not None:
+      oprot.writeFieldBegin('replication_count', TType.I32, 15)
+      oprot.writeI32(self.replication_count)
+      oprot.writeFieldEnd()
+    if self.requested_memonheap is not None:
+      oprot.writeFieldBegin('requested_memonheap', TType.DOUBLE, 521)
+      oprot.writeDouble(self.requested_memonheap)
+      oprot.writeFieldEnd()
+    if self.requested_memoffheap is not None:
+      oprot.writeFieldBegin('requested_memoffheap', TType.DOUBLE, 522)
+      oprot.writeDouble(self.requested_memoffheap)
+      oprot.writeFieldEnd()
+    if self.requested_cpu is not None:
+      oprot.writeFieldBegin('requested_cpu', TType.DOUBLE, 523)
+      oprot.writeDouble(self.requested_cpu)
+      oprot.writeFieldEnd()
+    if self.assigned_memonheap is not None:
+      oprot.writeFieldBegin('assigned_memonheap', TType.DOUBLE, 524)
+      oprot.writeDouble(self.assigned_memonheap)
+      oprot.writeFieldEnd()
+    if self.assigned_memoffheap is not None:
+      oprot.writeFieldBegin('assigned_memoffheap', TType.DOUBLE, 525)
+      oprot.writeDouble(self.assigned_memoffheap)
+      oprot.writeFieldEnd()
+    if self.assigned_cpu is not None:
+      oprot.writeFieldBegin('assigned_cpu', TType.DOUBLE, 526)
+      oprot.writeDouble(self.assigned_cpu)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.id is None:
+      raise TProtocol.TProtocolException(message='Required field id is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.id)
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.uptime_secs)
+    value = (value * 31) ^ hash(self.status)
+    value = (value * 31) ^ hash(self.num_tasks)
+    value = (value * 31) ^ hash(self.num_workers)
+    value = (value * 31) ^ hash(self.num_executors)
+    value = (value * 31) ^ hash(self.topology_conf)
+    value = (value * 31) ^ hash(self.id_to_spout_agg_stats)
+    value = (value * 31) ^ hash(self.id_to_bolt_agg_stats)
+    value = (value * 31) ^ hash(self.sched_status)
+    value = (value * 31) ^ hash(self.topology_stats)
+    value = (value * 31) ^ hash(self.owner)
+    value = (value * 31) ^ hash(self.debug_options)
+    value = (value * 31) ^ hash(self.replication_count)
+    value = (value * 31) ^ hash(self.requested_memonheap)
+    value = (value * 31) ^ hash(self.requested_memoffheap)
+    value = (value * 31) ^ hash(self.requested_cpu)
+    value = (value * 31) ^ hash(self.assigned_memonheap)
+    value = (value * 31) ^ hash(self.assigned_memoffheap)
+    value = (value * 31) ^ hash(self.assigned_cpu)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ExecutorAggregateStats:
+  """
+  Attributes:
+   - exec_summary
+   - stats
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'exec_summary', (ExecutorSummary, ExecutorSummary.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'stats', (ComponentAggregateStats, ComponentAggregateStats.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, exec_summary=None, stats=None,):
+    self.exec_summary = exec_summary
+    self.stats = stats
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.exec_summary = ExecutorSummary()
+          self.exec_summary.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.stats = ComponentAggregateStats()
+          self.stats.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ExecutorAggregateStats')
+    if self.exec_summary is not None:
+      oprot.writeFieldBegin('exec_summary', TType.STRUCT, 1)
+      self.exec_summary.write(oprot)
+      oprot.writeFieldEnd()
+    if self.stats is not None:
+      oprot.writeFieldBegin('stats', TType.STRUCT, 2)
+      self.stats.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.exec_summary)
+    value = (value * 31) ^ hash(self.stats)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ComponentPageInfo:
+  """
+  Attributes:
+   - component_id
+   - component_type
+   - topology_id
+   - topology_name
+   - num_executors
+   - num_tasks
+   - window_to_stats
+   - gsid_to_input_stats
+   - sid_to_output_stats
+   - exec_stats
+   - errors
+   - eventlog_host
+   - eventlog_port
+   - debug_options
+   - topology_status
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'component_id', None, None, ), # 1
+    (2, TType.I32, 'component_type', None, None, ), # 2
+    (3, TType.STRING, 'topology_id', None, None, ), # 3
+    (4, TType.STRING, 'topology_name', None, None, ), # 4
+    (5, TType.I32, 'num_executors', None, None, ), # 5
+    (6, TType.I32, 'num_tasks', None, None, ), # 6
+    (7, TType.MAP, 'window_to_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 7
+    (8, TType.MAP, 'gsid_to_input_stats', (TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 8
+    (9, TType.MAP, 'sid_to_output_stats', (TType.STRING,None,TType.STRUCT,(ComponentAggregateStats, ComponentAggregateStats.thrift_spec)), None, ), # 9
+    (10, TType.LIST, 'exec_stats', (TType.STRUCT,(ExecutorAggregateStats, ExecutorAggregateStats.thrift_spec)), None, ), # 10
+    (11, TType.LIST, 'errors', (TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec)), None, ), # 11
+    (12, TType.STRING, 'eventlog_host', None, None, ), # 12
+    (13, TType.I32, 'eventlog_port', None, None, ), # 13
+    (14, TType.STRUCT, 'debug_options', (DebugOptions, DebugOptions.thrift_spec), None, ), # 14
+    (15, TType.STRING, 'topology_status', None, None, ), # 15
+  )
+
+  def __init__(self, component_id=None, component_type=None, topology_id=None, topology_name=None, num_executors=None, num_tasks=None, window_to_stats=None, gsid_to_input_stats=None, sid_to_output_stats=None, exec_stats=None, errors=None, eventlog_host=None, eventlog_port=None, debug_options=None, topology_status=None,):
+    self.component_id = component_id
+    self.component_type = component_type
+    self.topology_id = topology_id
+    self.topology_name = topology_name
+    self.num_executors = num_executors
+    self.num_tasks = num_tasks
+    self.window_to_stats = window_to_stats
+    self.gsid_to_input_stats = gsid_to_input_stats
+    self.sid_to_output_stats = sid_to_output_stats
+    self.exec_stats = exec_stats
+    self.errors = errors
+    self.eventlog_host = eventlog_host
+    self.eventlog_port = eventlog_port
+    self.debug_options = debug_options
+    self.topology_status = topology_status
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.component_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.component_type = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.topology_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.topology_name = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.num_executors = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.I32:
+          self.num_tasks = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.MAP:
+          self.window_to_stats = {}
+          (_ktype379, _vtype380, _size378 ) = iprot.readMapBegin()
+          for _i382 in xrange(_size378):
+            _key383 = iprot.readString().decode('utf-8')
+            _val384 = ComponentAggregateStats()
+            _val384.read(iprot)
+            self.window_to_stats[_key383] = _val384
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.MAP:
+          self.gsid_to_input_stats = {}
+          (_ktype386, _vtype387, _size385 ) = iprot.readMapBegin()
+          for _i389 in xrange(_size385):
+            _key390 = GlobalStreamId()
+            _key390.read(iprot)
+            _val391 = ComponentAggregateStats()
+            _val391.read(iprot)
+            self.gsid_to_input_stats[_key390] = _val391
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.MAP:
+          self.sid_to_output_stats = {}
+          (_ktype393, _vtype394, _size392 ) = iprot.readMapBegin()
+          for _i396 in xrange(_size392):
+            _key397 = iprot.readString().decode('utf-8')
+            _val398 = ComponentAggregateStats()
+            _val398.read(iprot)
+            self.sid_to_output_stats[_key397] = _val398
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 10:
+        if ftype == TType.LIST:
+          self.exec_stats = []
+          (_etype402, _size399) = iprot.readListBegin()
+          for _i403 in xrange(_size399):
+            _elem404 = ExecutorAggregateStats()
+            _elem404.read(iprot)
+            self.exec_stats.append(_elem404)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 11:
+        if ftype == TType.LIST:
+          self.errors = []
+          (_etype408, _size405) = iprot.readListBegin()
+          for _i409 in xrange(_size405):
+            _elem410 = ErrorInfo()
+            _elem410.read(iprot)
+            self.errors.append(_elem410)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 12:
+        if ftype == TType.STRING:
+          self.eventlog_host = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 13:
+        if ftype == TType.I32:
+          self.eventlog_port = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 14:
+        if ftype == TType.STRUCT:
+          self.debug_options = DebugOptions()
+          self.debug_options.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 15:
+        if ftype == TType.STRING:
+          self.topology_status = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ComponentPageInfo')
+    if self.component_id is not None:
+      oprot.writeFieldBegin('component_id', TType.STRING, 1)
+      oprot.writeString(self.component_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.component_type is not None:
+      oprot.writeFieldBegin('component_type', TType.I32, 2)
+      oprot.writeI32(self.component_type)
+      oprot.writeFieldEnd()
+    if self.topology_id is not None:
+      oprot.writeFieldBegin('topology_id', TType.STRING, 3)
+      oprot.writeString(self.topology_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.topology_name is not None:
+      oprot.writeFieldBegin('topology_name', TType.STRING, 4)
+      oprot.writeString(self.topology_name.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.num_executors is not None:
+      oprot.writeFieldBegin('num_executors', TType.I32, 5)
+      oprot.writeI32(self.num_executors)
+      oprot.writeFieldEnd()
+    if self.num_tasks is not None:
+      oprot.writeFieldBegin('num_tasks', TType.I32, 6)
+      oprot.writeI32(self.num_tasks)
+      oprot.writeFieldEnd()
+    if self.window_to_stats is not None:
+      oprot.writeFieldBegin('window_to_stats', TType.MAP, 7)
+      oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.window_to_stats))
+      for kiter411,viter412 in self.window_to_stats.items():
+        oprot.writeString(kiter411.encode('utf-8'))
+        viter412.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 kiter413,viter414 in self.gsid_to_input_stats.items():
+        kiter413.write(oprot)
+        viter414.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 kiter415,viter416 in self.sid_to_output_stats.items():
+        oprot.writeString(kiter415.encode('utf-8'))
+        viter416.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 iter417 in self.exec_stats:
+        iter417.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 iter418 in self.errors:
+        iter418.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.eventlog_host is not None:
+      oprot.writeFieldBegin('eventlog_host', TType.STRING, 12)
+      oprot.writeString(self.eventlog_host.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.eventlog_port is not None:
+      oprot.writeFieldBegin('eventlog_port', TType.I32, 13)
+      oprot.writeI32(self.eventlog_port)
+      oprot.writeFieldEnd()
+    if self.debug_options is not None:
+      oprot.writeFieldBegin('debug_options', TType.STRUCT, 14)
+      self.debug_options.write(oprot)
+      oprot.writeFieldEnd()
+    if self.topology_status is not None:
+      oprot.writeFieldBegin('topology_status', TType.STRING, 15)
+      oprot.writeString(self.topology_status.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.component_id is None:
+      raise TProtocol.TProtocolException(message='Required field component_id is unset!')
+    if self.component_type is None:
+      raise TProtocol.TProtocolException(message='Required field component_type is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.component_id)
+    value = (value * 31) ^ hash(self.component_type)
+    value = (value * 31) ^ hash(self.topology_id)
+    value = (value * 31) ^ hash(self.topology_name)
+    value = (value * 31) ^ hash(self.num_executors)
+    value = (value * 31) ^ hash(self.num_tasks)
+    value = (value * 31) ^ hash(self.window_to_stats)
+    value = (value * 31) ^ hash(self.gsid_to_input_stats)
+    value = (value * 31) ^ hash(self.sid_to_output_stats)
+    value = (value * 31) ^ hash(self.exec_stats)
+    value = (value * 31) ^ hash(self.errors)
+    value = (value * 31) ^ hash(self.eventlog_host)
+    value = (value * 31) ^ hash(self.eventlog_port)
+    value = (value * 31) ^ hash(self.debug_options)
+    value = (value * 31) ^ hash(self.topology_status)
     return value
 
   def __repr__(self):
@@ -4548,7 +6811,7 @@
         break
       if fid == 1:
         if ftype == TType.I32:
-          self.wait_secs = iprot.readI32();
+          self.wait_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       else:
@@ -4619,22 +6882,22 @@
         break
       if fid == 1:
         if ftype == TType.I32:
-          self.wait_secs = iprot.readI32();
+          self.wait_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I32:
-          self.num_workers = iprot.readI32();
+          self.num_workers = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.num_executors = {}
-          (_ktype300, _vtype301, _size299 ) = iprot.readMapBegin()
-          for _i303 in xrange(_size299):
-            _key304 = iprot.readString().decode('utf-8')
-            _val305 = iprot.readI32();
-            self.num_executors[_key304] = _val305
+          (_ktype420, _vtype421, _size419 ) = iprot.readMapBegin()
+          for _i423 in xrange(_size419):
+            _key424 = iprot.readString().decode('utf-8')
+            _val425 = iprot.readI32()
+            self.num_executors[_key424] = _val425
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4659,9 +6922,9 @@
     if self.num_executors is not None:
       oprot.writeFieldBegin('num_executors', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.num_executors))
-      for kiter306,viter307 in self.num_executors.items():
-        oprot.writeString(kiter306.encode('utf-8'))
-        oprot.writeI32(viter307)
+      for kiter426,viter427 in self.num_executors.items():
+        oprot.writeString(kiter426.encode('utf-8'))
+        oprot.writeI32(viter427)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4715,11 +6978,11 @@
       if fid == 1:
         if ftype == TType.MAP:
           self.creds = {}
-          (_ktype309, _vtype310, _size308 ) = iprot.readMapBegin()
-          for _i312 in xrange(_size308):
-            _key313 = iprot.readString().decode('utf-8')
-            _val314 = iprot.readString().decode('utf-8')
-            self.creds[_key313] = _val314
+          (_ktype429, _vtype430, _size428 ) = iprot.readMapBegin()
+          for _i432 in xrange(_size428):
+            _key433 = iprot.readString().decode('utf-8')
+            _val434 = iprot.readString().decode('utf-8')
+            self.creds[_key433] = _val434
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -4736,9 +6999,9 @@
     if self.creds is not None:
       oprot.writeFieldBegin('creds', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds))
-      for kiter315,viter316 in self.creds.items():
-        oprot.writeString(kiter315.encode('utf-8'))
-        oprot.writeString(viter316.encode('utf-8'))
+      for kiter435,viter436 in self.creds.items():
+        oprot.writeString(kiter435.encode('utf-8'))
+        oprot.writeString(viter436.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -4794,7 +7057,7 @@
         break
       if fid == 1:
         if ftype == TType.I32:
-          self.initial_status = iprot.readI32();
+          self.initial_status = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 2:
@@ -4858,6 +7121,7 @@
    - scheduler_meta
    - uptime_secs
    - version
+   - resources_map
   """
 
   thrift_spec = (
@@ -4870,9 +7134,10 @@
     (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
@@ -4881,6 +7146,7 @@
     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:
@@ -4893,7 +7159,7 @@
         break
       if fid == 1:
         if ftype == TType.I64:
-          self.time_secs = iprot.readI64();
+          self.time_secs = iprot.readI64()
         else:
           iprot.skip(ftype)
       elif fid == 2:
@@ -4909,37 +7175,37 @@
       elif fid == 4:
         if ftype == TType.LIST:
           self.used_ports = []
-          (_etype320, _size317) = iprot.readListBegin()
-          for _i321 in xrange(_size317):
-            _elem322 = iprot.readI64();
-            self.used_ports.append(_elem322)
+          (_etype440, _size437) = iprot.readListBegin()
+          for _i441 in xrange(_size437):
+            _elem442 = iprot.readI64()
+            self.used_ports.append(_elem442)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.meta = []
-          (_etype326, _size323) = iprot.readListBegin()
-          for _i327 in xrange(_size323):
-            _elem328 = iprot.readI64();
-            self.meta.append(_elem328)
+          (_etype446, _size443) = iprot.readListBegin()
+          for _i447 in xrange(_size443):
+            _elem448 = iprot.readI64()
+            self.meta.append(_elem448)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.MAP:
           self.scheduler_meta = {}
-          (_ktype330, _vtype331, _size329 ) = iprot.readMapBegin()
-          for _i333 in xrange(_size329):
-            _key334 = iprot.readString().decode('utf-8')
-            _val335 = iprot.readString().decode('utf-8')
-            self.scheduler_meta[_key334] = _val335
+          (_ktype450, _vtype451, _size449 ) = iprot.readMapBegin()
+          for _i453 in xrange(_size449):
+            _key454 = iprot.readString().decode('utf-8')
+            _val455 = iprot.readString().decode('utf-8')
+            self.scheduler_meta[_key454] = _val455
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.I64:
-          self.uptime_secs = iprot.readI64();
+          self.uptime_secs = iprot.readI64()
         else:
           iprot.skip(ftype)
       elif fid == 8:
@@ -4947,6 +7213,17 @@
           self.version = iprot.readString().decode('utf-8')
         else:
           iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.MAP:
+          self.resources_map = {}
+          (_ktype457, _vtype458, _size456 ) = iprot.readMapBegin()
+          for _i460 in xrange(_size456):
+            _key461 = iprot.readString().decode('utf-8')
+            _val462 = iprot.readDouble()
+            self.resources_map[_key461] = _val462
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -4972,23 +7249,23 @@
     if self.used_ports is not None:
       oprot.writeFieldBegin('used_ports', TType.LIST, 4)
       oprot.writeListBegin(TType.I64, len(self.used_ports))
-      for iter336 in self.used_ports:
-        oprot.writeI64(iter336)
+      for iter463 in self.used_ports:
+        oprot.writeI64(iter463)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.meta is not None:
       oprot.writeFieldBegin('meta', TType.LIST, 5)
       oprot.writeListBegin(TType.I64, len(self.meta))
-      for iter337 in self.meta:
-        oprot.writeI64(iter337)
+      for iter464 in self.meta:
+        oprot.writeI64(iter464)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.scheduler_meta is not None:
       oprot.writeFieldBegin('scheduler_meta', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.scheduler_meta))
-      for kiter338,viter339 in self.scheduler_meta.items():
-        oprot.writeString(kiter338.encode('utf-8'))
-        oprot.writeString(viter339.encode('utf-8'))
+      for kiter465,viter466 in self.scheduler_meta.items():
+        oprot.writeString(kiter465.encode('utf-8'))
+        oprot.writeString(viter466.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.uptime_secs is not None:
@@ -4999,6 +7276,14 @@
       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 kiter467,viter468 in self.resources_map.items():
+        oprot.writeString(kiter467.encode('utf-8'))
+        oprot.writeDouble(viter468)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -5020,6 +7305,7 @@
     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):
@@ -5067,10 +7353,10 @@
       elif fid == 2:
         if ftype == TType.SET:
           self.port = set()
-          (_etype343, _size340) = iprot.readSetBegin()
-          for _i344 in xrange(_size340):
-            _elem345 = iprot.readI64();
-            self.port.add(_elem345)
+          (_etype472, _size469) = iprot.readSetBegin()
+          for _i473 in xrange(_size469):
+            _elem474 = iprot.readI64()
+            self.port.add(_elem474)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -5091,8 +7377,8 @@
     if self.port is not None:
       oprot.writeFieldBegin('port', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.port))
-      for iter346 in self.port:
-        oprot.writeI64(iter346)
+      for iter475 in self.port:
+        oprot.writeI64(iter475)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5123,6 +7409,97 @@
   def __ne__(self, other):
     return not (self == other)
 
+class WorkerResources:
+  """
+  Attributes:
+   - mem_on_heap
+   - mem_off_heap
+   - cpu
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.DOUBLE, 'mem_on_heap', None, None, ), # 1
+    (2, TType.DOUBLE, 'mem_off_heap', None, None, ), # 2
+    (3, TType.DOUBLE, 'cpu', None, None, ), # 3
+  )
+
+  def __init__(self, mem_on_heap=None, mem_off_heap=None, cpu=None,):
+    self.mem_on_heap = mem_on_heap
+    self.mem_off_heap = mem_off_heap
+    self.cpu = cpu
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.DOUBLE:
+          self.mem_on_heap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.DOUBLE:
+          self.mem_off_heap = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.DOUBLE:
+          self.cpu = iprot.readDouble()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WorkerResources')
+    if self.mem_on_heap is not None:
+      oprot.writeFieldBegin('mem_on_heap', TType.DOUBLE, 1)
+      oprot.writeDouble(self.mem_on_heap)
+      oprot.writeFieldEnd()
+    if self.mem_off_heap is not None:
+      oprot.writeFieldBegin('mem_off_heap', TType.DOUBLE, 2)
+      oprot.writeDouble(self.mem_off_heap)
+      oprot.writeFieldEnd()
+    if self.cpu is not None:
+      oprot.writeFieldBegin('cpu', TType.DOUBLE, 3)
+      oprot.writeDouble(self.cpu)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.mem_on_heap)
+    value = (value * 31) ^ hash(self.mem_off_heap)
+    value = (value * 31) ^ hash(self.cpu)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class Assignment:
   """
   Attributes:
@@ -5130,6 +7507,7 @@
    - node_host
    - executor_node_port
    - executor_start_time_secs
+   - worker_resources
   """
 
   thrift_spec = (
@@ -5141,9 +7519,11 @@
     }, ), # 3
     (4, TType.MAP, 'executor_start_time_secs', (TType.LIST,(TType.I64,None),TType.I64,None), {
     }, ), # 4
+    (5, TType.MAP, 'worker_resources', (TType.STRUCT,(NodeInfo, NodeInfo.thrift_spec),TType.STRUCT,(WorkerResources, WorkerResources.thrift_spec)), {
+    }, ), # 5
   )
 
-  def __init__(self, master_code_dir=None, node_host=thrift_spec[2][4], executor_node_port=thrift_spec[3][4], executor_start_time_secs=thrift_spec[4][4],):
+  def __init__(self, master_code_dir=None, node_host=thrift_spec[2][4], executor_node_port=thrift_spec[3][4], executor_start_time_secs=thrift_spec[4][4], worker_resources=thrift_spec[5][4],):
     self.master_code_dir = master_code_dir
     if node_host is self.thrift_spec[2][4]:
       node_host = {
@@ -5157,6 +7537,10 @@
       executor_start_time_secs = {
     }
     self.executor_start_time_secs = executor_start_time_secs
+    if worker_resources is self.thrift_spec[5][4]:
+      worker_resources = {
+    }
+    self.worker_resources = worker_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:
@@ -5175,44 +7559,57 @@
       elif fid == 2:
         if ftype == TType.MAP:
           self.node_host = {}
-          (_ktype348, _vtype349, _size347 ) = iprot.readMapBegin()
-          for _i351 in xrange(_size347):
-            _key352 = iprot.readString().decode('utf-8')
-            _val353 = iprot.readString().decode('utf-8')
-            self.node_host[_key352] = _val353
+          (_ktype477, _vtype478, _size476 ) = iprot.readMapBegin()
+          for _i480 in xrange(_size476):
+            _key481 = iprot.readString().decode('utf-8')
+            _val482 = iprot.readString().decode('utf-8')
+            self.node_host[_key481] = _val482
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.MAP:
           self.executor_node_port = {}
-          (_ktype355, _vtype356, _size354 ) = iprot.readMapBegin()
-          for _i358 in xrange(_size354):
-            _key359 = []
-            (_etype364, _size361) = iprot.readListBegin()
-            for _i365 in xrange(_size361):
-              _elem366 = iprot.readI64();
-              _key359.append(_elem366)
+          (_ktype484, _vtype485, _size483 ) = iprot.readMapBegin()
+          for _i487 in xrange(_size483):
+            _key488 = []
+            (_etype493, _size490) = iprot.readListBegin()
+            for _i494 in xrange(_size490):
+              _elem495 = iprot.readI64()
+              _key488.append(_elem495)
             iprot.readListEnd()
-            _val360 = NodeInfo()
-            _val360.read(iprot)
-            self.executor_node_port[_key359] = _val360
+            _val489 = NodeInfo()
+            _val489.read(iprot)
+            self.executor_node_port[_key488] = _val489
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.executor_start_time_secs = {}
-          (_ktype368, _vtype369, _size367 ) = iprot.readMapBegin()
-          for _i371 in xrange(_size367):
-            _key372 = []
-            (_etype377, _size374) = iprot.readListBegin()
-            for _i378 in xrange(_size374):
-              _elem379 = iprot.readI64();
-              _key372.append(_elem379)
+          (_ktype497, _vtype498, _size496 ) = iprot.readMapBegin()
+          for _i500 in xrange(_size496):
+            _key501 = []
+            (_etype506, _size503) = iprot.readListBegin()
+            for _i507 in xrange(_size503):
+              _elem508 = iprot.readI64()
+              _key501.append(_elem508)
             iprot.readListEnd()
-            _val373 = iprot.readI64();
-            self.executor_start_time_secs[_key372] = _val373
+            _val502 = iprot.readI64()
+            self.executor_start_time_secs[_key501] = _val502
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.MAP:
+          self.worker_resources = {}
+          (_ktype510, _vtype511, _size509 ) = iprot.readMapBegin()
+          for _i513 in xrange(_size509):
+            _key514 = NodeInfo()
+            _key514.read(iprot)
+            _val515 = WorkerResources()
+            _val515.read(iprot)
+            self.worker_resources[_key514] = _val515
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5233,31 +7630,39 @@
     if self.node_host is not None:
       oprot.writeFieldBegin('node_host', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.node_host))
-      for kiter380,viter381 in self.node_host.items():
-        oprot.writeString(kiter380.encode('utf-8'))
-        oprot.writeString(viter381.encode('utf-8'))
+      for kiter516,viter517 in self.node_host.items():
+        oprot.writeString(kiter516.encode('utf-8'))
+        oprot.writeString(viter517.encode('utf-8'))
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_node_port is not None:
       oprot.writeFieldBegin('executor_node_port', TType.MAP, 3)
       oprot.writeMapBegin(TType.LIST, TType.STRUCT, len(self.executor_node_port))
-      for kiter382,viter383 in self.executor_node_port.items():
-        oprot.writeListBegin(TType.I64, len(kiter382))
-        for iter384 in kiter382:
-          oprot.writeI64(iter384)
+      for kiter518,viter519 in self.executor_node_port.items():
+        oprot.writeListBegin(TType.I64, len(kiter518))
+        for iter520 in kiter518:
+          oprot.writeI64(iter520)
         oprot.writeListEnd()
-        viter383.write(oprot)
+        viter519.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.executor_start_time_secs is not None:
       oprot.writeFieldBegin('executor_start_time_secs', TType.MAP, 4)
       oprot.writeMapBegin(TType.LIST, TType.I64, len(self.executor_start_time_secs))
-      for kiter385,viter386 in self.executor_start_time_secs.items():
-        oprot.writeListBegin(TType.I64, len(kiter385))
-        for iter387 in kiter385:
-          oprot.writeI64(iter387)
+      for kiter521,viter522 in self.executor_start_time_secs.items():
+        oprot.writeListBegin(TType.I64, len(kiter521))
+        for iter523 in kiter521:
+          oprot.writeI64(iter523)
         oprot.writeListEnd()
-        oprot.writeI64(viter386)
+        oprot.writeI64(viter522)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.worker_resources is not None:
+      oprot.writeFieldBegin('worker_resources', TType.MAP, 5)
+      oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.worker_resources))
+      for kiter524,viter525 in self.worker_resources.items():
+        kiter524.write(oprot)
+        viter525.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5275,6 +7680,7 @@
     value = (value * 31) ^ hash(self.node_host)
     value = (value * 31) ^ hash(self.executor_node_port)
     value = (value * 31) ^ hash(self.executor_start_time_secs)
+    value = (value * 31) ^ hash(self.worker_resources)
     return value
 
   def __repr__(self):
@@ -5422,28 +7828,28 @@
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.I32:
-          self.status = iprot.readI32();
+          self.status = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.I32:
-          self.num_workers = iprot.readI32();
+          self.num_workers = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.MAP:
           self.component_executors = {}
-          (_ktype389, _vtype390, _size388 ) = iprot.readMapBegin()
-          for _i392 in xrange(_size388):
-            _key393 = iprot.readString().decode('utf-8')
-            _val394 = iprot.readI32();
-            self.component_executors[_key393] = _val394
+          (_ktype527, _vtype528, _size526 ) = iprot.readMapBegin()
+          for _i530 in xrange(_size526):
+            _key531 = iprot.readString().decode('utf-8')
+            _val532 = iprot.readI32()
+            self.component_executors[_key531] = _val532
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.I32:
-          self.launch_time_secs = iprot.readI32();
+          self.launch_time_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 6:
@@ -5459,18 +7865,18 @@
           iprot.skip(ftype)
       elif fid == 8:
         if ftype == TType.I32:
-          self.prev_status = iprot.readI32();
+          self.prev_status = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 9:
         if ftype == TType.MAP:
           self.component_debug = {}
-          (_ktype396, _vtype397, _size395 ) = iprot.readMapBegin()
-          for _i399 in xrange(_size395):
-            _key400 = iprot.readString().decode('utf-8')
-            _val401 = DebugOptions()
-            _val401.read(iprot)
-            self.component_debug[_key400] = _val401
+          (_ktype534, _vtype535, _size533 ) = iprot.readMapBegin()
+          for _i537 in xrange(_size533):
+            _key538 = iprot.readString().decode('utf-8')
+            _val539 = DebugOptions()
+            _val539.read(iprot)
+            self.component_debug[_key538] = _val539
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5499,9 +7905,9 @@
     if self.component_executors is not None:
       oprot.writeFieldBegin('component_executors', TType.MAP, 4)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.component_executors))
-      for kiter402,viter403 in self.component_executors.items():
-        oprot.writeString(kiter402.encode('utf-8'))
-        oprot.writeI32(viter403)
+      for kiter540,viter541 in self.component_executors.items():
+        oprot.writeString(kiter540.encode('utf-8'))
+        oprot.writeI32(viter541)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.launch_time_secs is not None:
@@ -5523,9 +7929,9 @@
     if self.component_debug is not None:
       oprot.writeFieldBegin('component_debug', TType.MAP, 9)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.component_debug))
-      for kiter404,viter405 in self.component_debug.items():
-        oprot.writeString(kiter404.encode('utf-8'))
-        viter405.write(oprot)
+      for kiter542,viter543 in self.component_debug.items():
+        oprot.writeString(kiter542.encode('utf-8'))
+        viter543.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5605,24 +8011,24 @@
       elif fid == 2:
         if ftype == TType.MAP:
           self.executor_stats = {}
-          (_ktype407, _vtype408, _size406 ) = iprot.readMapBegin()
-          for _i410 in xrange(_size406):
-            _key411 = ExecutorInfo()
-            _key411.read(iprot)
-            _val412 = ExecutorStats()
-            _val412.read(iprot)
-            self.executor_stats[_key411] = _val412
+          (_ktype545, _vtype546, _size544 ) = iprot.readMapBegin()
+          for _i548 in xrange(_size544):
+            _key549 = ExecutorInfo()
+            _key549.read(iprot)
+            _val550 = ExecutorStats()
+            _val550.read(iprot)
+            self.executor_stats[_key549] = _val550
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.I32:
-          self.time_secs = iprot.readI32();
+          self.time_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.I32:
-          self.uptime_secs = iprot.readI32();
+          self.uptime_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       else:
@@ -5642,9 +8048,9 @@
     if self.executor_stats is not None:
       oprot.writeFieldBegin('executor_stats', TType.MAP, 2)
       oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.executor_stats))
-      for kiter413,viter414 in self.executor_stats.items():
-        kiter413.write(oprot)
-        viter414.write(oprot)
+      for kiter551,viter552 in self.executor_stats.items():
+        kiter551.write(oprot)
+        viter552.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.time_secs is not None:
@@ -5722,7 +8128,7 @@
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.STRING:
-          self.bits = iprot.readString();
+          self.bits = iprot.readString()
         else:
           iprot.skip(ftype)
       else:
@@ -5797,12 +8203,12 @@
       if fid == 1:
         if ftype == TType.MAP:
           self.serialized_parts = {}
-          (_ktype416, _vtype417, _size415 ) = iprot.readMapBegin()
-          for _i419 in xrange(_size415):
-            _key420 = iprot.readString().decode('utf-8')
-            _val421 = ThriftSerializedObject()
-            _val421.read(iprot)
-            self.serialized_parts[_key420] = _val421
+          (_ktype554, _vtype555, _size553 ) = iprot.readMapBegin()
+          for _i557 in xrange(_size553):
+            _key558 = iprot.readString().decode('utf-8')
+            _val559 = ThriftSerializedObject()
+            _val559.read(iprot)
+            self.serialized_parts[_key558] = _val559
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -5819,9 +8225,9 @@
     if self.serialized_parts is not None:
       oprot.writeFieldBegin('serialized_parts', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.serialized_parts))
-      for kiter422,viter423 in self.serialized_parts.items():
-        oprot.writeString(kiter422.encode('utf-8'))
-        viter423.write(oprot)
+      for kiter560,viter561 in self.serialized_parts.items():
+        oprot.writeString(kiter560.encode('utf-8'))
+        viter561.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -5854,17 +8260,20 @@
   Attributes:
    - topology_id
    - executors
+   - resources
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRING, 'topology_id', None, None, ), # 1
     (2, TType.LIST, 'executors', (TType.STRUCT,(ExecutorInfo, ExecutorInfo.thrift_spec)), None, ), # 2
+    (3, TType.STRUCT, 'resources', (WorkerResources, WorkerResources.thrift_spec), None, ), # 3
   )
 
-  def __init__(self, topology_id=None, executors=None,):
+  def __init__(self, topology_id=None, executors=None, resources=None,):
     self.topology_id = topology_id
     self.executors = executors
+    self.resources = 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:
@@ -5883,14 +8292,20 @@
       elif fid == 2:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype427, _size424) = iprot.readListBegin()
-          for _i428 in xrange(_size424):
-            _elem429 = ExecutorInfo()
-            _elem429.read(iprot)
-            self.executors.append(_elem429)
+          (_etype565, _size562) = iprot.readListBegin()
+          for _i566 in xrange(_size562):
+            _elem567 = ExecutorInfo()
+            _elem567.read(iprot)
+            self.executors.append(_elem567)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.resources = WorkerResources()
+          self.resources.read(iprot)
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -5908,10 +8323,14 @@
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter430 in self.executors:
-        iter430.write(oprot)
+      for iter568 in self.executors:
+        iter568.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
+    if self.resources is not None:
+      oprot.writeFieldBegin('resources', TType.STRUCT, 3)
+      self.resources.write(oprot)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -5927,6 +8346,7 @@
     value = 17
     value = (value * 31) ^ hash(self.topology_id)
     value = (value * 31) ^ hash(self.executors)
+    value = (value * 31) ^ hash(self.resources)
     return value
 
   def __repr__(self):
@@ -6033,11 +8453,11 @@
       if fid == 1:
         if ftype == TType.MAP:
           self.approved_workers = {}
-          (_ktype432, _vtype433, _size431 ) = iprot.readMapBegin()
-          for _i435 in xrange(_size431):
-            _key436 = iprot.readString().decode('utf-8')
-            _val437 = iprot.readI32();
-            self.approved_workers[_key436] = _val437
+          (_ktype570, _vtype571, _size569 ) = iprot.readMapBegin()
+          for _i573 in xrange(_size569):
+            _key574 = iprot.readString().decode('utf-8')
+            _val575 = iprot.readI32()
+            self.approved_workers[_key574] = _val575
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6054,9 +8474,9 @@
     if self.approved_workers is not None:
       oprot.writeFieldBegin('approved_workers', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.I32, len(self.approved_workers))
-      for kiter438,viter439 in self.approved_workers.items():
-        oprot.writeString(kiter438.encode('utf-8'))
-        oprot.writeI32(viter439)
+      for kiter576,viter577 in self.approved_workers.items():
+        oprot.writeString(kiter576.encode('utf-8'))
+        oprot.writeI32(viter577)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6110,12 +8530,12 @@
       if fid == 1:
         if ftype == TType.MAP:
           self.assignments = {}
-          (_ktype441, _vtype442, _size440 ) = iprot.readMapBegin()
-          for _i444 in xrange(_size440):
-            _key445 = iprot.readI32();
-            _val446 = LocalAssignment()
-            _val446.read(iprot)
-            self.assignments[_key445] = _val446
+          (_ktype579, _vtype580, _size578 ) = iprot.readMapBegin()
+          for _i582 in xrange(_size578):
+            _key583 = iprot.readI32()
+            _val584 = LocalAssignment()
+            _val584.read(iprot)
+            self.assignments[_key583] = _val584
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -6132,9 +8552,9 @@
     if self.assignments is not None:
       oprot.writeFieldBegin('assignments', TType.MAP, 1)
       oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.assignments))
-      for kiter447,viter448 in self.assignments.items():
-        oprot.writeI32(kiter447)
-        viter448.write(oprot)
+      for kiter585,viter586 in self.assignments.items():
+        oprot.writeI32(kiter585)
+        viter586.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -6196,7 +8616,7 @@
         break
       if fid == 1:
         if ftype == TType.I32:
-          self.time_secs = iprot.readI32();
+          self.time_secs = iprot.readI32()
         else:
           iprot.skip(ftype)
       elif fid == 2:
@@ -6207,17 +8627,17 @@
       elif fid == 3:
         if ftype == TType.LIST:
           self.executors = []
-          (_etype452, _size449) = iprot.readListBegin()
-          for _i453 in xrange(_size449):
-            _elem454 = ExecutorInfo()
-            _elem454.read(iprot)
-            self.executors.append(_elem454)
+          (_etype590, _size587) = iprot.readListBegin()
+          for _i591 in xrange(_size587):
+            _elem592 = ExecutorInfo()
+            _elem592.read(iprot)
+            self.executors.append(_elem592)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.I32:
-          self.port = iprot.readI32();
+          self.port = iprot.readI32()
         else:
           iprot.skip(ftype)
       else:
@@ -6241,8 +8661,8 @@
     if self.executors is not None:
       oprot.writeFieldBegin('executors', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.executors))
-      for iter455 in self.executors:
-        iter455.write(oprot)
+      for iter593 in self.executors:
+        iter593.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.port is not None:
@@ -6283,6 +8703,306 @@
   def __ne__(self, other):
     return not (self == other)
 
+class LSTopoHistory:
+  """
+  Attributes:
+   - topology_id
+   - time_stamp
+   - users
+   - groups
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'topology_id', None, None, ), # 1
+    (2, TType.I64, 'time_stamp', None, None, ), # 2
+    (3, TType.LIST, 'users', (TType.STRING,None), None, ), # 3
+    (4, TType.LIST, 'groups', (TType.STRING,None), None, ), # 4
+  )
+
+  def __init__(self, topology_id=None, time_stamp=None, users=None, groups=None,):
+    self.topology_id = topology_id
+    self.time_stamp = time_stamp
+    self.users = users
+    self.groups = groups
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.topology_id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.time_stamp = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.users = []
+          (_etype597, _size594) = iprot.readListBegin()
+          for _i598 in xrange(_size594):
+            _elem599 = iprot.readString().decode('utf-8')
+            self.users.append(_elem599)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.groups = []
+          (_etype603, _size600) = iprot.readListBegin()
+          for _i604 in xrange(_size600):
+            _elem605 = iprot.readString().decode('utf-8')
+            self.groups.append(_elem605)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('LSTopoHistory')
+    if self.topology_id is not None:
+      oprot.writeFieldBegin('topology_id', TType.STRING, 1)
+      oprot.writeString(self.topology_id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.time_stamp is not None:
+      oprot.writeFieldBegin('time_stamp', TType.I64, 2)
+      oprot.writeI64(self.time_stamp)
+      oprot.writeFieldEnd()
+    if self.users is not None:
+      oprot.writeFieldBegin('users', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRING, len(self.users))
+      for iter606 in self.users:
+        oprot.writeString(iter606.encode('utf-8'))
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.groups is not None:
+      oprot.writeFieldBegin('groups', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRING, len(self.groups))
+      for iter607 in self.groups:
+        oprot.writeString(iter607.encode('utf-8'))
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.topology_id is None:
+      raise TProtocol.TProtocolException(message='Required field topology_id is unset!')
+    if self.time_stamp is None:
+      raise TProtocol.TProtocolException(message='Required field time_stamp is unset!')
+    if self.users is None:
+      raise TProtocol.TProtocolException(message='Required field users is unset!')
+    if self.groups is None:
+      raise TProtocol.TProtocolException(message='Required field groups is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.topology_id)
+    value = (value * 31) ^ hash(self.time_stamp)
+    value = (value * 31) ^ hash(self.users)
+    value = (value * 31) ^ hash(self.groups)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class LSTopoHistoryList:
+  """
+  Attributes:
+   - topo_history
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'topo_history', (TType.STRUCT,(LSTopoHistory, LSTopoHistory.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, topo_history=None,):
+    self.topo_history = topo_history
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.topo_history = []
+          (_etype611, _size608) = iprot.readListBegin()
+          for _i612 in xrange(_size608):
+            _elem613 = LSTopoHistory()
+            _elem613.read(iprot)
+            self.topo_history.append(_elem613)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('LSTopoHistoryList')
+    if self.topo_history is not None:
+      oprot.writeFieldBegin('topo_history', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.topo_history))
+      for iter614 in self.topo_history:
+        iter614.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.topo_history is None:
+      raise TProtocol.TProtocolException(message='Required field topo_history is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.topo_history)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ProfileRequest:
+  """
+  Attributes:
+   - nodeInfo
+   - action
+   - time_stamp
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'nodeInfo', (NodeInfo, NodeInfo.thrift_spec), None, ), # 1
+    (2, TType.I32, 'action', None, None, ), # 2
+    (3, TType.I64, 'time_stamp', None, None, ), # 3
+  )
+
+  def __init__(self, nodeInfo=None, action=None, time_stamp=None,):
+    self.nodeInfo = nodeInfo
+    self.action = action
+    self.time_stamp = time_stamp
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.nodeInfo = NodeInfo()
+          self.nodeInfo.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.action = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I64:
+          self.time_stamp = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ProfileRequest')
+    if self.nodeInfo is not None:
+      oprot.writeFieldBegin('nodeInfo', TType.STRUCT, 1)
+      self.nodeInfo.write(oprot)
+      oprot.writeFieldEnd()
+    if self.action is not None:
+      oprot.writeFieldBegin('action', TType.I32, 2)
+      oprot.writeI32(self.action)
+      oprot.writeFieldEnd()
+    if self.time_stamp is not None:
+      oprot.writeFieldBegin('time_stamp', TType.I64, 3)
+      oprot.writeI64(self.time_stamp)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.nodeInfo is None:
+      raise TProtocol.TProtocolException(message='Required field nodeInfo is unset!')
+    if self.action is None:
+      raise TProtocol.TProtocolException(message='Required field action is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.nodeInfo)
+    value = (value * 31) ^ hash(self.action)
+    value = (value * 31) ^ hash(self.time_stamp)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class GetInfoOptions:
   """
   Attributes:
@@ -6308,7 +9028,7 @@
         break
       if fid == 1:
         if ftype == TType.I32:
-          self.num_err_choice = iprot.readI32();
+          self.num_err_choice = iprot.readI32()
         else:
           iprot.skip(ftype)
       else:
@@ -6348,6 +9068,275 @@
   def __ne__(self, other):
     return not (self == other)
 
+class LogLevel:
+  """
+  Attributes:
+   - action
+   - target_log_level
+   - reset_log_level_timeout_secs
+   - reset_log_level_timeout_epoch
+   - reset_log_level
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'action', None, None, ), # 1
+    (2, TType.STRING, 'target_log_level', None, None, ), # 2
+    (3, TType.I32, 'reset_log_level_timeout_secs', None, None, ), # 3
+    (4, TType.I64, 'reset_log_level_timeout_epoch', None, None, ), # 4
+    (5, TType.STRING, 'reset_log_level', None, None, ), # 5
+  )
+
+  def __init__(self, action=None, target_log_level=None, reset_log_level_timeout_secs=None, reset_log_level_timeout_epoch=None, reset_log_level=None,):
+    self.action = action
+    self.target_log_level = target_log_level
+    self.reset_log_level_timeout_secs = reset_log_level_timeout_secs
+    self.reset_log_level_timeout_epoch = reset_log_level_timeout_epoch
+    self.reset_log_level = reset_log_level
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I32:
+          self.action = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.target_log_level = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.reset_log_level_timeout_secs = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I64:
+          self.reset_log_level_timeout_epoch = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.reset_log_level = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('LogLevel')
+    if self.action is not None:
+      oprot.writeFieldBegin('action', TType.I32, 1)
+      oprot.writeI32(self.action)
+      oprot.writeFieldEnd()
+    if self.target_log_level is not None:
+      oprot.writeFieldBegin('target_log_level', TType.STRING, 2)
+      oprot.writeString(self.target_log_level.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.reset_log_level_timeout_secs is not None:
+      oprot.writeFieldBegin('reset_log_level_timeout_secs', TType.I32, 3)
+      oprot.writeI32(self.reset_log_level_timeout_secs)
+      oprot.writeFieldEnd()
+    if self.reset_log_level_timeout_epoch is not None:
+      oprot.writeFieldBegin('reset_log_level_timeout_epoch', TType.I64, 4)
+      oprot.writeI64(self.reset_log_level_timeout_epoch)
+      oprot.writeFieldEnd()
+    if self.reset_log_level is not None:
+      oprot.writeFieldBegin('reset_log_level', TType.STRING, 5)
+      oprot.writeString(self.reset_log_level.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.action is None:
+      raise TProtocol.TProtocolException(message='Required field action is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.action)
+    value = (value * 31) ^ hash(self.target_log_level)
+    value = (value * 31) ^ hash(self.reset_log_level_timeout_secs)
+    value = (value * 31) ^ hash(self.reset_log_level_timeout_epoch)
+    value = (value * 31) ^ hash(self.reset_log_level)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class LogConfig:
+  """
+  Attributes:
+   - named_logger_level
+  """
+
+  thrift_spec = (
+    None, # 0
+    None, # 1
+    (2, TType.MAP, 'named_logger_level', (TType.STRING,None,TType.STRUCT,(LogLevel, LogLevel.thrift_spec)), None, ), # 2
+  )
+
+  def __init__(self, named_logger_level=None,):
+    self.named_logger_level = named_logger_level
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 2:
+        if ftype == TType.MAP:
+          self.named_logger_level = {}
+          (_ktype616, _vtype617, _size615 ) = iprot.readMapBegin()
+          for _i619 in xrange(_size615):
+            _key620 = iprot.readString().decode('utf-8')
+            _val621 = LogLevel()
+            _val621.read(iprot)
+            self.named_logger_level[_key620] = _val621
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('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 kiter622,viter623 in self.named_logger_level.items():
+        oprot.writeString(kiter622.encode('utf-8'))
+        viter623.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.named_logger_level)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class TopologyHistoryInfo:
+  """
+  Attributes:
+   - topo_ids
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'topo_ids', (TType.STRING,None), None, ), # 1
+  )
+
+  def __init__(self, topo_ids=None,):
+    self.topo_ids = topo_ids
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.topo_ids = []
+          (_etype627, _size624) = iprot.readListBegin()
+          for _i628 in xrange(_size624):
+            _elem629 = iprot.readString().decode('utf-8')
+            self.topo_ids.append(_elem629)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TopologyHistoryInfo')
+    if self.topo_ids is not None:
+      oprot.writeFieldBegin('topo_ids', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRING, len(self.topo_ids))
+      for iter630 in self.topo_ids:
+        oprot.writeString(iter630.encode('utf-8'))
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.topo_ids)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class DRPCRequest:
   """
   Attributes:
@@ -6499,3 +9488,596 @@
 
   def __ne__(self, other):
     return not (self == other)
+
+class HBMessageData:
+  """
+  Attributes:
+   - path
+   - pulse
+   - boolval
+   - records
+   - nodes
+   - message_blob
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'path', None, None, ), # 1
+    (2, TType.STRUCT, 'pulse', (HBPulse, HBPulse.thrift_spec), None, ), # 2
+    (3, TType.BOOL, 'boolval', None, None, ), # 3
+    (4, TType.STRUCT, 'records', (HBRecords, HBRecords.thrift_spec), None, ), # 4
+    (5, TType.STRUCT, 'nodes', (HBNodes, HBNodes.thrift_spec), None, ), # 5
+    None, # 6
+    (7, TType.STRING, 'message_blob', None, None, ), # 7
+  )
+
+  def __init__(self, path=None, pulse=None, boolval=None, records=None, nodes=None, message_blob=None,):
+    self.path = path
+    self.pulse = pulse
+    self.boolval = boolval
+    self.records = records
+    self.nodes = nodes
+    self.message_blob = message_blob
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.path = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.pulse = HBPulse()
+          self.pulse.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.BOOL:
+          self.boolval = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRUCT:
+          self.records = HBRecords()
+          self.records.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRUCT:
+          self.nodes = HBNodes()
+          self.nodes.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRING:
+          self.message_blob = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('HBMessageData')
+    if self.path is not None:
+      oprot.writeFieldBegin('path', TType.STRING, 1)
+      oprot.writeString(self.path.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.pulse is not None:
+      oprot.writeFieldBegin('pulse', TType.STRUCT, 2)
+      self.pulse.write(oprot)
+      oprot.writeFieldEnd()
+    if self.boolval is not None:
+      oprot.writeFieldBegin('boolval', TType.BOOL, 3)
+      oprot.writeBool(self.boolval)
+      oprot.writeFieldEnd()
+    if self.records is not None:
+      oprot.writeFieldBegin('records', TType.STRUCT, 4)
+      self.records.write(oprot)
+      oprot.writeFieldEnd()
+    if self.nodes is not None:
+      oprot.writeFieldBegin('nodes', TType.STRUCT, 5)
+      self.nodes.write(oprot)
+      oprot.writeFieldEnd()
+    if self.message_blob is not None:
+      oprot.writeFieldBegin('message_blob', TType.STRING, 7)
+      oprot.writeString(self.message_blob)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.path)
+    value = (value * 31) ^ hash(self.pulse)
+    value = (value * 31) ^ hash(self.boolval)
+    value = (value * 31) ^ hash(self.records)
+    value = (value * 31) ^ hash(self.nodes)
+    value = (value * 31) ^ hash(self.message_blob)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class HBMessage:
+  """
+  Attributes:
+   - type
+   - data
+   - message_id
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'type', None, None, ), # 1
+    (2, TType.STRUCT, 'data', (HBMessageData, HBMessageData.thrift_spec), None, ), # 2
+    (3, TType.I32, 'message_id', None, -1, ), # 3
+  )
+
+  def __init__(self, type=None, data=None, message_id=thrift_spec[3][4],):
+    self.type = type
+    self.data = data
+    self.message_id = message_id
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I32:
+          self.type = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.data = HBMessageData()
+          self.data.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.message_id = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('HBMessage')
+    if self.type is not None:
+      oprot.writeFieldBegin('type', TType.I32, 1)
+      oprot.writeI32(self.type)
+      oprot.writeFieldEnd()
+    if self.data is not None:
+      oprot.writeFieldBegin('data', TType.STRUCT, 2)
+      self.data.write(oprot)
+      oprot.writeFieldEnd()
+    if self.message_id is not None:
+      oprot.writeFieldBegin('message_id', TType.I32, 3)
+      oprot.writeI32(self.message_id)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.type)
+    value = (value * 31) ^ hash(self.data)
+    value = (value * 31) ^ hash(self.message_id)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class HBAuthorizationException(TException):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('HBAuthorizationException')
+    if self.msg is not None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.msg is None:
+      raise TProtocol.TProtocolException(message='Required field msg is unset!')
+    return
+
+
+  def __str__(self):
+    return repr(self)
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.msg)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class HBExecutionException(TException):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('HBExecutionException')
+    if self.msg is not None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'))
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.msg is None:
+      raise TProtocol.TProtocolException(message='Required field msg is unset!')
+    return
+
+
+  def __str__(self):
+    return repr(self)
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.msg)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class HBPulse:
+  """
+  Attributes:
+   - id
+   - details
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRING, 'details', None, None, ), # 2
+  )
+
+  def __init__(self, id=None, details=None,):
+    self.id = id
+    self.details = details
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8')
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.details = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('HBPulse')
+    if self.id is not None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'))
+      oprot.writeFieldEnd()
+    if self.details is not None:
+      oprot.writeFieldBegin('details', TType.STRING, 2)
+      oprot.writeString(self.details)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.id is None:
+      raise TProtocol.TProtocolException(message='Required field id is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.id)
+    value = (value * 31) ^ hash(self.details)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class HBRecords:
+  """
+  Attributes:
+   - pulses
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'pulses', (TType.STRUCT,(HBPulse, HBPulse.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, pulses=None,):
+    self.pulses = pulses
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.pulses = []
+          (_etype627, _size624) = iprot.readListBegin()
+          for _i628 in xrange(_size624):
+            _elem629 = HBPulse()
+            _elem629.read(iprot)
+            self.pulses.append(_elem629)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('HBRecords')
+    if self.pulses is not None:
+      oprot.writeFieldBegin('pulses', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.pulses))
+      for iter630 in self.pulses:
+        iter630.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.pulses)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class HBNodes:
+  """
+  Attributes:
+   - pulseIds
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'pulseIds', (TType.STRING,None), None, ), # 1
+  )
+
+  def __init__(self, pulseIds=None,):
+    self.pulseIds = pulseIds
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.pulseIds = []
+          (_etype634, _size631) = iprot.readListBegin()
+          for _i635 in xrange(_size631):
+            _elem636 = iprot.readString().decode('utf-8')
+            self.pulseIds.append(_elem636)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('HBNodes')
+    if self.pulseIds is not None:
+      oprot.writeFieldBegin('pulseIds', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRING, len(self.pulseIds))
+      for iter637 in self.pulseIds:
+        oprot.writeString(iter637.encode('utf-8'))
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.pulseIds)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
diff --git a/storm-core/src/storm.thrift b/storm-core/src/storm.thrift
index f687681..677de2b 100644
--- a/storm-core/src/storm.thrift
+++ b/storm-core/src/storm.thrift
@@ -117,6 +117,7 @@
   1: required map<string, SpoutSpec> spouts;
   2: required map<string, Bolt> bolts;
   3: required map<string, StateSpoutSpec> state_spouts;
+  4: optional list<binary> worker_hooks;
 }
 
 exception AlreadyAliveException {
@@ -146,6 +147,12 @@
 513: optional string sched_status;
 514: optional string owner;
 515: optional i32 replication_count;
+521: optional double requested_memonheap;
+522: optional double requested_memoffheap;
+523: optional double requested_cpu;
+524: optional double assigned_memonheap;
+525: optional double assigned_memoffheap;
+526: optional double assigned_cpu;
 }
 
 struct SupervisorSummary {
@@ -155,6 +162,9 @@
   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;
+  8: optional double used_mem;
+  9: optional double used_cpu;
 }
 
 struct NimbusSummary {
@@ -167,6 +177,8 @@
 
 struct ClusterSummary {
   1: required list<SupervisorSummary> supervisors;
+  //@deprecated, please use nimbuses.uptime_secs instead.
+  2: optional i32 nimbus_uptime_secs = 0;
   3: required list<TopologySummary> topologies;
   4: required list<NimbusSummary> nimbuses;
 }
@@ -220,6 +232,11 @@
   7: optional ExecutorStats stats;
 }
 
+struct DebugOptions {
+  1: optional bool enable
+  2: optional double samplingpct
+}
+
 struct TopologyInfo {
   1: required string id;
   2: required string name;
@@ -231,11 +248,104 @@
 513: optional string sched_status;
 514: optional string owner;
 515: optional i32 replication_count;
+521: optional double requested_memonheap;
+522: optional double requested_memoffheap;
+523: optional double requested_cpu;
+524: optional double assigned_memonheap;
+525: optional double assigned_memoffheap;
+526: optional double assigned_cpu;
 }
 
-struct DebugOptions {
-  1: optional bool enable
-  2: optional double samplingpct
+struct CommonAggregateStats {
+1: optional i32 num_executors;
+2: optional i32 num_tasks;
+3: optional i64 emitted;
+4: optional i64 transferred;
+5: optional i64 acked;
+6: optional i64 failed;
+}
+
+struct SpoutAggregateStats {
+1: optional double complete_latency_ms;
+}
+
+struct BoltAggregateStats {
+1: optional double execute_latency_ms;
+2: optional double process_latency_ms;
+3: optional i64    executed;
+4: optional double capacity;
+}
+
+union SpecificAggregateStats {
+1: BoltAggregateStats  bolt;
+2: SpoutAggregateStats spout;
+}
+
+enum ComponentType {
+  BOLT = 1,
+  SPOUT = 2
+}
+
+struct ComponentAggregateStats {
+1: optional ComponentType type;
+2: optional CommonAggregateStats common_stats;
+3: optional SpecificAggregateStats specific_stats;
+4: optional ErrorInfo last_error;
+}
+
+struct TopologyStats {
+1: optional map<string, i64> window_to_emitted;
+2: optional map<string, i64> window_to_transferred;
+3: optional map<string, double> window_to_complete_latencies_ms;
+4: optional map<string, i64> window_to_acked;
+5: optional map<string, i64> window_to_failed;
+}
+
+struct TopologyPageInfo {
+ 1: required string id;
+ 2: optional string name;
+ 3: optional i32 uptime_secs;
+ 4: optional string status;
+ 5: optional i32 num_tasks;
+ 6: optional i32 num_workers;
+ 7: optional i32 num_executors;
+ 8: optional string topology_conf;
+ 9: optional map<string,ComponentAggregateStats> id_to_spout_agg_stats;
+10: optional map<string,ComponentAggregateStats> id_to_bolt_agg_stats;
+11: optional string sched_status;
+12: optional TopologyStats topology_stats;
+13: optional string owner;
+14: optional DebugOptions debug_options;
+15: optional i32 replication_count;
+521: optional double requested_memonheap;
+522: optional double requested_memoffheap;
+523: optional double requested_cpu;
+524: optional double assigned_memonheap;
+525: optional double assigned_memoffheap;
+526: optional double assigned_cpu;
+}
+
+struct ExecutorAggregateStats {
+1: optional ExecutorSummary exec_summary;
+2: optional ComponentAggregateStats stats;
+}
+
+struct ComponentPageInfo {
+ 1: required string component_id;
+ 2: required ComponentType component_type;
+ 3: optional string topology_id;
+ 4: optional string topology_name;
+ 5: optional i32 num_executors;
+ 6: optional i32 num_tasks;
+ 7: optional map<string,ComponentAggregateStats> window_to_stats;
+ 8: optional map<GlobalStreamId,ComponentAggregateStats> gsid_to_input_stats;
+ 9: optional map<string,ComponentAggregateStats> sid_to_output_stats;
+10: optional list<ExecutorAggregateStats> exec_stats;
+11: optional list<ErrorInfo> errors;
+12: optional string eventlog_host;
+13: optional i32 eventlog_port;
+14: optional DebugOptions debug_options;
+15: optional string topology_status;
 }
 
 struct KillOptions {
@@ -270,6 +380,7 @@
     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 {
@@ -277,11 +388,17 @@
     2: required set<i64> port;
 }
 
+struct WorkerResources {
+    1: optional double mem_on_heap;
+    2: optional double mem_off_heap;
+    3: optional double cpu;
+}
 struct Assignment {
     1: required string master_code_dir;
     2: optional map<string, string> node_host = {};
     3: optional map<list<i64>, NodeInfo> executor_node_port = {};
     4: optional map<list<i64>, i64> executor_start_time_secs = {};
+    5: optional map<NodeInfo, WorkerResources> worker_resources = {};
 }
 
 enum TopologyStatus {
@@ -327,6 +444,7 @@
 struct LocalAssignment {
   1: required string topology_id;
   2: required list<ExecutorInfo> executors;
+  3: optional WorkerResources resources;
 }
 
 struct LSSupervisorId {
@@ -348,16 +466,78 @@
    4: required i32 port;
 }
 
+struct LSTopoHistory {
+   1: required string topology_id;
+   2: required i64 time_stamp;
+   3: required list<string> users;
+   4: required list<string> groups;
+}
+
+struct LSTopoHistoryList {
+  1: required list<LSTopoHistory> topo_history;
+}
+
 enum NumErrorsChoice {
   ALL,
   NONE,
   ONE
 }
 
+enum ProfileAction {
+  JPROFILE_STOP,
+  JPROFILE_START,
+  JPROFILE_DUMP,
+  JMAP_DUMP,
+  JSTACK_DUMP,
+  JVM_RESTART
+}
+
+struct ProfileRequest {
+  1: required NodeInfo nodeInfo,
+  2: required ProfileAction action,
+  3: optional i64 time_stamp; 
+}
+
 struct GetInfoOptions {
   1: optional NumErrorsChoice num_err_choice;
 }
 
+enum LogLevelAction {
+  UNCHANGED = 1,
+  UPDATE    = 2,
+  REMOVE    = 3
+}
+
+struct LogLevel {
+  1: required LogLevelAction action;
+
+  // during this thrift call, we'll move logger to target_log_level
+  2: optional string target_log_level;
+
+  // number of seconds that target_log_level should be kept
+  // after this timeout, the loggers will be reset to reset_log_level
+  // if timeout is 0, we will not reset 
+  3: optional i32 reset_log_level_timeout_secs;
+
+  // number of seconds since unix epoch corresponding to 
+  // current time (when message gets to nimbus) + reset_log_level_timeout_se
+  // NOTE: this field gets set in Nimbus 
+  4: optional i64 reset_log_level_timeout_epoch;
+
+  // if reset timeout was set, then we would reset 
+  // to this level after timeout (or INFO by default)
+  5: optional string reset_log_level;
+}
+
+struct LogConfig { 
+  // logger name -> log level map
+  2: optional map<string, LogLevel> named_logger_level;
+}
+
+struct TopologyHistoryInfo {
+  1: list<string> topo_ids;
+}
+
 service Nimbus {
   void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
   void submitTopologyWithOpts(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology, 5: SubmitOptions options) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
@@ -366,6 +546,11 @@
   void activate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void deactivate(1: string name) throws (1: NotAliveException e, 2: AuthorizationException aze);
   void rebalance(1: string name, 2: RebalanceOptions options) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
+
+  // dynamic log levels
+  void setLogConfig(1: string name, 2: LogConfig config);
+  LogConfig getLogConfig(1: string name);
+
   /**
   * Enable/disable logging the tuples generated in topology via an internal EventLogger bolt. The component name is optional
   * and if null or empty, the debug flag will apply to the entire topology.
@@ -373,6 +558,11 @@
   * The 'samplingPercentage' will limit loggging to a percentage of generated tuples.
   **/
   void debug(1: string name, 2: string component, 3: bool enable, 4: double samplingPercentage) throws (1: NotAliveException e, 2: AuthorizationException aze);
+
+  // dynamic profile actions
+  void setWorkerProfiler(1: string id, 2: ProfileRequest  profileRequest);
+  list<ProfileRequest> getComponentPendingProfileActions(1: string id, 2: string component_id, 3: ProfileAction action);
+
   void uploadNewCredentials(1: string name, 2: Credentials creds) throws (1: NotAliveException e, 2: InvalidTopologyException ite, 3: AuthorizationException aze);
 
   // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs
@@ -391,6 +581,8 @@
   ClusterSummary getClusterInfo() throws (1: AuthorizationException aze);
   TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
   TopologyInfo getTopologyInfoWithOpts(1: string id, 2: GetInfoOptions options) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  TopologyPageInfo getTopologyPageInfo(1: string id, 2: string window, 3: bool is_include_sys) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  ComponentPageInfo getComponentPageInfo(1: string topology_id, 2: string component_id, 3: string window, 4: bool is_include_sys) throws (1: NotAliveException e, 2: AuthorizationException aze);
   //returns json
   string getTopologyConf(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
   /**
@@ -401,6 +593,7 @@
    * Returns the user specified topology as submitted originally. Compare {@link #getTopology(String id)}.
    */
   StormTopology getUserTopology(1: string id) throws (1: NotAliveException e, 2: AuthorizationException aze);
+  TopologyHistoryInfo getTopologyHistory(1: string user) throws (1: AuthorizationException aze);
 }
 
 struct DRPCRequest {
@@ -421,3 +614,62 @@
   DRPCRequest fetchRequest(1: string functionName) throws (1: AuthorizationException aze);
   void failRequest(1: string id) throws (1: AuthorizationException aze);  
 }
+
+enum HBServerMessageType {
+  CREATE_PATH,
+  CREATE_PATH_RESPONSE,
+  EXISTS,
+  EXISTS_RESPONSE,
+  SEND_PULSE,
+  SEND_PULSE_RESPONSE,
+  GET_ALL_PULSE_FOR_PATH,
+  GET_ALL_PULSE_FOR_PATH_RESPONSE,
+  GET_ALL_NODES_FOR_PATH,
+  GET_ALL_NODES_FOR_PATH_RESPONSE,
+  GET_PULSE,
+  GET_PULSE_RESPONSE,
+  DELETE_PATH,
+  DELETE_PATH_RESPONSE,
+  DELETE_PULSE_ID,
+  DELETE_PULSE_ID_RESPONSE,
+  CONTROL_MESSAGE,
+  SASL_MESSAGE_TOKEN,
+  NOT_AUTHORIZED
+}
+
+union HBMessageData {
+  1: string path,
+  2: HBPulse pulse,
+  3: bool boolval,
+  4: HBRecords records,
+  5: HBNodes nodes,
+  7: optional binary message_blob;
+}
+
+struct HBMessage {
+  1: HBServerMessageType type,
+  2: HBMessageData data,
+  3: optional i32 message_id = -1,
+}
+
+
+exception HBAuthorizationException {
+  1: required string msg;
+}
+
+exception HBExecutionException {
+  1: required string msg;
+}
+
+struct HBPulse {
+  1: required string id;
+  2: binary details;
+}
+
+struct HBRecords {
+  1: list<HBPulse> pulses;
+}
+
+struct HBNodes {
+  1: list<string> pulseIds;
+}
\ No newline at end of file
diff --git a/storm-core/src/ui/public/component.html b/storm-core/src/ui/public/component.html
index 4be5860..ae2463c 100644
--- a/storm-core/src/ui/public/component.html
+++ b/storm-core/src/ui/public/component.html
@@ -58,6 +58,9 @@
     <div id="component-output-stats" class="col-md-12"></div>
   </div>
   <div class="row">
+    <div id="profiler-control" class="col-md-12"></div>
+  </div>
+  <div class="row">
     <div id="component-executor-stats" class="col-md-12"></div>
   </div>
   <div class="row">
@@ -76,6 +79,15 @@
 $(document).ajaxStart(function(){
     $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading component summary...</h3>'});
 });
+function jsError(other) {
+    try {
+      other();
+    } catch (err) {
+      $.get("/templates/json-error-template.html", function(template) {
+        $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),{error: "JS Error", errorMessage: err}));
+      });
+    }
+}
 $(document).ready(function() {
     var componentId = $.url("?id");
     var topologyId = $.url("?topology_id");
@@ -121,11 +133,47 @@
         var componentStatsDetail = $("#component-stats-detail")
         var inputStats = $("#component-input-stats");
         var outputStats = $("#component-output-stats");
+        var profilerControl = $("#profiler-control");
         var executorStats = $("#component-executor-stats");
         var componentErrors = $("#component-errors");
         $.get("/templates/component-page-template.html", function(template) {
-            componentSummary.append(Mustache.render($(template).filter("#component-summary-template").html(),response));
-            componentActions.append(Mustache.render($(template).filter("#component-actions-template").html(),buttonJsonData));
+            response["hosts"] = [];
+            for(var comp_index in response["executorStats"]) {
+                var comp = response["executorStats"][comp_index];
+                var host_port = comp["host"] + ":" + comp["port"];
+                if($.inArray(host_port, response["hosts"]) == -1) {
+                    response["hosts"].push(host_port);
+                }
+            }
+
+            response["hosts"] = $.map(response["hosts"], function(host_port) {
+                return {"name": host_port};
+            });
+
+            response["profilerActive"] = $.map(response["profilerActive"], function(active_map) {
+                var date = new Date();
+                var millis = date.getTime() + parseInt(active_map["timestamp"]);
+                date = new Date(millis);
+                active_map["timestamp"] = date.toTimeString();
+                return active_map;
+            });
+
+            jsError(function() {
+              componentSummary.append(Mustache.render($(template).filter("#component-summary-template").html(),response));
+            });
+
+            jsError(function() {
+              componentActions.append(Mustache.render($(template).filter("#component-actions-template").html(),buttonJsonData));
+            });
+
+            if(response["profileActionEnabled"] == true) {
+                jsError(function () {
+                    var part = $(template).filter('#profiler-active-partial').html();
+                    var partials = {"profilerActive": part};
+                    profilerControl.append(Mustache.render($(template).filter("#profiling-template").html(), response, partials));
+                });
+            }
+
             if(response["componentType"] == "spout") {
                 componentStatsDetail.append(Mustache.render($(template).filter("#spout-stats-detail-template").html(),response));
                 //window, emitted, transferred, complete latency, acked, failed
@@ -213,6 +261,121 @@
         });
     });
 });
+
+function profiler_selected_worker() {
+    return $("#selected_worker").val();
+}
+
+function start_profiling(id) {
+    var topologyId = $.url("?topology_id");
+    var timeout = $("#timeout").val();
+
+    if(timeout == "") { timeout = 10; }
+    if(isNaN(parseFloat(timeout)) || !isFinite(timeout)) {
+        alert("Must specify a numeric timeout");
+        return;
+    }
+
+    var url = "/api/v1/topology/"+topologyId+"/profiling/start/" + id + "/" + timeout;
+    $.get(url, function(response,status,jqXHR) {
+        jsError(function() {
+            $.get("/templates/component-page-template.html", function(template) {
+                var host_port_split = id.split(":");
+                var host = host_port_split[0];
+                var port = host_port_split[1];
+                var millis = new Date().getTime() + (timeout * 60000);
+                var timestamp = new Date(millis).toTimeString();
+                
+                var mustache = Mustache.render($(template).filter("#profiler-active-partial").html(), {"profilerActive": [{
+                    "host": host,
+                    "port": port,
+                    "timestamp": timestamp,
+                    "dumplink": response["dumplink"]}]});
+                $("#profiler-table-body").append(mustache);
+            });
+        });
+    })
+    .fail(function(response) {
+        alert( "Starting profiler for " + id + " failed: \n" + JSON.stringify(response));
+    });
+}
+
+function stop_profiling(id) {
+    var topologyId = $.url("?topology_id");
+    var url = "/api/v1/topology/"+topologyId+"/profiling/stop/" + id;
+
+    $("#stop_" + id).prop('disabled', true);
+    setTimeout(function(){ $("#stop_" + id).prop('disabled', false); }, 5000);
+    
+    $.get(url, function(response,status,jqXHR) {
+        alert("Submitted request to stop profiling...");
+    })
+    .fail(function(response) {
+        alert( "Stopping profiler for " + id + " failed: \n" + JSON.stringify(response));
+    });
+    
+}
+
+function dump_profile(id) {
+    var topologyId = $.url("?topology_id");
+    var url = "/api/v1/topology/"+topologyId+"/profiling/dumpprofile/" + id;
+
+    $("#dump_profile_" + id).prop('disabled', true);
+    setTimeout(function(){ $("#dump_profile_" + id).prop('disabled', false); }, 5000);
+    
+    $.get(url, function(response,status,jqXHR) {
+        alert("Submitted request to dump profile snapshot...");
+    })
+    .fail(function(response) {
+        alert( "Dumping profile data for " + id + " failed: \n" + JSON.stringify(response));
+    });
+}
+
+function dump_jstack(id) {
+    var topologyId = $.url("?topology_id");
+    var url = "/api/v1/topology/"+topologyId+"/profiling/dumpjstack/" + id;
+
+    $("#dump_jstack_" + id).prop('disabled', true);
+    setTimeout(function(){ $("#dump_jstack_" + id).prop('disabled', false); }, 5000);
+    
+    $.get(url, function(response,status,jqXHR) {
+        alert("Submitted request for jstack dump...");
+    })
+    .fail(function(response) {
+        alert( "Dumping JStack for " + id + " failed: \n" + JSON.stringify(response));
+    });
+}
+
+function restart_worker_jvm(id) {
+    var topologyId = $.url("?topology_id");
+    var url = "/api/v1/topology/"+topologyId+"/profiling/restartworker/" + id;
+
+    $("#restart_worker_jvm_" + id).prop('disabled', true);
+    setTimeout(function(){ $("#restart_worker_jvm_" + id).prop('disabled', false); }, 5000);
+
+    $.get(url, function(response,status,jqXHR) {
+        alert("Submitted request for restarting worker...");
+    })
+    .fail(function(response) {
+        alert( "Failed to restart worker for " + id + " failed: \n" + JSON.stringify(response));
+    });
+}
+
+function dump_heap(id) {
+    var topologyId = $.url("?topology_id");
+    var url = "/api/v1/topology/"+topologyId+"/profiling/dumpheap/" + id;
+    var heap = $("#dump_heap_" + id);
+    $("#dump_heap_" + id).prop('disabled', true);
+    setTimeout(function(){ $("#dump_heap_" + id).prop('disabled', false); }, 5000);
+    
+    $.get(url, function(response,status,jqXHR) {
+        alert("Submitted request for jmap dump...");
+    })
+    .fail(function(response) {
+        alert( "Dumping Heap for " + id + " failed: \n" + JSON.stringify(response));
+    });
+}
+
 </script>
 </div>
 </body>
diff --git a/storm-core/src/ui/public/css/style.css b/storm-core/src/ui/public/css/style.css
index e058850..ddae2d5 100644
--- a/storm-core/src/ui/public/css/style.css
+++ b/storm-core/src/ui/public/css/style.css
@@ -24,6 +24,14 @@
     padding: 0.5em;
 }
 
+#change-log-level select{
+    width:10em;
+}
+
+#change-log-level input.timeout_input{
+    width:5em;
+}
+
 body,
 .dataTables_wrapper label,
 div.dataTables_info[role="status"],
@@ -96,3 +104,11 @@
   padding: 0px;
   background-color: transparent;
 }
+
+.image_pre_format {
+  border: none;
+  padding: 0px;
+  height: 1em;
+  width: 1em;
+}
+
diff --git a/storm-core/src/ui/public/deep_search_result.html b/storm-core/src/ui/public/deep_search_result.html
new file mode 100644
index 0000000..406a101
--- /dev/null
+++ b/storm-core/src/ui/public/deep_search_result.html
@@ -0,0 +1,155 @@
+<html><head>
+    <!--
+     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.
+    -->
+<meta charset="UTF-8">
+<meta name="viewport" content="width=device-width, initial-scale=1">
+<title>Storm UI</title>
+<link href="/css/bootstrap-3.3.1.min.css" rel="stylesheet" type="text/css">
+<link href="/css/jquery.dataTables.1.10.4.min.css" rel="stylesheet" type="text/css">
+<link href="/css/dataTables.bootstrap.css" rel="stylesheet" type="text/css">
+<link href="/css/style.css" rel="stylesheet" type="text/css">
+<script src="/js/jquery-1.11.1.min.js" type="text/javascript"></script>
+<script src="/js/jquery.dataTables.1.10.4.min.js" type="text/javascript"></script>
+<script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
+<script src="/js/jquery.mustache.js" type="text/javascript"></script>
+<script src="/js/typeahead.jquery.min.js" type="text/javascript"></script>
+<script src="/js/bootstrap-3.3.1.min.js" type="text/javascript"></script>
+<script src="/js/jquery.blockUI.min.js" type="text/javascript"></script>
+<script src="/js/url.min.js" type="text/javascript"></script>
+<script src="/js/dataTables.bootstrap.min.js" type="text/javascript"></script>
+<script src="/js/script.js" type="text/javascript"></script>
+</head>
+<body>
+<div class="container-fluid">
+  <div class="row">
+    <div id="search-form"></div>
+  </div>
+  <div class="row">
+    <div id="result"></div>
+  </div>
+</div>
+</body>
+<script type="text/javascript">
+
+function elem_id_for_host(host) {
+    return "search-results-"+host.replace(/\./g,"-");
+}
+
+$(document).ready(function() {
+    var id = $.url("?id");
+    var search = $.url("?search");
+    var count = $.url("?count") || 2;
+    var port = $.url("?port") || "*";
+    var search_archived = $.url("?search-archived");
+
+    $.get("/templates/deep-search-result-page-template.html", function(template) {
+        var checked;
+        if (search_archived) {
+            checked = "checked";
+        }
+        $.get("/api/v1/history/summary", function (response, status, jqXHR) {
+            var findIds = function findIds(query, cb) {
+                var found = [];
+                var re = new RegExp(query, 'i');
+                for (var j in response["topo-history"]) {
+                    var id = response["topo-history"][j];
+                    if (re.test(id)) {
+                        found.push({ value: id });
+                    }
+               }
+               cb(found);
+            };
+
+            $("#search-form").append(Mustache.render($(template).filter("#search-form-template").html(),{id: id, search: search, count: count, search_archived: checked, topologies: response["topo-history"]}));
+            $('#search-id').typeahead({
+              hint: true,
+              highlight: true,
+              minLength: 1
+            },
+            {
+              name: 'ids',
+              displayKey: 'value',
+              source: findIds 
+            });
+            $('#search-form [data-toggle="tooltip"]').tooltip();
+        });
+
+        function render_file(file, host, logviewerPort, count, template) {
+            file.host = host;
+            file.id = id;
+            file.count = count;
+            file.logviewerPort = logviewerPort;
+            file.search_archived = search_archived;
+            var searchTemplate = $(template).filter("#search-result-identified-template").html();
+            var rendered = Mustache.render(searchTemplate, file);
+            var elemId = elem_id_for_host(host);
+            $("#"+elemId).remove();
+            $("#search-result-table > tbody").append(rendered);
+        }
+
+
+        var result = $("#result");
+        if(search) {
+            $.get("/api/v1/supervisor/summary", function(response, status, jqXHR) {
+
+                for(var i in response.supervisors) {
+                    response.supervisors[i].elemId = elem_id_for_host(response.supervisors[i].host);
+                }
+
+                result.append(Mustache.render($(template).filter("#search-result-files-template").html(),response));
+
+                var logviewerPort = response.logviewerPort;
+                var distinct_hosts = {};
+                for (var index in response.supervisors) {
+                    distinct_hosts[response.supervisors[index].host] = true;
+                }
+
+
+                for (var host in distinct_hosts) {
+                    var searchURL = "http://"+host+":"+logviewerPort+"/deepSearch/"+id+"?search-string="+search+"&num-matches="+count+"&port="+port;
+                    if(search_archived)
+                        searchURL += "&search-archived=" + search_archived;
+                    
+                    
+                    $.ajax({dataType: "json",
+                            url: searchURL,
+                            xhrFields: {
+                                withCredentials: true
+                            },
+                            success: $.proxy(function(data,status,jqXHR) {
+                                if(port == "*") {
+                                    for(var by_port in data) {
+                                        for(var i in data[by_port].matches) {
+                                            render_file(data[by_port].matches[i], this.host, logviewerPort, count, template);
+                                        }
+                                    }
+                                }
+                                else {
+                                    for(var i in data.matches) {
+                                        render_file(data.matches[i], this.host, logviewerPort, count, template);
+                                    }
+                                }
+                            }, {host: host, id: id})
+                           });
+                }
+            });
+        }
+    });
+});
+
+</script>
+</html>
diff --git a/storm-core/src/ui/public/images/bug.png b/storm-core/src/ui/public/images/bug.png
new file mode 100644
index 0000000..e98e156
--- /dev/null
+++ b/storm-core/src/ui/public/images/bug.png
Binary files differ
diff --git a/storm-core/src/ui/public/images/search.png b/storm-core/src/ui/public/images/search.png
new file mode 100644
index 0000000..fa3a689
--- /dev/null
+++ b/storm-core/src/ui/public/images/search.png
Binary files differ
diff --git a/storm-core/src/ui/public/images/statistic.png b/storm-core/src/ui/public/images/statistic.png
new file mode 100644
index 0000000..f9ad38a
--- /dev/null
+++ b/storm-core/src/ui/public/images/statistic.png
Binary files differ
diff --git a/storm-core/src/ui/public/index.html b/storm-core/src/ui/public/index.html
index 1cd6445..eae423c 100644
--- a/storm-core/src/ui/public/index.html
+++ b/storm-core/src/ui/public/index.html
@@ -37,10 +37,10 @@
 <body>
 <div class="container-fluid">
   <div class="row">
-    <div class="col-md-11">
+    <div class="col-md-9">
       <h1><a href="/">Storm UI</a></h1>
     </div>
-    <div id="ui-user" class="col-md-1"></div>
+    <div id="ui-user" class="col-md-3"></div>
   </div>
   <div class="row">
     <div class="col-md-12">
@@ -135,11 +135,11 @@
     $.getJSON("/api/v1/topology/summary",function(response,status,jqXHR) {
       $.get("/templates/index-page-template.html", function(template) {
           topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));
-          //name, id, owner, status, uptime, num workers, num executors, num tasks, scheduler info
+          //name, owner, status, uptime, num workers, num executors, num tasks, replication count, assigned total mem, assigned total cpu, scheduler info
           dtAutoPage("#topology-summary-table", {
             columnDefs: [
-              {type: "num", targets: [5, 6, 7]},
-              {type: "time-str", targets: [4]}
+              {type: "num", targets: [4, 5, 6, 7, 8, 9]},
+              {type: "time-str", targets: [3]}
             ]
           });
           $('#topology-summary [data-toggle="tooltip"]').tooltip();
diff --git a/storm-core/src/ui/public/js/script.js b/storm-core/src/ui/public/js/script.js
index fe8643d..a880205 100644
--- a/storm-core/src/ui/public/js/script.js
+++ b/storm-core/src/ui/public/js/script.js
@@ -81,6 +81,26 @@
     return isInt;
 }
 
+function sendRequest(id, action, extra, body, cb){
+   var opts = {
+        type:'POST',
+        url:'/api/v1/topology/' + id + '/' + action
+    };
+
+    if (body) {
+        opts.data = JSON.stringify(body);
+        opts.contentType = 'application/json; charset=utf-8';
+    }
+
+    opts.url += extra ? "/" + extra : "";
+
+    $.ajax(opts).always(function(data){
+        cb (data);
+    }).fail (function(){
+        alert("Error while communicating with Nimbus.");
+    });
+}
+
 function confirmComponentAction(topologyId, componentId, componentName, action, param, defaultParamValue, paramText, actionText) {
     var opts = {
         type:'POST',
diff --git a/storm-core/src/ui/public/js/typeahead.jquery.min.js b/storm-core/src/ui/public/js/typeahead.jquery.min.js
new file mode 100644
index 0000000..3ee29eb
--- /dev/null
+++ b/storm-core/src/ui/public/js/typeahead.jquery.min.js
@@ -0,0 +1,7 @@
+/*!
+ * typeahead.js 0.10.5
+ * https://github.com/twitter/typeahead.js
+ * Copyright 2013-2014 Twitter, Inc. and other contributors; Licensed MIT
+ */
+
+!function(a){var b=function(){"use strict";return{isMsie:function(){return/(msie|trident)/i.test(navigator.userAgent)?navigator.userAgent.match(/(msie |rv:)(\d+(.\d+)?)/i)[2]:!1},isBlankString:function(a){return!a||/^\s*$/.test(a)},escapeRegExChars:function(a){return a.replace(/[\-\[\]\/\{\}\(\)\*\+\?\.\\\^\$\|]/g,"\\$&")},isString:function(a){return"string"==typeof a},isNumber:function(a){return"number"==typeof a},isArray:a.isArray,isFunction:a.isFunction,isObject:a.isPlainObject,isUndefined:function(a){return"undefined"==typeof a},toStr:function(a){return b.isUndefined(a)||null===a?"":a+""},bind:a.proxy,each:function(b,c){function d(a,b){return c(b,a)}a.each(b,d)},map:a.map,filter:a.grep,every:function(b,c){var d=!0;return b?(a.each(b,function(a,e){return(d=c.call(null,e,a,b))?void 0:!1}),!!d):d},some:function(b,c){var d=!1;return b?(a.each(b,function(a,e){return(d=c.call(null,e,a,b))?!1:void 0}),!!d):d},mixin:a.extend,getUniqueId:function(){var a=0;return function(){return a++}}(),templatify:function(b){function c(){return String(b)}return a.isFunction(b)?b:c},defer:function(a){setTimeout(a,0)},debounce:function(a,b,c){var d,e;return function(){var f,g,h=this,i=arguments;return f=function(){d=null,c||(e=a.apply(h,i))},g=c&&!d,clearTimeout(d),d=setTimeout(f,b),g&&(e=a.apply(h,i)),e}},throttle:function(a,b){var c,d,e,f,g,h;return g=0,h=function(){g=new Date,e=null,f=a.apply(c,d)},function(){var i=new Date,j=b-(i-g);return c=this,d=arguments,0>=j?(clearTimeout(e),e=null,g=i,f=a.apply(c,d)):e||(e=setTimeout(h,j)),f}},noop:function(){}}}(),c=function(){return{wrapper:'<span class="twitter-typeahead"></span>',dropdown:'<span class="tt-dropdown-menu"></span>',dataset:'<div class="tt-dataset-%CLASS%"></div>',suggestions:'<span class="tt-suggestions"></span>',suggestion:'<div class="tt-suggestion"></div>'}}(),d=function(){"use strict";var a={wrapper:{position:"relative",display:"inline-block"},hint:{position:"absolute",top:"0",left:"0",borderColor:"transparent",boxShadow:"none",opacity:"1"},input:{position:"relative",verticalAlign:"top",backgroundColor:"transparent"},inputWithNoHint:{position:"relative",verticalAlign:"top"},dropdown:{position:"absolute",top:"100%",left:"0",zIndex:"100",display:"none"},suggestions:{display:"block"},suggestion:{whiteSpace:"nowrap",cursor:"pointer"},suggestionChild:{whiteSpace:"normal"},ltr:{left:"0",right:"auto"},rtl:{left:"auto",right:" 0"}};return b.isMsie()&&b.mixin(a.input,{backgroundImage:"url(data:image/gif;base64,R0lGODlhAQABAIAAAAAAAP///yH5BAEAAAAALAAAAAABAAEAAAIBRAA7)"}),b.isMsie()&&b.isMsie()<=7&&b.mixin(a.input,{marginTop:"-1px"}),a}(),e=function(){"use strict";function c(b){b&&b.el||a.error("EventBus initialized without el"),this.$el=a(b.el)}var d="typeahead:";return b.mixin(c.prototype,{trigger:function(a){var b=[].slice.call(arguments,1);this.$el.trigger(d+a,b)}}),c}(),f=function(){"use strict";function a(a,b,c,d){var e;if(!c)return this;for(b=b.split(i),c=d?h(c,d):c,this._callbacks=this._callbacks||{};e=b.shift();)this._callbacks[e]=this._callbacks[e]||{sync:[],async:[]},this._callbacks[e][a].push(c);return this}function b(b,c,d){return a.call(this,"async",b,c,d)}function c(b,c,d){return a.call(this,"sync",b,c,d)}function d(a){var b;if(!this._callbacks)return this;for(a=a.split(i);b=a.shift();)delete this._callbacks[b];return this}function e(a){var b,c,d,e,g;if(!this._callbacks)return this;for(a=a.split(i),d=[].slice.call(arguments,1);(b=a.shift())&&(c=this._callbacks[b]);)e=f(c.sync,this,[b].concat(d)),g=f(c.async,this,[b].concat(d)),e()&&j(g);return this}function f(a,b,c){function d(){for(var d,e=0,f=a.length;!d&&f>e;e+=1)d=a[e].apply(b,c)===!1;return!d}return d}function g(){var a;return a=window.setImmediate?function(a){setImmediate(function(){a()})}:function(a){setTimeout(function(){a()},0)}}function h(a,b){return a.bind?a.bind(b):function(){a.apply(b,[].slice.call(arguments,0))}}var i=/\s+/,j=g();return{onSync:c,onAsync:b,off:d,trigger:e}}(),g=function(a){"use strict";function c(a,c,d){for(var e,f=[],g=0,h=a.length;h>g;g++)f.push(b.escapeRegExChars(a[g]));return e=d?"\\b("+f.join("|")+")\\b":"("+f.join("|")+")",c?new RegExp(e):new RegExp(e,"i")}var d={node:null,pattern:null,tagName:"strong",className:null,wordsOnly:!1,caseSensitive:!1};return function(e){function f(b){var c,d,f;return(c=h.exec(b.data))&&(f=a.createElement(e.tagName),e.className&&(f.className=e.className),d=b.splitText(c.index),d.splitText(c[0].length),f.appendChild(d.cloneNode(!0)),b.parentNode.replaceChild(f,d)),!!c}function g(a,b){for(var c,d=3,e=0;e<a.childNodes.length;e++)c=a.childNodes[e],c.nodeType===d?e+=b(c)?1:0:g(c,b)}var h;e=b.mixin({},d,e),e.node&&e.pattern&&(e.pattern=b.isArray(e.pattern)?e.pattern:[e.pattern],h=c(e.pattern,e.caseSensitive,e.wordsOnly),g(e.node,f))}}(window.document),h=function(){"use strict";function c(c){var e,f,g,i,j=this;c=c||{},c.input||a.error("input is missing"),e=b.bind(this._onBlur,this),f=b.bind(this._onFocus,this),g=b.bind(this._onKeydown,this),i=b.bind(this._onInput,this),this.$hint=a(c.hint),this.$input=a(c.input).on("blur.tt",e).on("focus.tt",f).on("keydown.tt",g),0===this.$hint.length&&(this.setHint=this.getHint=this.clearHint=this.clearHintIfInvalid=b.noop),b.isMsie()?this.$input.on("keydown.tt keypress.tt cut.tt paste.tt",function(a){h[a.which||a.keyCode]||b.defer(b.bind(j._onInput,j,a))}):this.$input.on("input.tt",i),this.query=this.$input.val(),this.$overflowHelper=d(this.$input)}function d(b){return a('<pre aria-hidden="true"></pre>').css({position:"absolute",visibility:"hidden",whiteSpace:"pre",fontFamily:b.css("font-family"),fontSize:b.css("font-size"),fontStyle:b.css("font-style"),fontVariant:b.css("font-variant"),fontWeight:b.css("font-weight"),wordSpacing:b.css("word-spacing"),letterSpacing:b.css("letter-spacing"),textIndent:b.css("text-indent"),textRendering:b.css("text-rendering"),textTransform:b.css("text-transform")}).insertAfter(b)}function e(a,b){return c.normalizeQuery(a)===c.normalizeQuery(b)}function g(a){return a.altKey||a.ctrlKey||a.metaKey||a.shiftKey}var h;return h={9:"tab",27:"esc",37:"left",39:"right",13:"enter",38:"up",40:"down"},c.normalizeQuery=function(a){return(a||"").replace(/^\s*/g,"").replace(/\s{2,}/g," ")},b.mixin(c.prototype,f,{_onBlur:function(){this.resetInputValue(),this.trigger("blurred")},_onFocus:function(){this.trigger("focused")},_onKeydown:function(a){var b=h[a.which||a.keyCode];this._managePreventDefault(b,a),b&&this._shouldTrigger(b,a)&&this.trigger(b+"Keyed",a)},_onInput:function(){this._checkInputValue()},_managePreventDefault:function(a,b){var c,d,e;switch(a){case"tab":d=this.getHint(),e=this.getInputValue(),c=d&&d!==e&&!g(b);break;case"up":case"down":c=!g(b);break;default:c=!1}c&&b.preventDefault()},_shouldTrigger:function(a,b){var c;switch(a){case"tab":c=!g(b);break;default:c=!0}return c},_checkInputValue:function(){var a,b,c;a=this.getInputValue(),b=e(a,this.query),c=b?this.query.length!==a.length:!1,this.query=a,b?c&&this.trigger("whitespaceChanged",this.query):this.trigger("queryChanged",this.query)},focus:function(){this.$input.focus()},blur:function(){this.$input.blur()},getQuery:function(){return this.query},setQuery:function(a){this.query=a},getInputValue:function(){return this.$input.val()},setInputValue:function(a,b){this.$input.val(a),b?this.clearHint():this._checkInputValue()},resetInputValue:function(){this.setInputValue(this.query,!0)},getHint:function(){return this.$hint.val()},setHint:function(a){this.$hint.val(a)},clearHint:function(){this.setHint("")},clearHintIfInvalid:function(){var a,b,c,d;a=this.getInputValue(),b=this.getHint(),c=a!==b&&0===b.indexOf(a),d=""!==a&&c&&!this.hasOverflow(),!d&&this.clearHint()},getLanguageDirection:function(){return(this.$input.css("direction")||"ltr").toLowerCase()},hasOverflow:function(){var a=this.$input.width()-2;return this.$overflowHelper.text(this.getInputValue()),this.$overflowHelper.width()>=a},isCursorAtEnd:function(){var a,c,d;return a=this.$input.val().length,c=this.$input[0].selectionStart,b.isNumber(c)?c===a:document.selection?(d=document.selection.createRange(),d.moveStart("character",-a),a===d.text.length):!0},destroy:function(){this.$hint.off(".tt"),this.$input.off(".tt"),this.$hint=this.$input=this.$overflowHelper=null}}),c}(),i=function(){"use strict";function e(d){d=d||{},d.templates=d.templates||{},d.source||a.error("missing source"),d.name&&!j(d.name)&&a.error("invalid dataset name: "+d.name),this.query=null,this.highlight=!!d.highlight,this.name=d.name||b.getUniqueId(),this.source=d.source,this.displayFn=h(d.display||d.displayKey),this.templates=i(d.templates,this.displayFn),this.$el=a(c.dataset.replace("%CLASS%",this.name))}function h(a){function c(b){return b[a]}return a=a||"value",b.isFunction(a)?a:c}function i(a,c){function d(a){return"<p>"+c(a)+"</p>"}return{empty:a.empty&&b.templatify(a.empty),header:a.header&&b.templatify(a.header),footer:a.footer&&b.templatify(a.footer),suggestion:a.suggestion||d}}function j(a){return/^[_a-zA-Z0-9-]+$/.test(a)}var k="ttDataset",l="ttValue",m="ttDatum";return e.extractDatasetName=function(b){return a(b).data(k)},e.extractValue=function(b){return a(b).data(l)},e.extractDatum=function(b){return a(b).data(m)},b.mixin(e.prototype,f,{_render:function(e,f){function h(){return p.templates.empty({query:e,isEmpty:!0})}function i(){function h(b){var e;return e=a(c.suggestion).append(p.templates.suggestion(b)).data(k,p.name).data(l,p.displayFn(b)).data(m,b),e.children().each(function(){a(this).css(d.suggestionChild)}),e}var i,j;return i=a(c.suggestions).css(d.suggestions),j=b.map(f,h),i.append.apply(i,j),p.highlight&&g({className:"tt-highlight",node:i[0],pattern:e}),i}function j(){return p.templates.header({query:e,isEmpty:!o})}function n(){return p.templates.footer({query:e,isEmpty:!o})}if(this.$el){var o,p=this;this.$el.empty(),o=f&&f.length,!o&&this.templates.empty?this.$el.html(h()).prepend(p.templates.header?j():null).append(p.templates.footer?n():null):o&&this.$el.html(i()).prepend(p.templates.header?j():null).append(p.templates.footer?n():null),this.trigger("rendered")}},getRoot:function(){return this.$el},update:function(a){function b(b){c.canceled||a!==c.query||c._render(a,b)}var c=this;this.query=a,this.canceled=!1,this.source(a,b)},cancel:function(){this.canceled=!0},clear:function(){this.cancel(),this.$el.empty(),this.trigger("rendered")},isEmpty:function(){return this.$el.is(":empty")},destroy:function(){this.$el=null}}),e}(),j=function(){"use strict";function c(c){var d,f,g,h=this;c=c||{},c.menu||a.error("menu is required"),this.isOpen=!1,this.isEmpty=!0,this.datasets=b.map(c.datasets,e),d=b.bind(this._onSuggestionClick,this),f=b.bind(this._onSuggestionMouseEnter,this),g=b.bind(this._onSuggestionMouseLeave,this),this.$menu=a(c.menu).on("click.tt",".tt-suggestion",d).on("mouseenter.tt",".tt-suggestion",f).on("mouseleave.tt",".tt-suggestion",g),b.each(this.datasets,function(a){h.$menu.append(a.getRoot()),a.onSync("rendered",h._onRendered,h)})}function e(a){return new i(a)}return b.mixin(c.prototype,f,{_onSuggestionClick:function(b){this.trigger("suggestionClicked",a(b.currentTarget))},_onSuggestionMouseEnter:function(b){this._removeCursor(),this._setCursor(a(b.currentTarget),!0)},_onSuggestionMouseLeave:function(){this._removeCursor()},_onRendered:function(){function a(a){return a.isEmpty()}this.isEmpty=b.every(this.datasets,a),this.isEmpty?this._hide():this.isOpen&&this._show(),this.trigger("datasetRendered")},_hide:function(){this.$menu.hide()},_show:function(){this.$menu.css("display","block")},_getSuggestions:function(){return this.$menu.find(".tt-suggestion")},_getCursor:function(){return this.$menu.find(".tt-cursor").first()},_setCursor:function(a,b){a.first().addClass("tt-cursor"),!b&&this.trigger("cursorMoved")},_removeCursor:function(){this._getCursor().removeClass("tt-cursor")},_moveCursor:function(a){var b,c,d,e;if(this.isOpen){if(c=this._getCursor(),b=this._getSuggestions(),this._removeCursor(),d=b.index(c)+a,d=(d+1)%(b.length+1)-1,-1===d)return void this.trigger("cursorRemoved");-1>d&&(d=b.length-1),this._setCursor(e=b.eq(d)),this._ensureVisible(e)}},_ensureVisible:function(a){var b,c,d,e;b=a.position().top,c=b+a.outerHeight(!0),d=this.$menu.scrollTop(),e=this.$menu.height()+parseInt(this.$menu.css("paddingTop"),10)+parseInt(this.$menu.css("paddingBottom"),10),0>b?this.$menu.scrollTop(d+b):c>e&&this.$menu.scrollTop(d+(c-e))},close:function(){this.isOpen&&(this.isOpen=!1,this._removeCursor(),this._hide(),this.trigger("closed"))},open:function(){this.isOpen||(this.isOpen=!0,!this.isEmpty&&this._show(),this.trigger("opened"))},setLanguageDirection:function(a){this.$menu.css("ltr"===a?d.ltr:d.rtl)},moveCursorUp:function(){this._moveCursor(-1)},moveCursorDown:function(){this._moveCursor(1)},getDatumForSuggestion:function(a){var b=null;return a.length&&(b={raw:i.extractDatum(a),value:i.extractValue(a),datasetName:i.extractDatasetName(a)}),b},getDatumForCursor:function(){return this.getDatumForSuggestion(this._getCursor().first())},getDatumForTopSuggestion:function(){return this.getDatumForSuggestion(this._getSuggestions().first())},update:function(a){function c(b){b.update(a)}b.each(this.datasets,c)},empty:function(){function a(a){a.clear()}b.each(this.datasets,a),this.isEmpty=!0},isVisible:function(){return this.isOpen&&!this.isEmpty},destroy:function(){function a(a){a.destroy()}this.$menu.off(".tt"),this.$menu=null,b.each(this.datasets,a)}}),c}(),k=function(){"use strict";function f(c){var d,f,i;c=c||{},c.input||a.error("missing input"),this.isActivated=!1,this.autoselect=!!c.autoselect,this.minLength=b.isNumber(c.minLength)?c.minLength:1,this.$node=g(c.input,c.withHint),d=this.$node.find(".tt-dropdown-menu"),f=this.$node.find(".tt-input"),i=this.$node.find(".tt-hint"),f.on("blur.tt",function(a){var c,e,g;c=document.activeElement,e=d.is(c),g=d.has(c).length>0,b.isMsie()&&(e||g)&&(a.preventDefault(),a.stopImmediatePropagation(),b.defer(function(){f.focus()}))}),d.on("mousedown.tt",function(a){a.preventDefault()}),this.eventBus=c.eventBus||new e({el:f}),this.dropdown=new j({menu:d,datasets:c.datasets}).onSync("suggestionClicked",this._onSuggestionClicked,this).onSync("cursorMoved",this._onCursorMoved,this).onSync("cursorRemoved",this._onCursorRemoved,this).onSync("opened",this._onOpened,this).onSync("closed",this._onClosed,this).onAsync("datasetRendered",this._onDatasetRendered,this),this.input=new h({input:f,hint:i}).onSync("focused",this._onFocused,this).onSync("blurred",this._onBlurred,this).onSync("enterKeyed",this._onEnterKeyed,this).onSync("tabKeyed",this._onTabKeyed,this).onSync("escKeyed",this._onEscKeyed,this).onSync("upKeyed",this._onUpKeyed,this).onSync("downKeyed",this._onDownKeyed,this).onSync("leftKeyed",this._onLeftKeyed,this).onSync("rightKeyed",this._onRightKeyed,this).onSync("queryChanged",this._onQueryChanged,this).onSync("whitespaceChanged",this._onWhitespaceChanged,this),this._setLanguageDirection()}function g(b,e){var f,g,h,j;f=a(b),g=a(c.wrapper).css(d.wrapper),h=a(c.dropdown).css(d.dropdown),j=f.clone().css(d.hint).css(i(f)),j.val("").removeData().addClass("tt-hint").removeAttr("id name placeholder required").prop("readonly",!0).attr({autocomplete:"off",spellcheck:"false",tabindex:-1}),f.data(l,{dir:f.attr("dir"),autocomplete:f.attr("autocomplete"),spellcheck:f.attr("spellcheck"),style:f.attr("style")}),f.addClass("tt-input").attr({autocomplete:"off",spellcheck:!1}).css(e?d.input:d.inputWithNoHint);try{!f.attr("dir")&&f.attr("dir","auto")}catch(k){}return f.wrap(g).parent().prepend(e?j:null).append(h)}function i(a){return{backgroundAttachment:a.css("background-attachment"),backgroundClip:a.css("background-clip"),backgroundColor:a.css("background-color"),backgroundImage:a.css("background-image"),backgroundOrigin:a.css("background-origin"),backgroundPosition:a.css("background-position"),backgroundRepeat:a.css("background-repeat"),backgroundSize:a.css("background-size")}}function k(a){var c=a.find(".tt-input");b.each(c.data(l),function(a,d){b.isUndefined(a)?c.removeAttr(d):c.attr(d,a)}),c.detach().removeData(l).removeClass("tt-input").insertAfter(a),a.remove()}var l="ttAttrs";return b.mixin(f.prototype,{_onSuggestionClicked:function(a,b){var c;(c=this.dropdown.getDatumForSuggestion(b))&&this._select(c)},_onCursorMoved:function(){var a=this.dropdown.getDatumForCursor();this.input.setInputValue(a.value,!0),this.eventBus.trigger("cursorchanged",a.raw,a.datasetName)},_onCursorRemoved:function(){this.input.resetInputValue(),this._updateHint()},_onDatasetRendered:function(){this._updateHint()},_onOpened:function(){this._updateHint(),this.eventBus.trigger("opened")},_onClosed:function(){this.input.clearHint(),this.eventBus.trigger("closed")},_onFocused:function(){this.isActivated=!0,this.dropdown.open()},_onBlurred:function(){this.isActivated=!1,this.dropdown.empty(),this.dropdown.close()},_onEnterKeyed:function(a,b){var c,d;c=this.dropdown.getDatumForCursor(),d=this.dropdown.getDatumForTopSuggestion(),c?(this._select(c),b.preventDefault()):this.autoselect&&d&&(this._select(d),b.preventDefault())},_onTabKeyed:function(a,b){var c;(c=this.dropdown.getDatumForCursor())?(this._select(c),b.preventDefault()):this._autocomplete(!0)},_onEscKeyed:function(){this.dropdown.close(),this.input.resetInputValue()},_onUpKeyed:function(){var a=this.input.getQuery();this.dropdown.isEmpty&&a.length>=this.minLength?this.dropdown.update(a):this.dropdown.moveCursorUp(),this.dropdown.open()},_onDownKeyed:function(){var a=this.input.getQuery();this.dropdown.isEmpty&&a.length>=this.minLength?this.dropdown.update(a):this.dropdown.moveCursorDown(),this.dropdown.open()},_onLeftKeyed:function(){"rtl"===this.dir&&this._autocomplete()},_onRightKeyed:function(){"ltr"===this.dir&&this._autocomplete()},_onQueryChanged:function(a,b){this.input.clearHintIfInvalid(),b.length>=this.minLength?this.dropdown.update(b):this.dropdown.empty(),this.dropdown.open(),this._setLanguageDirection()},_onWhitespaceChanged:function(){this._updateHint(),this.dropdown.open()},_setLanguageDirection:function(){var a;this.dir!==(a=this.input.getLanguageDirection())&&(this.dir=a,this.$node.css("direction",a),this.dropdown.setLanguageDirection(a))},_updateHint:function(){var a,c,d,e,f,g;a=this.dropdown.getDatumForTopSuggestion(),a&&this.dropdown.isVisible()&&!this.input.hasOverflow()?(c=this.input.getInputValue(),d=h.normalizeQuery(c),e=b.escapeRegExChars(d),f=new RegExp("^(?:"+e+")(.+$)","i"),g=f.exec(a.value),g?this.input.setHint(c+g[1]):this.input.clearHint()):this.input.clearHint()},_autocomplete:function(a){var b,c,d,e;b=this.input.getHint(),c=this.input.getQuery(),d=a||this.input.isCursorAtEnd(),b&&c!==b&&d&&(e=this.dropdown.getDatumForTopSuggestion(),e&&this.input.setInputValue(e.value),this.eventBus.trigger("autocompleted",e.raw,e.datasetName))},_select:function(a){this.input.setQuery(a.value),this.input.setInputValue(a.value,!0),this._setLanguageDirection(),this.eventBus.trigger("selected",a.raw,a.datasetName),this.dropdown.close(),b.defer(b.bind(this.dropdown.empty,this.dropdown))},open:function(){this.dropdown.open()},close:function(){this.dropdown.close()},setVal:function(a){a=b.toStr(a),this.isActivated?this.input.setInputValue(a):(this.input.setQuery(a),this.input.setInputValue(a,!0)),this._setLanguageDirection()},getVal:function(){return this.input.getQuery()},destroy:function(){this.input.destroy(),this.dropdown.destroy(),k(this.$node),this.$node=null}}),f}();!function(){"use strict";var c,d,f;c=a.fn.typeahead,d="ttTypeahead",f={initialize:function(c,f){function g(){var g,h,i=a(this);b.each(f,function(a){a.highlight=!!c.highlight}),h=new k({input:i,eventBus:g=new e({el:i}),withHint:b.isUndefined(c.hint)?!0:!!c.hint,minLength:c.minLength,autoselect:c.autoselect,datasets:f}),i.data(d,h)}return f=b.isArray(f)?f:[].slice.call(arguments,1),c=c||{},this.each(g)},open:function(){function b(){var b,c=a(this);(b=c.data(d))&&b.open()}return this.each(b)},close:function(){function b(){var b,c=a(this);(b=c.data(d))&&b.close()}return this.each(b)},val:function(b){function c(){var c,e=a(this);(c=e.data(d))&&c.setVal(b)}function e(a){var b,c;return(b=a.data(d))&&(c=b.getVal()),c}return arguments.length?this.each(c):e(this.first())},destroy:function(){function b(){var b,c=a(this);(b=c.data(d))&&(b.destroy(),c.removeData(d))}return this.each(b)}},a.fn.typeahead=function(b){var c;return f[b]&&"initialize"!==b?(c=this.filter(function(){return!!a(this).data(d)}),f[b].apply(c,[].slice.call(arguments,1))):f.initialize.apply(this,arguments)},a.fn.typeahead.noConflict=function(){return a.fn.typeahead=c,this}}()}(window.jQuery);
\ No newline at end of file
diff --git a/storm-core/src/ui/public/js/visualization.js b/storm-core/src/ui/public/js/visualization.js
index 1e1c4d6..9aab087 100644
--- a/storm-core/src/ui/public/js/visualization.js
+++ b/storm-core/src/ui/public/js/visualization.js
@@ -375,45 +375,67 @@
     });
 }
 
+function jsError(other) {
+  try {
+    other();
+  } catch (err) {
+    $.get("/templates/json-error-template.html", function(template) {
+      $("#json-response-error").append(Mustache.render($(template).filter("#json-error-template").html(),{error: "JS Error", errorMessage: err}));
+    });
+  }
+}
+
 var should_update;
 function show_visualization(sys) {
+    $.getJSON("/api/v1/topology/"+$.url("?id")+"/visualization-init",function(response,status,jqXHR) {
+        $.get("/templates/topology-page-template.html", function(template) {
+            jsError(function() {
+                var topologyVisualization = $("#visualization-container");
+                topologyVisualization.append(
+                    Mustache.render($(template)
+                        .filter("#topology-visualization-container-template")
+                        .html(),
+                        response));
+                });
 
-    if(sys == null)
-    {
-        sys = arbor.ParticleSystem(20, 1000, 0.15, true, 55, 0.02, 0.6);
-        sys.renderer = renderGraph("#topoGraph");
-        sys.stop();
+            if(sys == null)
+            {
+                sys = arbor.ParticleSystem(20, 1000, 0.15, true, 55, 0.02, 0.6);
+                sys.renderer = renderGraph("#topoGraph");
+                sys.stop();
 
-        $(".stream-box").click(function () { rechoose(topology_data, sys, this) });    
-    }
-
-    should_update = true;
-    var update_freq_ms = 10000;
-    var update = function(should_rechoose) {
-      if(should_update) {
-        $.ajax({
-            url: "/api/v1/topology/"+$.url("?id")+"/visualization",
-            success: function (data, status, jqXHR) {
-                topology_data = data;
-                update_data(topology_data, sys);
-                sys.renderer.signal_update();
-                sys.renderer.redraw();
-                if (should_update)
-                    setTimeout(update, update_freq_ms);
-                if (should_rechoose)
-                    $(".stream-box").each(function () {
-                        rechoose(topology_data, sys, this)
-                    });
+                $(".stream-box").click(function () { rechoose(topology_data, sys, this) });
             }
-        });
-      }
-    };
-    
-    update(true);
-    $("#visualization-container").show(500);
-    $("#show-hide-visualization").attr('value', 'Hide Visualization');
-    $("#show-hide-visualization").unbind("click");
-    $("#show-hide-visualization").click(function () { hide_visualization(sys) });
+
+            should_update = true;
+            var update_freq_ms = 10000;
+            var update = function(should_rechoose){
+              if(should_update) {
+                $.ajax({
+                    url: "/api/v1/topology/"+$.url("?id")+"/visualization",
+                    success: function(data, status, jqXHR) {
+                        topology_data = data;
+                        update_data(topology_data, sys);
+                        sys.renderer.signal_update();
+                        sys.renderer.redraw();
+                        if(should_update)
+                            setTimeout(update, update_freq_ms);
+                        if(should_rechoose)
+                            $(".stream-box").each(function () {
+                                rechoose(topology_data, sys, this)
+                            });
+                    }
+                });
+              }
+            };
+
+            update(true);
+            $("#visualization-container").show(500);
+            $("#show-hide-visualization").attr('value', 'Hide Visualization');
+            $("#show-hide-visualization").unbind("click");
+            $("#show-hide-visualization").click(function () { hide_visualization(sys) });
+        })
+    });
 }
 
 function hide_visualization(sys) {
diff --git a/storm-core/src/ui/public/logviewer_search.html b/storm-core/src/ui/public/logviewer_search.html
new file mode 100644
index 0000000..670f3fb
--- /dev/null
+++ b/storm-core/src/ui/public/logviewer_search.html
@@ -0,0 +1,65 @@
+<html><head>
+    <!--
+     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.
+    -->
+<meta charset="UTF-8">
+<meta name="viewport" content="width=device-width, initial-scale=1">
+<title>Storm UI</title>
+<link href="/css/bootstrap-3.3.1.min.css" rel="stylesheet" type="text/css">
+<link href="/css/jquery.dataTables.1.10.4.min.css" rel="stylesheet" type="text/css">
+<link href="/css/dataTables.bootstrap.css" rel="stylesheet" type="text/css">
+<link href="/css/style.css" rel="stylesheet" type="text/css">
+<script src="/js/jquery-1.11.1.min.js" type="text/javascript"></script>
+<script src="/js/jquery.dataTables.1.10.4.min.js" type="text/javascript"></script>
+<script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
+<script src="/js/jquery.mustache.js" type="text/javascript"></script>
+<script src="/js/bootstrap-3.3.1.min.js" type="text/javascript"></script>
+<script src="/js/jquery.blockUI.min.js" type="text/javascript"></script>
+<script src="/js/url.min.js" type="text/javascript"></script>
+<script src="/js/dataTables.bootstrap.min.js" type="text/javascript"></script>
+<script src="/js/script.js" type="text/javascript"></script>
+</head>
+<body>
+<div class="container-fluid">
+  <div class="row">
+    <div id="search-form"></div>
+  </div>
+  <div class="row">
+    <div id="result"></div>
+  </div>
+</div>
+</body>
+<script>
+$(document).ready(function() {
+    var file = $.url("?file");
+    var search = $.url("?search");
+    var offset = $.url("?offset") || 0;
+    file = decodeURIComponent(file);
+    search = decodeURIComponent(search);
+
+    $.get("/templates/logviewer-search-page-template.html", function(template) {
+        $("#search-form").append(Mustache.render($(template).filter("#search-single-file").html(),{file: file, search: search}));
+
+        var result = $("#result");
+        var url = "/search/"+encodeURIComponent(file)+"?search-string="+search+"&start-byte-offset="+offset;
+        $.getJSON(url,function(response,status,jqXHR) {
+           response.file = file;
+           result.append(Mustache.render($(template).filter("#logviewer-search-result-template").html(),response));
+       });
+    });
+  });
+</script>
+</html>
diff --git a/storm-core/src/ui/public/search_result.html b/storm-core/src/ui/public/search_result.html
new file mode 100644
index 0000000..ec44aa9
--- /dev/null
+++ b/storm-core/src/ui/public/search_result.html
@@ -0,0 +1,100 @@
+<html><head>
+    <!--
+     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.
+    -->
+<meta charset="UTF-8">
+<meta name="viewport" content="width=device-width, initial-scale=1">
+<title>Storm UI</title>
+<link href="/css/bootstrap-3.3.1.min.css" rel="stylesheet" type="text/css">
+<link href="/css/jquery.dataTables.1.10.4.min.css" rel="stylesheet" type="text/css">
+<link href="/css/dataTables.bootstrap.css" rel="stylesheet" type="text/css">
+<link href="/css/style.css" rel="stylesheet" type="text/css">
+<script src="/js/jquery-1.11.1.min.js" type="text/javascript"></script>
+<script src="/js/jquery.dataTables.1.10.4.min.js" type="text/javascript"></script>
+<script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
+<script src="/js/jquery.mustache.js" type="text/javascript"></script>
+<script src="/js/bootstrap-3.3.1.min.js" type="text/javascript"></script>
+<script src="/js/jquery.blockUI.min.js" type="text/javascript"></script>
+<script src="/js/url.min.js" type="text/javascript"></script>
+<script src="/js/dataTables.bootstrap.min.js" type="text/javascript"></script>
+<script src="/js/script.js" type="text/javascript"></script>
+</head>
+<body>
+<div class="container-fluid">
+  <div class="row">
+    <div id="search-form"></div>
+  </div>
+  <div class="row">
+    <div id="result"></div>
+  </div>
+</div>
+</body>
+<script>
+$(document).ready(function() {
+    var id = $.url("?id");
+    var search = $.url("?search");
+    var count = $.url("?count") || 2;
+    var searchArchived = $.url("?searchArchived") || "";
+
+    $.get("/templates/search-result-page-template.html", function(template) {
+        $("#search-form").append(Mustache.render($(template).filter("#search-form-template").html(),{id: id, search: search, count: count, searchArchived: searchArchived}));
+
+        var result = $("#result");
+        $.getJSON("/api/v1/topology-workers/"+id, function(response,status,jqXHR) {
+           for (var index in response.hostPortList) {
+             var host = response.hostPortList[index].host;
+             var port = response.hostPortList[index].port;
+             var elemId = "search-results-"+host.replace(/\./g,"-")+"-"+port;
+             response.hostPortList[index].elemId = elemId;
+           }
+           result.append(Mustache.render($(template).filter("#search-result-files-template").html(),response));
+
+           var logviewerPort = response.logviewerPort;
+           for (var index in response.hostPortList) {
+             var host = response.hostPortList[index].host;
+             var port = response.hostPortList[index].port;
+             var elemId = response.hostPortList[index].elemId;
+             var file = id+"/"+port+"/worker.log";
+             var searchURL = "http://"+host+":"+logviewerPort+"/search/"+encodeURIComponent(file)+"?search-string="+search+"&num-matches="+count;
+             if (searchArchived != "") {
+               searchURL = "http://"+host+":"+logviewerPort+"/deepSearch/"+id+"?search-string="+search+"&num-matches="+count+"&search-archived=true&port="+port;
+             }
+
+             $.ajax({dataType: "json",
+               url: searchURL,
+               xhrFields: {
+                 withCredentials: true
+               },
+               success: $.proxy(function(data,status,jqXHR) {
+                                  data.file = this.file;
+                                  data.host = this.host;
+                                  data.port = this.port;
+                                  data.id = id;
+                                  data.logviewerPort = logviewerPort;
+                                  var searchTemp = $(template).filter("#search-result-identified-template").html();
+                                  if (searchArchived != "") {
+                                    searchTemp = $(template).filter("#deepsearch-result-identified-template").html();
+                                  }
+                                  var rendered = Mustache.render(searchTemp, data);
+                                  $("#"+this.elemId).replaceWith(rendered);
+                                  }, {host: host, port: port, file: file, elemId: elemId})
+             });
+           }
+       });
+    });
+  });
+</script>
+</html>
diff --git a/storm-core/src/ui/public/templates/component-page-template.html b/storm-core/src/ui/public/templates/component-page-template.html
index ccb4d5a..f21b30d 100644
--- a/storm-core/src/ui/public/templates/component-page-template.html
+++ b/storm-core/src/ui/public/templates/component-page-template.html
@@ -227,6 +227,59 @@
     </tbody>
   </table>
 </script>
+<script id="profiling-template" type="text/html">
+  <h2>Profiling and Debugging</h2>
+  Use the following controls to profile and debug the components on this page.
+  <table class="table table-striped compact">
+    <thead>
+      <tr>
+        <th class="header">Component</th>
+        <th class="header">
+          <span data-original-title="The status of a running profiler or the timeout for one you're starting (in minutes)" data-toggle="tooltip">
+            Status / Timeout (Minutes)
+          </span>
+        </th>
+        <th class="header">Actions</th>
+      </tr>
+    </thead>
+    <tbody id="profiler-table-body">
+      <tr>
+        <td>
+          <select id="selected_worker">
+            {{#hosts}}
+            <option value="{{name}}">{{name}}</option>
+            {{/hosts}}
+          </select>
+        </td>
+        <td>
+          <input id="timeout" class="timeout_input" type="text" value="" placeholder="10"/>
+        </td>
+        <td>
+          <input type="button" value="Start" name="start" onClick="start_profiling(profiler_selected_worker())" class="btn btn-secondary"/>
+          <input type="button" value="JStack" name="jstack" onClick="dump_jstack(profiler_selected_worker())" class="btn btn-secondary"/>
+          <input type="button" value="Restart Worker" name="jvmrestart" onClick="restart_worker_jvm(profiler_selected_worker())" class="btn btn-secondary"/>
+          <input type="button" value="Heap" name="heap" onClick="dump_heap(profiler_selected_worker())" class="btn btn-secondary"/>
+        </td>
+      </tr>
+      {{> profilerActive}}
+    </tbody>
+  </table>
+</script>
+<script id="profiler-active-partial" type="text/html">
+  {{#profilerActive}}
+  <tr>
+    <td>{{host}}:{{port}}</td>
+    <td>Active until {{timestamp}}</td>
+    <td>
+      <input id="stop_{{host}}:{{port}}" type="button" value="Stop" name="stop" onClick="stop_profiling('{{host}}:{{port}}')" class="btn btn-secondary"/>
+      <input id="dump_profile_{{host}}:{{port}}" type="button" value="Dump Profile" name="dumpjprofile" onClick="dump_profile('{{host}}:{{port}}')" class="btn btn-secondary"/>
+      <input id="dump_jstack_{{host}}:{{port}}" type="button" value="JStack" name="jstack" onClick="dump_jstack('{{host}}:{{port}}')" class="btn btn-secondary"/>
+      <input id="restart_worker_jvm_{{host}}:{{port}}" type="button" value="Restart Worker" name="jvmrestart" onClick="restart_worker_jvm('{{host}}:{{port}}')" class="btn btn-secondary"/>
+      <input id="dump_heap_{{host}}:{{port}}" type="button" value="Heap" name="heap" onClick="dump_heap('{{host}}:{{port}}')" class="btn btn-secondary"/> <a href="{{dumplink}}">My Dump Files</a>
+    </td>
+  </tr>
+  {{/profilerActive}}
+</script>
 <script id="bolt-stats-template" type="text/html">
   <h2>Bolt stats</h2>
   <table class="table table-striped compact" id="bolt-stats-table">
@@ -481,7 +534,7 @@
     <tbody>
       {{#componentErrors}}
       <tr>
-        <td>{{time}}</td>
+        <td>{{errorTime}}</td>
         <td>{{errorHost}}</td>
         <td><a href="{{errorWorkerLogLink}}">{{errorPort}}</a></td>
         <td>
diff --git a/storm-core/src/ui/public/templates/deep-search-result-page-template.html b/storm-core/src/ui/public/templates/deep-search-result-page-template.html
new file mode 100644
index 0000000..a658e5c
--- /dev/null
+++ b/storm-core/src/ui/public/templates/deep-search-result-page-template.html
@@ -0,0 +1,66 @@
+<!--
+ 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.
+-->
+<script id="search-result-files-template" type="text/html">
+<table id="search-result-table" class="table table-striped compact">
+  <thead><tr><th>host:port</th><th>Match</th></tr></thead>
+  <tbody>
+    {{#supervisors}}
+    <tr id="{{elemId}}">
+      <td>{{host}}</td>
+      <td>LOADING ...</td>
+    </tr>
+    {{/supervisors}}
+  </tbody>
+</table>
+</script>
+<script id="search-result-identified-template" type="text/html">
+    {{#matches}}
+    <tr>
+      <td><a href="http://{{host}}:{{logviewerPort}}/logviewer_search.html?file={{fileName}}&search={{searchString}}">{{host}}:{{port}}</a></td>
+      <td><pre>{{beforeString}}<b><a href="{{logviewerURL}}">{{matchString}}</a></b>{{afterString}}</pre></td>
+    </tr>
+    {{/matches}}
+</script>
+<script id="search-form-template" type="text/html">
+  <div id="search-form-container" class="search-box">
+    <form action="deep_search_result.html" id="search-box">
+      <div class="row"  data-toggle="tooltip" data-placement="bottom" title="The ID of the topology, can be a regex for search, i.e. '.' lists everything">
+        <div class="col-md-2">Topology Id:</div>
+        <div class="col-md-10">
+          <input class="expand-box" id="search-id" name="id" class="typeahead" type="text" placeholder="Topology-ID" value="{{id}}" number="50">
+        </div>
+      </div>
+      <div class="row" data-toggle="tooltip" data-placement="bottom" title="Exact search string to look for (NOT A REGEX)">
+        <div class="col-md-2">Search:</div>
+        <div class="col-md-10">
+          <input class="expand-box" type="text" name="search" value="{{search}}" placeholder="Search" number="50">
+        </div>
+      </div>
+      <div class="row" data-toggle="tooltip" data-placement="bottom" title="Search older logs that have rotated and may have been compressed.  This could take much longer.">
+        <div class="col-md-2">Search archived logs:</div>
+        <div class="col-md-10"><input type="checkbox" name="search-archived" {{search_archived}}></div>
+      </div>
+      <div class="row">
+        <div class="col-md-12">
+          <input type="submit" value="Search">
+        </td>
+      </div>
+      <input type="hidden" name="count" value="{{count}}">
+    </form>
+  </div>
+</script>
+
diff --git a/storm-core/src/ui/public/templates/index-page-template.html b/storm-core/src/ui/public/templates/index-page-template.html
index 1e585ff..5173356 100644
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@ -87,7 +87,7 @@
                 <span data-toggle="tooltip" data-placement="above" title="Storm version this nimbus host is running.">Version</span>
             </th>
             <th>
-                <span data-toggle="tooltip" data-placement="left" title="Time since this nimbus host has been running.">UpTime Seconds</span>
+                <span data-toggle="tooltip" data-placement="left" title="Time since this nimbus host has been running.">UpTime</span>
             </th>
         </tr>
         </thead>
@@ -115,11 +115,6 @@
           </span>
         </th>
         <th>
-          <span data-toggle="tooltip" data-placement="right" title="The unique ID given to a Topology each time it is launched.">
-            Id
-          </span>
-        </th>
-        <th>
           <span data-toggle="tooltip" data-placement="above" title="The user that submitted the Topology, if authentication is enabled.">
             Owner
           </span>
@@ -155,6 +150,18 @@
           </span>
         </th>
         <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total Memory by Scheduler.">
+            Assigned Mem (MB)
+          </span>
+        </th>
+        {{#schedulerDisplayResource}}
+        <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total CPU by Scheduler. Every 100 means 1 core.">
+            Assigned CPU (%)
+          </span>
+        </th>
+        {{/schedulerDisplayResource}}
+        <th>
           <span data-toggle="tooltip" data-placement="left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
             Scheduler Info
           </span>
@@ -165,7 +172,6 @@
       {{#topologies}}
       <tr>
         <td><a href="/topology.html?id={{encodedId}}">{{name}}</a></td>
-        <td>{{id}}</td>
         <td>{{owner}}</td>
         <td>{{status}}</td>
         <td>{{uptime}}</td>
@@ -173,6 +179,10 @@
         <td>{{executorsTotal}}</td>
         <td>{{tasksTotal}}</td>
         <td>{{replicationCount}}</td>
+        <td>{{assignedTotalMem}}</td>
+        {{#schedulerDisplayResource}}
+        <td>{{assignedCpu}}</td>
+        {{/schedulerDisplayResource}}
         <td>{{schedulerInfo}}</td>
       </tr>
       {{/topologies}}
@@ -208,6 +218,30 @@
           Used slots
         </span>
       </th>
+      {{#schedulerDisplayResource}}
+      <th>
+        <span data-toggle="tooltip" data-placement="above" title="Memory capacity of a supervisor.">
+          Total Mem (MB)
+        </span>
+      </th>
+      {{/schedulerDisplayResource}}
+      <th>
+        <span data-toggle="tooltip" data-placement="left" title="Memory that has been allocated.">
+          Used Mem (MB)
+        </span>
+      </th>
+      {{#schedulerDisplayResource}}
+      <th>
+        <span data-toggle="tooltip" data-placement="above" title="CPU capacity of a supervisor. Every 100 means one core.">
+          Total CPU (%)
+        </span>
+      </th>
+      <th>
+        <span data-toggle="tooltip" data-placement="left" title="CPU that has been allocated. Every 100 means one core">
+          Used CPU (%)
+        </span>
+      </th>
+      {{/schedulerDisplayResource}}
       <th>
         <span data-toggle="tooltip" data-placement="left" title="Version">
           Version
@@ -223,6 +257,14 @@
       <td>{{uptime}}</td>
       <td>{{slotsTotal}}</td>
       <td>{{slotsUsed}}</td>
+      {{#schedulerDisplayResource}}
+      <td>{{totalMem}}</td>
+      {{/schedulerDisplayResource}}
+      <td>{{usedMem}}</td>
+      {{#schedulerDisplayResource}}
+      <td>{{totalCpu}}</td>
+      <td>{{usedCpu}}</td>
+      {{/schedulerDisplayResource}}
       <td>{{version}}</td>
     </tr>
     {{/supervisors}}
diff --git a/storm-core/src/ui/public/templates/logviewer-search-page-template.html b/storm-core/src/ui/public/templates/logviewer-search-page-template.html
new file mode 100644
index 0000000..9d208d3
--- /dev/null
+++ b/storm-core/src/ui/public/templates/logviewer-search-page-template.html
@@ -0,0 +1,44 @@
+<!--
+ 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.
+-->
+<script id="logviewer-search-result-template" type="text/html">
+{{#nextByteOffset}}
+<a href="/logviewer_search.html?file={{file}}&search={{searchString}}&offset={{nextByteOffset}}" class="btn btn-default enabled">Next</a>
+{{/nextByteOffset}}
+<table id="search-result-table" class="table table-striped compact">
+  <thead><tr><th>File offset</th><th>Match</th></tr></thead>
+  <tbody>
+    {{#matches}}
+    <tr>
+      <td>{{byteOffset}}</td>
+      <td><pre>{{beforeString}}<b><a href="{{logviewerURL}}">{{matchString}}</a></b>{{afterString}}</pre></td>
+    </tr>
+    {{/matches}}
+  </tbody>
+</table>
+{{#nextByteOffset}}
+<a href="/logviewer_search.html?file={{file}}&search={{searchString}}&offset={{nextByteOffset}}" class="btn btn-default enabled">Next</a>
+{{/nextByteOffset}}
+</script>
+<script id="search-single-file" type="text/html">
+<form action="logviewer_search.html" id="search-box">
+  Search {{file}}:
+  <input type="text" name="search" value="{{search}}">
+  <input type="hidden" name="file" value="{{file}}">
+  <input type="submit" value="Search">
+</form>
+</script>
+
diff --git a/storm-core/src/ui/public/templates/search-result-page-template.html b/storm-core/src/ui/public/templates/search-result-page-template.html
new file mode 100644
index 0000000..742a643
--- /dev/null
+++ b/storm-core/src/ui/public/templates/search-result-page-template.html
@@ -0,0 +1,60 @@
+<!--
+ 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.
+-->
+<script id="search-result-files-template" type="text/html">
+<table id="search-result-table" class="table table-striped compact">
+  <thead><tr><th>host:port</th><th>Match</th></tr></thead>
+  <tbody>
+    {{#hostPortList}}
+      <tr id="{{elemId}}">
+        <td>{{host}}:{{port}}</td>
+        <td>LOADING ...</td>
+      </tr>
+    {{/hostPortList}}
+  </tbody>
+</table>
+</script>
+<script id="search-result-identified-template" type="text/html">
+    {{#matches}}
+    <tr>
+      <td><a href="http://{{host}}:{{logviewerPort}}/logviewer_search.html?file={{file}}&search={{searchString}}">{{host}}:{{port}}</a></td>
+      <td><pre>{{beforeString}}<b><a href="{{logviewerURL}}">{{matchString}}</a></b>{{afterString}}</pre></td>
+    </tr>
+    {{/matches}}
+</script>
+<script id="deepsearch-result-identified-template" type="text/html">
+    {{#matches}}
+    {{#matches}}
+    <tr>
+      <td><a href="http://{{host}}:{{logviewerPort}}/logviewer_search.html?file={{fileName}}&search={{searchString}}">{{host}}:{{port}}</a></td>
+      <td><pre>{{beforeString}}<b><a href="{{logviewerURL}}">{{matchString}}</a></b>{{afterString}}</pre></td>
+    </tr>
+    {{/matches}}
+    {{/matches}}
+</script>
+<script id="search-form-template" type="text/html">
+  <div class="search-box">
+    <form action="search_result.html" id="search-box">
+      Search {{id}}:
+      <input type="text" name="search" value="{{search}}">
+      <input type="hidden" name="id" value="{{id}}">
+      <input type="hidden" name="count" value="{{count}}">
+      <input type="submit" value="Search">
+      Search Archived Logs: <input type="checkbox" name="searchArchived" value="checked" {{searchArchived}}>
+    </form>
+  </div>
+</script>
+
diff --git a/storm-core/src/ui/public/templates/topology-page-template.html b/storm-core/src/ui/public/templates/topology-page-template.html
index 3292ee0..1f81f1b 100644
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@ -64,6 +64,11 @@
           </span>
         </th>
         <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total Memory by Scheduler.">
+            Assigned Mem (MB)
+          </span>
+        </th>
+        <th>
           <span data-toggle="tooltip" data-placement="left" title="This shows information from the scheduler about the latest attempt to schedule the Topology on the cluster.">
             Scheduler Info
           </span>
@@ -81,11 +86,72 @@
         <td>{{executorsTotal}}</td>
         <td>{{tasksTotal}}</td>
         <td>{{replicationCount}}</td>
+        <td>{{assignedTotalMem}}</td>
         <td>{{schedulerInfo}}</td>
       </tr>
     </tbody>
   </table>
 </script>
+<script id="topology-resources-template" type="text/html">
+  <table id="topology-resources-table" class="table compact">
+    <thead>
+    <tr>
+      <th>
+          <span data-toggle="tooltip" data-placement="right" title="The name given to the topology by when it was submitted.">
+            Name
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="right" title="The unique ID given to a Topology each time it is launched.">
+            Id
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total On-Heap Memory by User.">
+            Requested On-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total On-Heap Memory by Scheduler.">
+            Assigned On-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total Off-Heap Memory by User.">
+            Requested Off-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Assigned Total Off-Heap Memory by Scheduler.">
+            Assigned Off-Heap Memory (MB)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="above" title="Requested Total CPU by User. Every 100 means 1 core.">
+            Requested CPU (%)
+          </span>
+      </th>
+      <th>
+          <span data-toggle="tooltip" data-placement="left" title="Assigned Total CPU by Scheduler. Every 100 means 1 core.">
+            Assigned CPU (%)
+          </span>
+      </th>
+    </tr>
+    </thead>
+    <tbody>
+    <tr>
+      <td>{{name}}</td>
+      <td>{{id}}</td>
+      <td>{{requestedMemOnHeap}}</td>
+      <td>{{assignedMemOnHeap}}</td>
+      <td>{{requestedMemOffHeap}}</td>
+      <td>{{assignedMemOffHeap}}</td>
+      <td>{{requestedCpu}}</td>
+      <td>{{assignedCpu}}</td>
+    </tr>
+    </tbody>
+  </table>
+</script>
 <script id="topology-stats-template" type="text/html">
   <h2>Topology stats</h2>
   <table class="table table-striped compact" id="topology-stats-table">
@@ -137,33 +203,37 @@
     </tbody>
   </table>
 </script>
+
+<script id="topology-visualization-container-template" type="text/html">
+  <p>
+    <table class="table table-striped compact">
+      <thead>
+        <tr>
+          <th class="header" colspan=4>
+            Streams
+          </th>
+        </tr>
+      </thead>
+      {{#visualizationTable}}
+      <tr>
+        {{#:row}}
+        <td>
+          <input type="checkbox" id={{:sani-stream}} class="stream-box" {{#:checked}}checked{{/:checked}}/>
+          {{:stream}}
+        </td>
+        {{/:row}}
+      </tr>
+      {{/visualizationTable}}
+    </table>
+  </p>
+  <canvas id="topoGraph" width=1024 height=768 style="border:1px solid #000000;">
+</script>
+
 <script id="topology-visualization-template" type="text/html">
   <h2>Topology Visualization</h2>
   <input type="button" id="show-hide-visualization" value="Show Visualization" class="btn btn-default"/>
   <p>
     <div id="visualization-container" style="display:none;">
-      <p>
-        <table class="table table-striped compact">
-          <thead>
-            <tr>
-              <th class="header" colspan=4>
-                Streams
-              </th>
-            </tr>
-          </thead>
-          {{#visualizationTable}}
-          <tr>
-            {{#:row}}
-            <td>
-              <input type="checkbox" id={{:sani-stream}} class="stream-box" {{#:checked}}checked{{/:checked}}/>
-              {{:stream}}
-            </td>
-            {{/:row}}
-          </tr>
-          {{/visualizationTable}}
-        </table>
-      </p>
-      <canvas id="topoGraph" width=1024 height=768 style="border:1px solid #000000;">
     </div>
   </p>
 </script>
@@ -253,7 +323,7 @@
           <span id="{{errorLapsedSecs}}" class="errorSpan">{{lastError}}</span>
         </td>
         <td>
-          <span id="{{time}}" class="errorTime" data-toggle="tooltip" title="{{errorLapsedSecs}}">{{time}}</span>
+          <span id="{{errorTime}}" class="errorTime" data-toggle="tooltip" title="{{errorLapsedSecs}}">{{errorTime}}</span>
         </td>
         {{/spouts}}
     </tbody>
@@ -347,7 +417,7 @@
           <span id="{{errorLapsedSecs}}" class="errorSpan">{{lastError}}</span>
         </td>
         <td>
-          <span id="{{time}}" class="errorTime" data-toggle="tooltip" title="{{errorLapsedSecs}}">{{time}}</span>
+          <span id="{{errorTime}}" class="errorTime" data-toggle="tooltip" title="{{errorLapsedSecs}}">{{errorTime}}</span>
         </td>
         {{/bolts}}
     </tbody>
@@ -362,5 +432,84 @@
     <input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30, 'wait time in seconds')" type="button" value="Kill" class="btn btn-default">
     <input {{startDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/enable', true, {{currentSamplingPct}}, 'sampling percentage', 'debug')" type="button" value="Debug" class="btn btn-default">
     <input {{stopDebugStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'debug/disable', false, 0, 'sampling percentage', 'stop debugging')" type="button" value="Stop Debug" class="btn btn-default">
+    <input type="button" {{logLevel}} value="Change Log Level" onclick="toggleChangeLogLevel()" class="btn btn-default">
   </p>
+  <!-- 
+    Contents of topology-change-log-level-template will be added to the div below
+    once user clicks on "Change Log Level"
+  -->
+  <div id="change-log-level" style="display:none"></div>
+</script>
+
+<script id="search-form-template" type="text/html">
+  <form action="search_result.html" id="search-box">
+    Search {{id}}:
+    <input type="text" name="search">
+    <input type="hidden" name="id" value="{{id}}">
+    <input type="hidden" name="count" value="1">
+    <input type="submit" value="Search">
+    Search Archived Logs: <input type="checkbox" name="searchArchived" value="checked">
+  </form>
+</script>
+
+<script id="log-level-and-timeout" type="text/html" class="partials">
+    <tr id="logger-{{loggerId}}" class="{{cls}}">
+        {{#loggerName}}
+          <td>{{loggerName}}</td>
+          <input type="hidden" id="loggerName-{{loggerId}}" value="{{loggerName}}"/>
+          <input type="hidden" id="loggerRemove-{{loggerId}}" value="false"/>
+        {{/loggerName}}
+
+        {{#isNew}}
+          <td>
+            <input type="text" id="loggerName-{{loggerId}}" placeholder="com.your.organization.LoggerName"/>
+          </td>
+        {{/isNew}}
+
+        <td>
+            <select id="loggerLevel-{{loggerId}}">
+                {{#levels}} 
+                    <option value="{{name}}" {{levelSelected}}>{{name}}</option>
+                {{/levels}}
+            </select>
+        </td>
+        <td>
+            <input type="text" class="timeout_input" id="loggerTimeout-{{loggerId}}" placeholder="30" value="{{timeout}}"/> 
+        </td>
+        <td>
+            {{absoluteTimeout}}
+        </td>
+        <td>
+            {{#isNew}}
+                <input type="button" value="Add" name="clear" onClick='sendLoggerLevel({{loggerId}})' class="btn btn-secondary"/>
+            {{/isNew}}
+            {{#loggerName}}
+                <input type="button" value="Apply" name="clear" onClick='sendLoggerLevel({{loggerId}})' class="btn btn-secondary"/>
+                {{#canClear}}
+                    <input type="button" value="Clear" name="clear" onClick='clearLoggerLevel({{loggerId}})' class="btn btn-secondary"/>
+                {{/canClear}}
+            {{/loggerName}}
+        </td>
+    </tr>
+</script>
+
+<script id="topology-change-log-level-template" type="text/html">
+  <div id="change-log-level">
+      <h3>Change Log Level</h3>
+      Modify the logger levels for topology. Note that applying a setting restarts the timer in the workers. To configure the root logger, use the name ROOT. 
+      <table class="table table-striped compact">
+          <thead>
+              <tr>
+                  <th class="header">Logger</th>
+                  <th class="header">Level</th>
+                  <th class="header">Timeout (sec)</th>
+                  <th class="header">Expires at</th>
+                  <th class="header">Actions</th>
+              </tr>
+          </thead>
+      {{#loggers}}
+          {{ > log-level-and-timeout }}
+      {{/loggers}}
+      </table>
+  </div>
 </script>
\ No newline at end of file
diff --git a/storm-core/src/ui/public/templates/user-template.html b/storm-core/src/ui/public/templates/user-template.html
index 4fa1338..853f0c9 100644
--- a/storm-core/src/ui/public/templates/user-template.html
+++ b/storm-core/src/ui/public/templates/user-template.html
@@ -15,11 +15,24 @@
  limitations under the License.
 -->
 <script id="user-template" type="text/html">
- {{#user}}
-  <div class="ui-user"><p>
-    <span data-toggle="tooltip" data-placement="below" data-original-title="This should be you.">
-            User: {{user}}
-    </span>
-  </p></div>
-  {{/user}}
+ <div class="ui-user">
+     {{#central-log-url}}
+        <span data-toggle="tooltip" data-placement="bottom" data-original-title="Link to Central Logging">
+                <a href="{{central-log-url}}" target="_blank"><image src="/images/statistic.png" class="image_pre_format"/></a>
+        </span>
+      {{/central-log-url}}
+     {{#bugtracker-url}}
+        <span data-toggle="tooltip" data-placement="bottom" data-original-title="Submit BUG for Help and tracking the issue.">
+                <a href="{{bugtracker-url}}" target="_blank"><image src="/images/bug.png" class="image_pre_format"/></a>
+        </span>
+     {{/bugtracker-url}}
+     <span data-toggle="tooltip" data-placement="bottom" data-original-title="Search through all logs for any topology.">
+             <a href="/deep_search_result.html" target="_blank"><image src="/images/search.png" class="image_pre_format"/></a>
+     </span>
+     {{#user}}
+        <span data-toggle="tooltip" data-placement="bottom" data-original-title="This should be you.">
+                User: {{user}}
+        </span>
+     {{/user}}
+ </div>
 </script>
diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html
index 2b1214c..5869d9a 100644
--- a/storm-core/src/ui/public/topology.html
+++ b/storm-core/src/ui/public/topology.html
@@ -48,12 +48,22 @@
     <div id="ui-user" class="col-md-1"></div>
   </div>
   <div class="row">
+      <div class="col-md-12" id="search-form">
+      </div>
+  </div>
+  <div class="row">
     <div class="col-md-12">
       <h2>Topology summary</h2>
       <div id="topology-summary"></div>
     </div>
   </div>
   <div class="row">
+      <div class="col-md-12">
+          <h2 id="topology-resources-header">Topology resources</h2>
+          <div id="topology-resources"></div>
+      </div>
+  </div>
+  <div class="row">
     <div id="topology-actions" class="col-md-12"></div>
   </div>
   <div class="row">
@@ -82,6 +92,146 @@
 </div>
 </body>
 <script>
+
+var toggleChangeLogLevel;
+
+function closeChangeLogLevel (){
+    var container = $("#change-log-level");
+    container.hide();
+}
+
+function clearLoggerLevel(id){
+    $("#loggerRemove-" + id).val("true");
+    sendLoggerLevel(id);
+    $("#logger-" + id).remove();
+}
+
+function sendLoggerLevel(id){
+    var topologyId = $.url("?id");
+    var shouldRemove = $("#loggerRemove-" + id).val() === "true";
+    var level = $("#loggerLevel-" + id).val();
+    var timeout = parseInt($("#loggerTimeout-" + id).val());
+        timeout = isNaN(timeout) ? 0 : timeout;
+    var loggerName = $("#loggerName-" + id).val();
+    if (level === 'Pick Level'){
+        alert ('Please pick a valid log level');
+        return;
+    }
+    var levelBelowInfo = level === 'DEBUG';
+
+    if (!shouldRemove && levelBelowInfo && timeout <= 0){
+        timeout = parseInt (
+            prompt ("You must provide a timeout > 0 for DEBUG log level. What timeout would you like (secs)?", 30));
+        if (!timeout){
+            return;
+        }
+    }
+    var data = {};
+    var loggerSetting;
+
+    if (id === 0) {
+        data["namedLoggerLevels"] = {};
+        data["namedLoggerLevels"]["ROOT"] = {};
+        loggerSetting = data["namedLoggerLevels"]["ROOT"];
+    } else {
+        data["namedLoggerLevels"] = {};
+        data["namedLoggerLevels"][loggerName] = {};
+        loggerSetting = data["namedLoggerLevels"][loggerName];
+    }
+
+    loggerSetting.target_level = shouldRemove ? null : level;
+    loggerSetting.reset_level  = "INFO";
+    loggerSetting.timeout = timeout;
+
+    sendRequest (topologyId, "logconfig", null, data, toggleChangeLogLevel);
+};
+
+function renderLogLevelForm (template, responseData){
+    var topologyId = $.url("?id");
+    var container = $("#change-log-level");
+
+    var levels = [
+        {name: "Pick Level"},
+        {name: "ALL"},
+        {name: "TRACE"},
+        {name: "DEBUG"},
+        {name: "INFO" },
+        {name: "WARN" },
+        {name: "ERROR"},
+        {name: "FATAL"},
+        {name: "OFF"}
+    ];
+    var partialTemplates = $(template).filter('.partials');
+    var partials = {};
+
+    $.each(partialTemplates, function (ix, partial){
+        var obj = $(partial);
+        partials [obj.attr('id')] = obj.html(); 
+    });
+
+    var logLevelTemplate = $(template).filter("#topology-change-log-level-template").html();
+
+    var calcAbsoluteTimeout = function (timeout_epoch) {
+        var absoluteTimeout = "";
+        if (timeout_epoch) {
+            var d = new Date(0); 
+            d.setUTCSeconds(timeout_epoch / 1000);
+            absoluteTimeout = d.toLocaleDateString() + " " + d.toLocaleTimeString();
+        }
+        return absoluteTimeout;
+    };
+    var renderImpl = function (data){
+        var loggers = [];
+        var loggerCount = 1;
+        var obj = data.namedLoggerLevels;
+        if (!obj) {
+            obj = {};
+            data.namedLoggerLevels = obj;
+        }
+
+        var sortedLoggers = Object.keys(obj).sort(function (l1, l2){
+            if (l1 === "ROOT") return -1;
+            if (l2 === "ROOT") return 1;
+            return l1 > l2;
+        });
+
+        sortedLoggers.forEach (function (l){
+            var obj = data.namedLoggerLevels[l];
+            obj.loggerId = loggerCount++;
+            obj.loggerName = l;
+            obj.named = l != "ROOT";
+            obj.cls = "namedLoggers";
+            obj.levelSelected = function (obj){
+                return function (){
+                    return this.name === obj.target_level ? "selected" : "";
+                }
+            }(obj);
+            obj.absoluteTimeout = calcAbsoluteTimeout (obj.timeout_epoch);
+            obj.canClear = true;
+            loggers.push(obj);
+        });
+
+        loggers.push({
+            loggerId: loggerCount,
+            isNew: true,
+            cls: 'newLogger'
+        });
+
+        var tmplData = {
+            loggers: loggers,
+            levels: levels
+        };
+
+        container.html(Mustache.render(logLevelTemplate, tmplData, partials));
+        container.show('fast');
+    };
+    if (!responseData) {
+        var topologyId = $.url("?id");
+        $.get ('/api/v1/topology/' + topologyId + '/logconfig', renderImpl);
+    } else {
+        renderImpl (responseData);
+    }
+} 
 $(document).ajaxStop($.unblockUI);
 $(document).ajaxStart(function(){
     if ($("#topology-visualization").children().size() == 0) {
@@ -125,17 +275,29 @@
         });
 
         var topologySummary = $("#topology-summary");
+        var topologyResources = $("#topology-resources");
         var topologyStats = $("#topology-stats");
         var spoutStats = $("#spout-stats");
         var boltStats = $("#bolt-stats");
         var config = $("#topology-configuration");
         var topologyActions = $("#topology-actions");
         var topologyVisualization = $("#topology-visualization")
+        var searchForm = $("#search-form")
         var formattedConfig = formatConfigData(response["configuration"]);
         var buttonJsonData = topologyActionJson(response["id"],response["encodedId"],response["name"],response["status"]
                                                 ,response["msgTimeout"],response["debug"],response["samplingPct"]);
-        $.get("/templates/topology-page-template.html", function(template) {
+        $.ajax ({url: "/templates/topology-page-template.html", success: function(template) {
+            toggleChangeLogLevel = function (data) {
+              renderLogLevelForm (template, data);
+            }
+            searchForm.append(Mustache.render($(template).filter("#search-form-template").html(),{id: topologyId}));
             topologySummary.append(Mustache.render($(template).filter("#topology-summary-template").html(),response));
+            topologyResources.append(Mustache.render($(template).filter("#topology-resources-template").html(),response));
+            var displayResource = response["schedulerDisplayResource"];
+            if (!displayResource){
+                $('#topology-resources-header').hide();
+                $('#topology-resources').hide();
+            }
             topologyActions.append(Mustache.render($(template).filter("#topology-actions-template").html(),buttonJsonData));
             topologyStats.append(Mustache.render($(template).filter("#topology-stats-template").html(),response));
             //window, emitted, transferred, complete latency, acked, failed
@@ -222,8 +384,8 @@
             $('#topology-configuration [data-toggle="tooltip"]').tooltip();
             $('#topology-actions [data-toggle="tooltip"]').tooltip();
             $('#topology-visualization [data-toggle="tooltip"]').tooltip();
-        });
+      }});
     });
  });
 </script>
-</html>
\ No newline at end of file
+</html>
diff --git a/storm-core/test/clj/backtype/storm/cluster_test.clj b/storm-core/test/clj/backtype/storm/cluster_test.clj
index b413d43..eea4637 100644
--- a/storm-core/test/clj/backtype/storm/cluster_test.clj
+++ b/storm-core/test/clj/backtype/storm/cluster_test.clj
@@ -23,6 +23,7 @@
   (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
   (:import [backtype.storm.utils Utils TestUtils ZookeeperAuthInfo])
+  (:import [backtype.storm.cluster ClusterState])
   (:require [backtype.storm [zookeeper :as zk]])
   (:require [conjure.core])
   (:use [conjure core])
@@ -170,8 +171,8 @@
 (deftest test-storm-cluster-state-basics
   (with-inprocess-zookeeper zk-port
     (let [state (mk-storm-state zk-port)
-          assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {})
-          assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {})
+          assignment1 (Assignment. "/aaa" {} {[1] ["1" 1001 1]} {} {})
+          assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {} {})
           nimbusInfo1 (NimbusInfo. "nimbus1" 6667 false)
           nimbusInfo2 (NimbusInfo. "nimbus2" 6667 false)
           nimbusSummary1 (NimbusSummary. "nimbus1" 6667 (current-time-secs) false "v1")
@@ -264,8 +265,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)
@@ -312,9 +313,9 @@
       (mk-distributed-cluster-state {})
       (verify-call-times-for zk/mkdirs 1)
       (verify-first-call-args-for-indices zk/mkdirs [2] nil))
-    (stubbing [mk-distributed-cluster-state nil
-               register nil
-               mkdirs nil]
+    (stubbing [mk-distributed-cluster-state (reify ClusterState
+                                              (register [this callback] nil)
+                                              (mkdirs [this path acls] nil))]
       (mk-storm-cluster-state {})
       (verify-call-times-for mk-distributed-cluster-state 1)
       (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))
diff --git a/storm-core/test/clj/backtype/storm/config_test.clj b/storm-core/test/clj/backtype/storm/config_test.clj
deleted file mode 100644
index 99032bd..0000000
--- a/storm-core/test/clj/backtype/storm/config_test.clj
+++ /dev/null
@@ -1,186 +0,0 @@
-;; Licensed to the Apache Software Foundation (ASF) under one
-;; or more contributor license agreements.  See the NOTICE file
-;; distributed with this work for additional information
-;; regarding copyright ownership.  The ASF licenses this file
-;; to you under the Apache License, Version 2.0 (the
-;; "License"); you may not use this file except in compliance
-;; with the License.  You may obtain a copy of the License at
-;;
-;; http://www.apache.org/licenses/LICENSE-2.0
-;;
-;; Unless required by applicable law or agreed to in writing, software
-;; distributed under the License is distributed on an "AS IS" BASIS,
-;; WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-;; See the License for the specific language governing permissions and
-;; limitations under the License.
-(ns backtype.storm.config-test
-  (:import [backtype.storm Config ConfigValidation])
-  (:import [backtype.storm.scheduler TopologyDetails])
-  (:import [backtype.storm.utils Utils])
-  (:use [clojure test])
-  (:use [backtype.storm config util])
-  )
-
-(deftest test-validity
-  (is (Utils/isValidConf {TOPOLOGY-DEBUG true "q" "asasdasd" "aaa" (Integer. "123") "bbb" (Long. "456") "eee" [1 2 (Integer. "3") (Long. "4")]}))
-  (is (not (Utils/isValidConf {"qqq" (backtype.storm.utils.Utils.)})))
-  )
-
-(deftest test-power-of-2-validator
-  (let [validator ConfigValidation/PowerOf2Validator]
-    (doseq [x [42.42 42 23423423423 -33 -32 -1 -0.00001 0 -0 "Forty-two"]]
-      (is (thrown-cause? java.lang.IllegalArgumentException
-        (.validateField validator "test" x))))
-
-    (doseq [x [64 4294967296 1 nil]]
-      (is (nil? (try
-                  (.validateField validator "test" x)
-                  (catch Exception e e)))))))
-
-(deftest test-list-validator
-  (let [validator ConfigValidation/StringsValidator]
-    (doseq [x [
-               ["Forty-two" 42]
-               [42]
-               [true "false"]
-               [nil]
-               [nil "nil"]
-              ]]
-      (is (thrown-cause-with-msg?
-            java.lang.IllegalArgumentException #"(?i).*each element.*"
-        (.validateField validator "test" x))))
-
-    (doseq [x ["not a list at all"]]
-      (is (thrown-cause-with-msg?
-            java.lang.IllegalArgumentException #"(?i).*must be an iterable.*"
-        (.validateField validator "test" x))))
-
-    (doseq [x [
-               ["one" "two" "three"]
-               [""]
-               ["42" "64"]
-               nil
-              ]]
-    (is (nil? (try
-                (.validateField validator "test" x)
-                (catch Exception e e)))))))
-
-(deftest test-integer-validator
-  (let [validator ConfigValidation/IntegerValidator]
-    (.validateField validator "test" nil)
-    (.validateField validator "test" 1000)
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" 1.34)))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" (inc Integer/MAX_VALUE))))))
-
-(deftest test-pos-integer-validator
-  (let [validator ConfigValidation/NotNullPosIntegerValidator]
-    (is (thrown-cause? java.lang.IllegalArgumentException
-        (.validateField validator "test" nil)))
-    (.validateField validator "test" 1000)
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" 1.34)))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" 0)))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" -100)))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" (inc Integer/MAX_VALUE))))))
-
-(deftest test-integers-validator
-  (let [validator ConfigValidation/NoDuplicateIntegersValidator]
-    (.validateField validator "test" nil)
-    (.validateField validator "test" [1000 0 -1000])
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" [0 10 10])))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" [0 10 1.34])))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" [0 nil])))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" [-100 (inc Integer/MAX_VALUE)])))))
-
-(deftest test-positive-number-validator
-  (let [validator ConfigValidation/PositiveNumberValidator]
-    (.validateField validator "test" nil)
-    (.validateField validator "test" 1.0)
-    (.validateField validator "test" 1)
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" -1.0)))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" -1)))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" 0)))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" 0.0)))))
-
-(deftest test-topology-workers-is-integer
-  (let [validator (CONFIG-SCHEMA-MAP TOPOLOGY-WORKERS)]
-    (.validateField validator "test" 42)
-    (is (thrown-cause? java.lang.IllegalArgumentException
-      (.validateField validator "test" 3.14159)))))
-
-(deftest test-topology-stats-sample-rate-is-float
-  (let [validator (CONFIG-SCHEMA-MAP TOPOLOGY-STATS-SAMPLE-RATE)]
-    (.validateField validator "test" 0.5)
-    (.validateField validator "test" 10)
-    (.validateField validator "test" Double/MAX_VALUE)))
-
-(deftest test-isolation-scheduler-machines-is-map
-  (let [validator (CONFIG-SCHEMA-MAP ISOLATION-SCHEDULER-MACHINES)]
-    (is (nil? (try
-                (.validateField validator "test" {})
-                (catch Exception e e))))
-    (is (nil? (try
-                (.validateField validator "test" {"host0" 1 "host1" 2})
-                (catch Exception e e))))
-    (is (thrown-cause? java.lang.IllegalArgumentException
-      (.validateField validator "test" 42)))))
-
-(deftest test-positive-integer-validator
-  (let [validator ConfigValidation/PositiveIntegerValidator]
-    (doseq [x [42.42 -32 0 -0 "Forty-two"]]
-      (is (thrown-cause? java.lang.IllegalArgumentException
-        (.validateField validator "test" x))))
-
-    (doseq [x [42 4294967296 1 nil]]
-      (is (nil? (try
-                  (.validateField validator "test" x)
-                  (catch Exception e e)))))))
-
-(deftest test-worker-childopts-is-string-or-string-list
-  (let [pass-cases [nil "some string" ["some" "string" "list"]]]
-    (testing "worker.childopts validates"
-      (let [validator (CONFIG-SCHEMA-MAP WORKER-CHILDOPTS)]
-        (doseq [value pass-cases]
-          (is (nil? (try
-                      (.validateField validator "test" value)
-                      (catch Exception e e)))))
-        (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" 42)))))
-
-    (testing "topology.worker.childopts validates"
-      (let [validator (CONFIG-SCHEMA-MAP TOPOLOGY-WORKER-CHILDOPTS)]
-        (doseq [value pass-cases]
-          (is (nil? (try
-                      (.validateField validator "test" value)
-                      (catch Exception e e)))))
-        (is (thrown-cause? java.lang.IllegalArgumentException
-          (.validateField validator "test" 42)))))))
-
-(deftest test-absolute-storm-local-dir
-  (let [storm-home-key "storm.home"
-        conf-relative {STORM-LOCAL-DIR "storm-local"}
-        conf-absolute {STORM-LOCAL-DIR
-                       (if on-windows?
-                         "C:\\storm-local"
-                         "/var/storm-local")}]
-    (testing
-      "for relative path"
-      (is (= (str (System/getProperty storm-home-key) file-path-separator (conf-relative STORM-LOCAL-DIR))
-             (absolute-storm-local-dir conf-relative))))
-    (testing
-      "for absolute path"
-      (is (= (if on-windows? "C:\\storm-local" "/var/storm-local")
-             (absolute-storm-local-dir conf-absolute))))))
\ No newline at end of file
diff --git a/storm-core/test/clj/backtype/storm/grouping_test.clj b/storm-core/test/clj/backtype/storm/grouping_test.clj
index 2bfa066..f914591 100644
--- a/storm-core/test/clj/backtype/storm/grouping_test.clj
+++ b/storm-core/test/clj/backtype/storm/grouping_test.clj
@@ -17,27 +17,85 @@
   (:use [clojure test])
   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestWordBytesCounter NGrouping]
            [backtype.storm.generated JavaObject JavaObjectArg])
-  (:use [backtype.storm testing clojure])
-  (:use [backtype.storm.daemon common])
+  (:import [backtype.storm.grouping LoadMapping])
+  (:use [backtype.storm testing clojure log config])
+  (:use [backtype.storm.daemon common executor])
   (:require [backtype.storm [thrift :as thrift]]))
 
 (deftest test-shuffle
+ (let [shuffle-fn (mk-shuffle-grouper [(int 1) (int 2)] {TOPOLOGY-DISABLE-LOADAWARE-MESSAGING true} nil "comp" "stream")
+       num-messages 100000
+       min-prcnt (int (* num-messages 0.49))
+       max-prcnt (int (* num-messages 0.51))
+       data [1 2]
+       freq (frequencies (for [x (range 0 num-messages)] (shuffle-fn (int 1) data nil)))
+       load1 (.get freq [(int 1)])
+       load2 (.get freq [(int 2)])]
+    (log-message "FREQ:" freq)
+    (is (>= load1 min-prcnt))
+    (is (<= load1 max-prcnt))
+    (is (>= load2 min-prcnt))
+    (is (<= load2 max-prcnt))))
+
+(deftest test-shuffle-load-even
+ (let [shuffle-fn (mk-shuffle-grouper [(int 1) (int 2)] {} nil "comp" "stream")
+       num-messages 100000
+       min-prcnt (int (* num-messages 0.49))
+       max-prcnt (int (* num-messages 0.51))
+       load (LoadMapping.)
+       _ (.setLocal load {(int 1) 0.0 (int 2) 0.0})
+       data [1 2]
+       freq (frequencies (for [x (range 0 num-messages)] (shuffle-fn (int 1) data load)))
+       load1 (.get freq [(int 1)])
+       load2 (.get freq [(int 2)])]
+    (log-message "FREQ:" freq)
+    (is (>= load1 min-prcnt))
+    (is (<= load1 max-prcnt))
+    (is (>= load2 min-prcnt))
+    (is (<= load2 max-prcnt))))
+
+(deftest test-shuffle-load-uneven
+ (let [shuffle-fn (mk-shuffle-grouper [(int 1) (int 2)] {} nil "comp" "stream")
+       num-messages 100000
+       min1-prcnt (int (* num-messages 0.32))
+       max1-prcnt (int (* num-messages 0.34))
+       min2-prcnt (int (* num-messages 0.65))
+       max2-prcnt (int (* num-messages 0.67))
+       load (LoadMapping.)
+       _ (.setLocal load {(int 1) 0.5 (int 2) 0.0})
+       data [1 2]
+       freq (frequencies (for [x (range 0 num-messages)] (shuffle-fn (int 1) data load)))
+       load1 (.get freq [(int 1)])
+       load2 (.get freq [(int 2)])]
+    (log-message "FREQ:" freq)
+    (is (>= load1 min1-prcnt))
+    (is (<= load1 max1-prcnt))
+    (is (>= load2 min2-prcnt))
+    (is (<= load2 max2-prcnt))))
+
+(deftest test-field
   (with-simulated-time-local-cluster [cluster :supervisors 4]
-    (let [topology (thrift/mk-topology
-                    {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
-                    {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.)
-                                            :parallelism-hint 6)
+    (let [spout-phint 4
+          bolt-phint 6
+          topology (thrift/mk-topology
+                    {"1" (thrift/mk-spout-spec (TestWordSpout. true)
+                                               :parallelism-hint spout-phint)}
+                    {"2" (thrift/mk-bolt-spec {"1" ["word"]}
+                                              (TestWordBytesCounter.)
+                                              :parallelism-hint bolt-phint)
                      })
-          results (complete-topology cluster
-                                     topology
-                                     ;; important for test that
-                                     ;; #tuples = multiple of 4 and 6
-                                     :mock-sources {"1" (->> [["a"] ["b"]]
-                                                             (repeat 12)
-                                                             (apply concat))})]
-      (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
-               (read-tuples results "2")))
-      )))
+          results (complete-topology
+                    cluster
+                    topology
+                    :mock-sources {"1" (->> [[(.getBytes "a")]
+                                             [(.getBytes "b")]]
+                                            (repeat (* spout-phint bolt-phint))
+                                            (apply concat))})]
+      (is (ms= (apply concat
+                      (for [value '("a" "b")
+                            sum (range 1 (inc (* spout-phint bolt-phint)))]
+                        [[value sum]]))
+               (read-tuples results "2"))))))
 
 (deftest test-field
   (with-simulated-time-local-cluster [cluster :supervisors 4]
diff --git a/storm-core/test/clj/backtype/storm/integration_test.clj b/storm-core/test/clj/backtype/storm/integration_test.clj
index 712b195..cc0208d 100644
--- a/storm-core/test/clj/backtype/storm/integration_test.clj
+++ b/storm-core/test/clj/backtype/storm/integration_test.clj
@@ -17,7 +17,7 @@
   (:use [clojure test])
   (:import [backtype.storm Config])
   (:import [backtype.storm.topology TopologyBuilder])
-  (:import [backtype.storm.generated InvalidTopologyException SubmitOptions TopologyInitialStatus])
+  (:import [backtype.storm.generated InvalidTopologyException SubmitOptions TopologyInitialStatus RebalanceOptions])
   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount
             TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout])
   (:import [backtype.storm.tuple Fields])
@@ -62,13 +62,13 @@
   (with-simulated-time-local-cluster [cluster :supervisors 4]
     (let [topology (thrift/mk-topology
                     {"1" (thrift/mk-spout-spec (TestWordSpout. true))}
-                    {"2" (thrift/mk-bolt-spec {"1" :shuffle} emit-task-id
+                    {"2" (thrift/mk-bolt-spec {"1" :all} emit-task-id
                       :parallelism-hint 3
                       :conf {TOPOLOGY-TASKS 6})
                      })
           results (complete-topology cluster
                                      topology
-                                     :mock-sources {"1" [["a"] ["a"] ["a"] ["a"] ["a"] ["a"]]})]
+                                     :mock-sources {"1" [["a"]]})]
       (is (ms= [[0] [1] [2] [3] [4] [5]]
                (read-tuples results "2")))
       )))
@@ -105,6 +105,7 @@
                              "timeout-tester"
                              {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10}
                              topology)
+      (advance-cluster-time cluster 11)
       (.feed feeder ["a"] 1)
       (.feed feeder ["b"] 2)
       (.feed feeder ["c"] 3)
@@ -236,6 +237,7 @@
                              "acking-test1"
                              {}
                              (:topology tracked))
+      (advance-cluster-time cluster 11)
       (.feed feeder1 [1])
       (tracked-wait tracked 1)
       (checker1 0)
@@ -278,6 +280,7 @@
                              "test-acking2"
                              {}
                              (:topology tracked))
+      (advance-cluster-time cluster 11)
       (.feed feeder [1])
       (tracked-wait tracked 1)
       (checker 0)
@@ -323,6 +326,7 @@
         {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10}
         topology
         (SubmitOptions. TopologyInitialStatus/INACTIVE))
+      (advance-cluster-time cluster 11)
       (.feed feeder ["a"] 1)
       (advance-cluster-time cluster 9)
       (is (not @bolt-prepared?))
@@ -347,6 +351,7 @@
                              "test"
                              {}
                              (:topology tracked))
+      (advance-cluster-time cluster 11)
       (.feed feeder [1])
       (tracked-wait tracked 1)
       (checker 1)
@@ -568,6 +573,7 @@
                                               TOPOLOGY-DEBUG true
                                               }
                                              (:topology tracked))
+            _ (advance-cluster-time cluster 11)
             storm-id (get-storm-id state "test-errors")
             errors-count (fn [] (count (.errors state storm-id "2")))]
 
diff --git a/storm-core/test/clj/backtype/storm/logviewer_test.clj b/storm-core/test/clj/backtype/storm/logviewer_test.clj
index a790ddf..2afa88c 100644
--- a/storm-core/test/clj/backtype/storm/logviewer_test.clj
+++ b/storm-core/test/clj/backtype/storm/logviewer_test.clj
@@ -20,30 +20,38 @@
   (:require [conjure.core])
   (:use [clojure test])
   (:use [conjure core])
+  (:use [backtype.storm.ui helpers])
+  (:import [java.nio.file Files])
+  (:import [java.nio.file.attribute FileAttribute])
+  (:import [java.io File])
   (:import [org.mockito Mockito]))
 
 (defmulti mk-mock-File #(:type %))
 
-(defmethod mk-mock-File :file [{file-name :name mtime :mtime
-                                :or {file-name "afile" mtime 1}}]
+(defmethod mk-mock-File :file [{file-name :name mtime :mtime length :length
+                                :or {file-name "afile"
+                                     mtime 1
+                                     length (* 10 (* 1024 (* 1024 1024))) }}] ; Length 10 GB
   (let [mockFile (Mockito/mock java.io.File)]
     (. (Mockito/when (.getName mockFile)) thenReturn file-name)
     (. (Mockito/when (.lastModified mockFile)) thenReturn mtime)
     (. (Mockito/when (.isFile mockFile)) thenReturn true)
     (. (Mockito/when (.getCanonicalPath mockFile))
-       thenReturn (str "/mock/canonical/path/to/" file-name))
+      thenReturn (str "/mock/canonical/path/to/" file-name))
+    (. (Mockito/when (.length mockFile)) thenReturn length)
     mockFile))
 
-(defmethod mk-mock-File :directory [{dir-name :name mtime :mtime
-                                     :or {dir-name "adir" mtime 1}}]
+(defmethod mk-mock-File :directory [{dir-name :name mtime :mtime files :files
+                                     :or {dir-name "adir" mtime 1 files []}}]
   (let [mockDir (Mockito/mock java.io.File)]
     (. (Mockito/when (.getName mockDir)) thenReturn dir-name)
     (. (Mockito/when (.lastModified mockDir)) thenReturn mtime)
     (. (Mockito/when (.isFile mockDir)) thenReturn false)
+    (. (Mockito/when (.listFiles mockDir)) thenReturn (into-array File files))
     mockDir))
 
 (deftest test-mk-FileFilter-for-log-cleanup
-  (testing "log file filter selects the correct log files for purge"
+  (testing "log file filter selects the correct worker-log dirs for purge"
     (let [now-millis (current-time-millis)
           conf {LOGVIEWER-CLEANUP-AGE-MINS 60
                 LOGVIEWER-CLEANUP-INTERVAL-SECS 300}
@@ -51,120 +59,181 @@
           old-mtime-millis (- cutoff-millis 500)
           new-mtime-millis (+ cutoff-millis 500)
           matching-files (map #(mk-mock-File %)
-                              [{:name "oldlog-1-2-worker-3.log"
-                                :type :file
-                                :mtime old-mtime-millis}
-                               {:name "oldlog-1-2-worker-3.log.8"
-                                :type :file
-                                :mtime old-mtime-millis}
-                               {:name "foobar*_topo-1-24242-worker-2834238.log"
-                                :type :file
-                                :mtime old-mtime-millis}])
+                           [{:name "3031"
+                             :type :directory
+                             :mtime old-mtime-millis}
+                            {:name "3032"
+                             :type :directory
+                             :mtime old-mtime-millis}
+                            {:name "7077"
+                             :type :directory
+                             :mtime old-mtime-millis}])
           excluded-files (map #(mk-mock-File %)
-                              [{:name "oldlog-1-2-worker-.log"
-                                :type :file
-                                :mtime old-mtime-millis}
-                               {:name "olddir-1-2-worker.log"
-                                :type :directory
-                                :mtime old-mtime-millis}
-                               {:name "newlog-1-2-worker.log"
-                                :type :file
-                                :mtime new-mtime-millis}
-                               {:name "some-old-file.txt"
-                                :type :file
-                                :mtime old-mtime-millis}
-                               {:name "metadata"
-                                :type :directory
-                                :mtime old-mtime-millis}
-                               {:name "newdir-1-2-worker.log"
-                                :type :directory
-                                :mtime new-mtime-millis}
-                               {:name "newdir"
-                                :type :directory
-                                :mtime new-mtime-millis}
-                              ])
+                           [{:name "oldlog-1-2-worker-.log"
+                             :type :file
+                             :mtime old-mtime-millis}
+                            {:name "newlog-1-2-worker.log"
+                             :type :file
+                             :mtime new-mtime-millis}
+                            {:name "some-old-file.txt"
+                             :type :file
+                             :mtime old-mtime-millis}
+                            {:name "olddir-1-2-worker.log"
+                             :type :directory
+                             :mtime new-mtime-millis}
+                            {:name "metadata"
+                             :type :directory
+                             :mtime new-mtime-millis}
+                            {:name "newdir"
+                             :type :directory
+                             :mtime new-mtime-millis}
+                            ])
           file-filter (logviewer/mk-FileFilter-for-log-cleanup conf now-millis)]
-        (is   (every? #(.accept file-filter %) matching-files))
-        (is (not-any? #(.accept file-filter %) excluded-files))
+      (is   (every? #(.accept file-filter %) matching-files))
+      (is (not-any? #(.accept file-filter %) excluded-files))
       )))
 
-(deftest test-get-log-root->files-map
-  (testing "returns map of root name to list of files"
-    (let [files (vec (map #(java.io.File. %) ["log-1-2-worker-3.log"
-                                              "log-1-2-worker-3.log.1.gz"
-                                              "log-1-2-worker-3.log.err"
-                                              "log-1-2-worker-3.log.out"
-                                              "log-1-2-worker-3.log.out.1.gz"
-                                              "log-1-2-worker-3.log.1"
-                                              "log-2-4-worker-6.log.1"]))
-          expected {"log-1-2-worker-3" #{(files 0) (files 1) (files 2) (files 3) (files 4) (files 5)}
-                    "log-2-4-worker-6" #{(files 6)}}]
-      (is (= expected (logviewer/get-log-root->files-map files))))))
+(deftest test-sort-worker-logs
+  (testing "cleaner sorts the log files in ascending ages for deletion"
+    (stubbing [logviewer/filter-candidate-files (fn [x _] x)]
+      (let [now-millis (current-time-millis)
+            files1 (into-array File (map #(mk-mock-File {:name (str %)
+                                                         :type :file
+                                                         :mtime (- now-millis (* 100 %))})
+                                      (range 1 6)))
+            files2 (into-array File (map #(mk-mock-File {:name (str %)
+                                                         :type :file
+                                                         :mtime (- now-millis (* 100 %))})
+                                      (range 6 11)))
+            files3 (into-array File (map #(mk-mock-File {:name (str %)
+                                                         :type :file
+                                                         :mtime (- now-millis (* 100 %))})
+                                      (range 11 16)))
+            port1-dir (mk-mock-File {:name "/workers-artifacts/topo1/port1"
+                                     :type :directory
+                                     :files files1})
+            port2-dir (mk-mock-File {:name "/workers-artifacts/topo1/port2"
+                                     :type :directory
+                                     :files files2})
+            port3-dir (mk-mock-File {:name "/workers-artifacts/topo2/port3"
+                                     :type :directory
+                                     :files files3})
+            topo1-files (into-array File [port1-dir port2-dir])
+            topo2-files (into-array File [port3-dir])
+            topo1-dir (mk-mock-File {:name "/workers-artifacts/topo1"
+                                     :type :directory
+                                     :files topo1-files})
+            topo2-dir (mk-mock-File {:name "/workers-artifacts/topo2"
+                                     :type :directory
+                                     :files topo2-files})
+            root-files (into-array File [topo1-dir topo2-dir])
+            root-dir (mk-mock-File {:name "/workers-artifacts"
+                                    :type :directory
+                                    :files root-files})
+            sorted-logs (logviewer/sorted-worker-logs root-dir)
+            sorted-ints (map #(Integer. (.getName %)) sorted-logs)]
+        (is (= (count sorted-logs) 15))
+        (is (= (count sorted-ints) 15))
+        (is (apply #'> sorted-ints))))))
 
-(deftest test-identify-worker-log-files
-  (testing "Does not include metadata file when there are any log files that
-           should not be cleaned up"
-    (let [cutoff-millis 2000
-          old-logFile (mk-mock-File {:name "mock-1-1-worker-1.log.1"
-                                     :type :file
-                                     :mtime (- cutoff-millis 1000)})
-          mock-metaFile (mk-mock-File {:name "mock-1-1-worker-1.yaml"
-                                       :type :file
-                                       :mtime 1})
-          new-logFile (mk-mock-File {:name "mock-1-1-worker-1.log"
-                                     :type :file
-                                     :mtime (+ cutoff-millis 1000)})
+(deftest test-per-workerdir-cleanup
+  (testing "cleaner deletes oldest files in each worker dir if files are larger than per-dir quota."
+    (stubbing [rmr nil]
+      (let [now-millis (current-time-millis)
+            files1 (into-array File (map #(mk-mock-File {:name (str "A" %)
+                                                         :type :file
+                                                         :mtime (+ now-millis (* 100 %))
+                                                         :length 200 })
+                                      (range 0 10)))
+            files2 (into-array File (map #(mk-mock-File {:name (str "B" %)
+                                                         :type :file
+                                                         :mtime (+ now-millis (* 100 %))
+                                                         :length 200 })
+                                      (range 0 10)))
+            files3 (into-array File (map #(mk-mock-File {:name (str "C" %)
+                                                         :type :file
+                                                         :mtime (+ now-millis (* 100 %))
+                                                         :length 200 })
+                                      (range 0 10)))
+            port1-dir (mk-mock-File {:name "/workers-artifacts/topo1/port1"
+                                     :type :directory
+                                     :files files1})
+            port2-dir (mk-mock-File {:name "/workers-artifacts/topo1/port2"
+                                     :type :directory
+                                     :files files2})
+            port3-dir (mk-mock-File {:name "/workers-artifacts/topo2/port3"
+                                     :type :directory
+                                     :files files3})
+            topo1-files (into-array File [port1-dir port2-dir])
+            topo2-files (into-array File [port3-dir])
+            topo1-dir (mk-mock-File {:name "/workers-artifacts/topo1"
+                                     :type :directory
+                                     :files topo1-files})
+            topo2-dir (mk-mock-File {:name "/workers-artifacts/topo2"
+                                     :type :directory
+                                     :files topo2-files})
+            root-files (into-array File [topo1-dir topo2-dir])
+            root-dir (mk-mock-File {:name "/workers-artifacts"
+                                    :type :directory
+                                    :files root-files})
+            remaining-logs (logviewer/per-workerdir-cleanup root-dir 1200)]
+        (is (= (count (first remaining-logs)) 6))
+        (is (= (count (second remaining-logs)) 6))
+        (is (= (count (last remaining-logs)) 6))))))
+
+(deftest test-delete-oldest-log-cleanup
+  (testing "delete oldest logs deletes the oldest set of logs when the total size gets too large."
+    (stubbing [rmr nil]
+      (let [now-millis (current-time-millis)
+            files (into-array File (map #(mk-mock-File {:name (str %)
+                                                        :type :file
+                                                        :mtime (+ now-millis (* 100 %))
+                                                        :length 100 })
+                                     (range 0 20)))
+            remaining-logs (logviewer/delete-oldest-while-logs-too-large files 501)]
+        (is (= (logviewer/sum-file-size files) 2000))
+        (is (= (count remaining-logs) 5))
+        (is (= (.getName (first remaining-logs)) "15"))))))
+
+(deftest test-identify-worker-log-dirs
+  (testing "Build up workerid-workerlogdir map for the old workers' dirs"
+    (let [port1-dir (mk-mock-File {:name "/workers-artifacts/topo1/port1"
+                                   :type :directory})
+          mock-metaFile (mk-mock-File {:name "worker.yaml"
+                                       :type :file})
           exp-id "id12345"
-          exp-user "alice"
-          expected {exp-id {:owner exp-user
-                            :files #{old-logFile}}}]
+          expected {exp-id port1-dir}]
       (stubbing [supervisor/read-worker-heartbeats nil
-                logviewer/get-metadata-file-for-log-root-name mock-metaFile
-                read-dir-contents [(.getName old-logFile) (.getName new-logFile)]
-                logviewer/get-worker-id-from-metadata-file exp-id
-                logviewer/get-topo-owner-from-metadata-file exp-user]
-        (is (= expected (logviewer/identify-worker-log-files [old-logFile] "/tmp/")))))))
+                 logviewer/get-metadata-file-for-wroker-logdir mock-metaFile
+                 logviewer/get-worker-id-from-metadata-file exp-id]
+        (is (= expected (logviewer/identify-worker-log-dirs [port1-dir])))))))
 
-(deftest test-get-dead-worker-files-and-owners
+(deftest test-get-dead-worker-dirs
   (testing "removes any files of workers that are still alive"
     (let [conf {SUPERVISOR-WORKER-TIMEOUT-SECS 5}
           id->hb {"42" {:time-secs 1}}
           now-secs 2
-          log-files #{:expected-file :unexpected-file}
-          exp-owner "alice"]
-      (stubbing [logviewer/identify-worker-log-files {"42" {:owner exp-owner
-                                                            :files #{:unexpected-file}}
-                                                      "007" {:owner exp-owner
-                                                             :files #{:expected-file}}}
-                 logviewer/get-topo-owner-from-metadata-file "alice"
+          unexpected-dir (mk-mock-File {:name "dir1" :type :directory})
+          expected-dir (mk-mock-File {:name "dir2" :type :directory})
+          log-dirs #{unexpected-dir expected-dir}]
+      (stubbing [logviewer/identify-worker-log-dirs {"42" unexpected-dir,
+                                                     "007" expected-dir}
                  supervisor/read-worker-heartbeats id->hb]
-        (is (= [{:owner exp-owner :files #{:expected-file}}]
-               (logviewer/get-dead-worker-files-and-owners conf now-secs log-files "/tmp/")))))))
+        (is (= #{expected-dir}
+              (logviewer/get-dead-worker-dirs conf now-secs log-dirs)))))))
 
 (deftest test-cleanup-fn
-  (testing "cleanup function removes file as user when one is specified"
-    (let [exp-user "mock-user"
-          mockfile1 (mk-mock-File {:name "file1" :type :file})
-          mockfile2 (mk-mock-File {:name "file2" :type :file})
-          mockfile3 (mk-mock-File {:name "file3" :type :file})
-          mockyaml  (mk-mock-File {:name "foo.yaml" :type :file})
-          exp-cmd (str "rmr /mock/canonical/path/to/" (.getName mockfile3))]
-      (stubbing [logviewer/select-files-for-cleanup
-                   [(mk-mock-File {:name "throwaway" :type :file})]
-                 logviewer/get-dead-worker-files-and-owners
-                   [{:owner nil :files #{mockfile1}}
-                    {:files #{mockfile2}}
-                    {:owner exp-user :files #{mockfile3 mockyaml}}]
-                 supervisor/worker-launcher nil
+  (testing "cleanup function rmr's files of dead workers"
+    (let [mockfile1 (mk-mock-File {:name "delete-me1" :type :file})
+          mockfile2 (mk-mock-File {:name "delete-me2" :type :file})]
+      (stubbing [logviewer/select-dirs-for-cleanup nil
+                 logviewer/get-dead-worker-dirs (sorted-set mockfile1 mockfile2)
+                 logviewer/cleanup-empty-topodir nil
                  rmr nil]
-        (logviewer/cleanup-fn! "/tmp/")
-        (verify-call-times-for supervisor/worker-launcher 1)
-        (verify-first-call-args-for-indices supervisor/worker-launcher
-                                            [1 2] exp-user exp-cmd)
-        (verify-call-times-for rmr 3)
+        (logviewer/cleanup-fn! "/bogus/path")
+        (verify-call-times-for rmr 2)
         (verify-nth-call-args-for 1 rmr (.getCanonicalPath mockfile1))
-        (verify-nth-call-args-for 2 rmr (.getCanonicalPath mockfile2))
-        (verify-nth-call-args-for 3 rmr (.getCanonicalPath mockyaml))))))
+        (verify-nth-call-args-for 2 rmr (.getCanonicalPath mockfile2))))))
 
 (deftest test-authorized-log-user
   (testing "allow cluster admin"
@@ -212,3 +281,456 @@
       (is (not (logviewer/authorized-log-user? "alice" "non-blank-fname" {})))
       (verify-first-call-args-for logviewer/get-log-user-group-whitelist "non-blank-fname")
       (verify-first-call-args-for logviewer/user-groups "alice"))))
+
+(deftest test-list-log-files
+  (testing "list-log-files filter selects the correct log files to return"
+    (let [attrs (make-array FileAttribute 0)
+          root-path (.getCanonicalPath (.toFile (Files/createTempDirectory "workers-artifacts" attrs)))
+          file1 (clojure.java.io/file root-path "topoA" "port1" "worker.log")
+          file2 (clojure.java.io/file root-path "topoA" "port2" "worker.log")
+          file3 (clojure.java.io/file root-path "topoB" "port1" "worker.log")
+          _ (clojure.java.io/make-parents file1)
+          _ (clojure.java.io/make-parents file2)
+          _ (clojure.java.io/make-parents file3)
+          _ (.createNewFile file1)
+          _ (.createNewFile file2)
+          _ (.createNewFile file3)
+          origin "www.origin.server.net"
+          expected-all (json-response '("topoA/port1/worker.log" "topoA/port2/worker.log"
+                                         "topoB/port1/worker.log")
+                         nil
+                         :headers {"Access-Control-Allow-Origin" origin
+                                   "Access-Control-Allow-Credentials" "true"})
+          expected-filter-port (json-response '("topoA/port1/worker.log" "topoB/port1/worker.log")
+                                 nil
+                                 :headers {"Access-Control-Allow-Origin" origin
+                                           "Access-Control-Allow-Credentials" "true"})
+          expected-filter-topoId (json-response '("topoB/port1/worker.log")
+                                   nil
+                                   :headers {"Access-Control-Allow-Origin" origin
+                                             "Access-Control-Allow-Credentials" "true"})
+          returned-all (logviewer/list-log-files "user" nil nil root-path nil origin)
+          returned-filter-port (logviewer/list-log-files "user" nil "port1" root-path nil origin)
+          returned-filter-topoId (logviewer/list-log-files "user" "topoB" nil root-path nil origin)]
+      (rmr root-path)
+      (is   (= expected-all returned-all))
+      (is   (= expected-filter-port returned-filter-port))
+      (is   (= expected-filter-topoId returned-filter-topoId)))))
+
+(deftest test-search-via-rest-api
+  (testing "Throws if bogus file is given"
+    (thrown-cause? java.lang.RuntimeException
+      (logviewer/substring-search nil "a string")))
+
+  (let [pattern "needle"
+        expected-host "dev.null.invalid"
+        expected-port 8888
+        ;; When we click a link to the logviewer, we expect the match line to
+        ;; be somewhere near the middle of the page.  So we subtract half of
+        ;; the default page length from the offset at which we found the
+        ;; match.
+        exp-offset-fn #(- (/ logviewer/default-bytes-per-page 2) %)]
+
+    (stubbing [local-hostname expected-host
+               logviewer/logviewer-port expected-port]
+
+      (testing "Logviewer link centers the match in the page"
+        (let [expected-fname "foobar.log"]
+          (is (= (str "http://"
+                   expected-host
+                   ":"
+                   expected-port
+                   "/log?file="
+                   expected-fname
+                   "&start=1947&length="
+                   logviewer/default-bytes-per-page)
+                (logviewer/url-to-match-centered-in-log-page (byte-array 42)
+                  expected-fname
+                  27526
+                  8888)))))
+
+      (let [file (->> "logviewer-search-context-tests.log"
+                   (clojure.java.io/file "src" "dev"))]
+        (testing "returns correct before/after context"
+          (is (= {"searchString" pattern
+                  "startByteOffset" 0
+                  "matches" [{"byteOffset" 0
+                              "beforeString" ""
+                              "afterString" " needle000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000needle "
+                              "matchString" pattern
+                              "logviewerURL" (str "http://"
+                                               expected-host
+                                               ":"
+                                               expected-port
+                                               "/log?file=src%2Fdev%2F"
+                                               (.getName file)
+                                               "&start=0&length=51200")}
+                             {"byteOffset" 7
+                              "beforeString" "needle "
+                              "afterString" "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000needle needle\n"
+                              "matchString" pattern
+                              "logviewerURL" (str "http://"
+                                               expected-host
+                                               ":"
+                                               expected-port
+                                               "/log?file=src%2Fdev%2F"
+                                               (.getName file)
+                                               "&start=0&length=51200")}
+                             {"byteOffset" 127
+                              "beforeString" "needle needle000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"
+                              "afterString" " needle\n"
+                              "matchString" pattern
+                              "logviewerURL" (str "http://"
+                                               expected-host
+                                               ":"
+                                               expected-port
+                                               "/log?file=src%2Fdev%2F"
+                                               (.getName file)
+                                               "&start=0&length=51200")}
+                             {"byteOffset" 134
+                              "beforeString" " needle000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000needle "
+                              "afterString" "\n"
+                              "matchString" pattern
+                              "logviewerURL" (str "http://"
+                                               expected-host
+                                               ":"
+                                               expected-port
+                                               "/log?file=src%2Fdev%2F"
+                                               (.getName file)
+                                               "&start=0&length=51200")}
+                             ]}
+                (logviewer/substring-search file pattern)))))
+
+      (let [file (clojure.java.io/file "src" "dev" "small-worker.log")]
+        (testing "a really small log file"
+          (is (= {"searchString" pattern
+                  "startByteOffset" 0
+                  "matches" [{"byteOffset" 7
+                              "beforeString" "000000 "
+                              "afterString" " 000000\n"
+                              "matchString" pattern
+                              "logviewerURL" (str "http://"
+                                               expected-host
+                                               ":"
+                                               expected-port
+                                               "/log?file=src%2Fdev%2F"
+                                               (.getName file)
+                                               "&start=0&length=51200")}]}
+                (logviewer/substring-search file pattern)))))
+
+      (let [file (clojure.java.io/file "src" "dev" "test-3072.log")]
+        (testing "no offset returned when file ends on buffer offset"
+          (let [expected
+                {"searchString" pattern
+                 "startByteOffset" 0
+                 "matches" [{"byteOffset" 3066
+                             "beforeString" (->>
+                                              (repeat 128 '.)
+                                              clojure.string/join)
+                             "afterString" ""
+                             "matchString" pattern
+                             "logviewerURL" (str "http://"
+                                              expected-host
+                                              ":"
+                                              expected-port
+                                              "/log?file=src%2Fdev%2F"
+                                              (.getName file)
+                                              "&start=0&length=51200")}]}]
+            (is (= expected
+                  (logviewer/substring-search file pattern)))
+            (is (= expected
+                  (logviewer/substring-search file pattern :num-matches 1))))))
+
+      (let [file (clojure.java.io/file "src" "dev" "test-worker.log")]
+
+        (testing "next byte offsets are correct for each match"
+          (doseq [[num-matches-sought
+                   num-matches-found
+                   expected-next-byte-offset] [[1 1 11]
+                                               [2 2 2042]
+                                               [3 3 2052]
+                                               [4 4 3078]
+                                               [5 5 3196]
+                                               [6 6 3202]
+                                               [7 7 6252]
+                                               [8 8 6321]
+                                               [9 9 6397]
+                                               [10 10 6476]
+                                               [11 11 6554]
+                                               [12 12 nil]
+                                               [13 12 nil]]]
+            (let [result
+                  (logviewer/substring-search file
+                    pattern
+                    :num-matches num-matches-sought)]
+              (is (= expected-next-byte-offset
+                    (get result "nextByteOffset")))
+              (is (= num-matches-found (count (get result "matches")))))))
+
+        (is
+          (= {"nextByteOffset" 6252
+              "searchString" pattern
+              "startByteOffset" 0
+              "matches" [
+                          {"byteOffset" 5
+                           "beforeString" "Test "
+                           "afterString" " is near the beginning of the file.\nThis file assumes a buffer size of 2048 bytes, a max search string size of 1024 bytes, and a"
+                           "matchString" pattern
+                           "logviewerURL" (str "http://"
+                                            expected-host
+                                            ":"
+                                            expected-port
+                                            "/log?file=src%2Fdev%2F"
+                                            (.getName file)
+                                            "&start=0&length=51200")}
+                          {"byteOffset" 2036
+                           "beforeString" "ng 146\npadding 147\npadding 148\npadding 149\npadding 150\npadding 151\npadding 152\npadding 153\nNear the end of a 1024 byte block, a "
+                           "afterString" ".\nA needle that straddles a 1024 byte boundary should also be detected.\n\npadding 157\npadding 158\npadding 159\npadding 160\npadding"
+                           "matchString" pattern
+                           "logviewerURL" (str "http://"
+                                            expected-host
+                                            ":"
+                                            expected-port
+                                            "/log?file=src%2Fdev%2F"
+                                            (.getName file)
+                                            "&start=0&length=51200")}
+                          {"byteOffset" 2046
+                           "beforeString" "ding 147\npadding 148\npadding 149\npadding 150\npadding 151\npadding 152\npadding 153\nNear the end of a 1024 byte block, a needle.\nA "
+                           "afterString" " that straddles a 1024 byte boundary should also be detected.\n\npadding 157\npadding 158\npadding 159\npadding 160\npadding 161\npaddi"
+                           "matchString" pattern
+                           "logviewerURL" (str "http://"
+                                            expected-host
+                                            ":"
+                                            expected-port
+                                            "/log?file=src%2Fdev%2F"
+                                            (.getName file)
+                                            "&start=0&length=51200")}
+                          {"byteOffset" 3072
+                           "beforeString" "adding 226\npadding 227\npadding 228\npadding 229\npadding 230\npadding 231\npadding 232\npadding 233\npadding 234\npadding 235\n\n\nHere a "
+                           "afterString" " occurs just after a 1024 byte boundary.  It should have the correct context.\n\nText with two adjoining matches: needleneedle\n\npa"
+                           "matchString" pattern
+                           "logviewerURL" (str "http://"
+                                            expected-host
+                                            ":"
+                                            expected-port
+                                            "/log?file=src%2Fdev%2F"
+                                            (.getName file)
+                                            "&start=0&length=51200")}
+                          {"byteOffset" 3190
+                           "beforeString" "\n\n\nHere a needle occurs just after a 1024 byte boundary.  It should have the correct context.\n\nText with two adjoining matches: "
+                           "afterString" "needle\n\npadding 243\npadding 244\npadding 245\npadding 246\npadding 247\npadding 248\npadding 249\npadding 250\npadding 251\npadding 252\n"
+                           "matchString" pattern
+                           "logviewerURL" (str "http://"
+                                            expected-host
+                                            ":"
+                                            expected-port
+                                            "/log?file=src%2Fdev%2F"
+                                            (.getName file)
+                                            "&start=0&length=51200")}
+                          {"byteOffset" 3196
+                           "beforeString" "e a needle occurs just after a 1024 byte boundary.  It should have the correct context.\n\nText with two adjoining matches: needle"
+                           "afterString" "\n\npadding 243\npadding 244\npadding 245\npadding 246\npadding 247\npadding 248\npadding 249\npadding 250\npadding 251\npadding 252\npaddin"
+                           "matchString" pattern
+                           "logviewerURL" (str "http://"
+                                            expected-host
+                                            ":"
+                                            expected-port
+                                            "/log?file=src%2Fdev%2F"
+                                            (.getName file)
+                                            "&start=0&length=51200")}
+                          {"byteOffset" 6246
+                           "beforeString" "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX\n\nHere are four non-ascii 1-byte UTF-8 characters: αβγδε\n\n"
+                           "afterString" "\n\nHere are four printable 2-byte UTF-8 characters: ¡¢£¤¥\n\nneedle\n\n\n\nHere are four printable 3-byte UTF-8 characters: ऄअ"
+                           "matchString" pattern
+                           "logviewerURL" (str "http://"
+                                            expected-host
+                                            ":"
+                                            expected-port
+                                            "/log?file=src%2Fdev%2F"
+                                            (.getName file)
+                                            "&start=0&length=51200")}
+                          ]}
+            (logviewer/substring-search file pattern :num-matches 7)))
+
+        (testing "Correct match offset is returned when skipping bytes"
+          (let [start-byte-offset 3197]
+            (is (= {"nextByteOffset" 6252
+                    "searchString" pattern
+                    "startByteOffset" start-byte-offset
+                    "matches" [{"byteOffset" 6246
+                                "beforeString" "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX\n\nHere are four non-ascii 1-byte UTF-8 characters: αβγδε\n\n"
+                                "afterString" "\n\nHere are four printable 2-byte UTF-8 characters: ¡¢£¤¥\n\nneedle\n\n\n\nHere are four printable 3-byte UTF-8 characters: ऄअ"
+                                "matchString" pattern
+                                "logviewerURL" (str "http://"
+                                                 expected-host
+                                                 ":"
+                                                 expected-port
+                                                 "/log?file=src%2Fdev%2F"
+                                                 (.getName file)
+                                                 "&start=0&length=51200")}]}
+                  (logviewer/substring-search file
+                    pattern
+                    :num-matches 1
+                    :start-byte-offset start-byte-offset)))))
+
+        (let [pattern (clojure.string/join (repeat 1024 'X))]
+          (is
+            (= {"nextByteOffset" 6183
+                "searchString" pattern
+                "startByteOffset" 0
+                "matches" [
+                            {"byteOffset" 4075
+                             "beforeString" "\n\nThe following match of 1024 bytes completely fills half the byte buffer.  It is a search substring of the maximum size......\n\n"
+                             "afterString" "\nThe following max-size match straddles a 1024 byte buffer.\nXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX"
+                             "matchString" pattern
+                             "logviewerURL" (str "http://"
+                                              expected-host
+                                              ":"
+                                              expected-port
+                                              "/log?file=src%2Fdev%2F"
+                                              (.getName file)
+                                              "&start=0&length=51200")}
+                            {"byteOffset" 5159
+                             "beforeString" "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX\nThe following max-size match straddles a 1024 byte buffer.\n"
+                             "afterString" "\n\nHere are four non-ascii 1-byte UTF-8 characters: αβγδε\n\nneedle\n\nHere are four printable 2-byte UTF-8 characters: ¡¢£¤"
+                             "matchString" pattern
+                             "logviewerURL" (str "http://"
+                                              expected-host
+                                              ":"
+                                              expected-port
+                                              "/log?file=src%2Fdev%2F"
+                                              (.getName file)
+                                              "&start=0&length=51200")}
+                            ]}
+              (logviewer/substring-search file pattern :num-matches 2))))
+
+        (let [pattern "𐄀𐄁𐄂"]
+          (is
+            (= {"nextByteOffset" 7176
+                "searchString" pattern
+                "startByteOffset" 0
+                "matches" [
+                            {"byteOffset" 7164
+                             "beforeString" "padding 372\npadding 373\npadding 374\npadding 375\n\nThe following tests multibyte UTF-8 Characters straddling the byte boundary:   "
+                             "afterString" "\n\nneedle"
+                             "matchString" pattern
+                             "logviewerURL" (str "http://"
+                                              expected-host
+                                              ":"
+                                              expected-port
+                                              "/log?file=src%2Fdev%2F"
+                                              (.getName file)
+                                              "&start=0&length=51200")}
+                            ]}
+              (logviewer/substring-search file pattern :num-matches 1))))
+
+        (testing "Returns 0 matches for unseen pattern"
+          (let [pattern "Not There"]
+            (is (= {"searchString" pattern
+                    "startByteOffset" 0
+                    "matches" []}
+                  (logviewer/substring-search file
+                    pattern
+                    :num-matches nil
+                    :start-byte-offset nil)))))))))
+
+(deftest test-find-n-matches
+  (testing "find-n-matches looks through logs properly"
+    (let [files [(clojure.java.io/file "src" "dev" "logviewer-search-context-tests.log")
+                 (clojure.java.io/file "src" "dev" "logviewer-search-context-tests.log.gz")]
+          matches1 ((logviewer/find-n-matches files 20 0 0 "needle") "matches")
+          matches2 ((logviewer/find-n-matches files 20 0 126 "needle") "matches")
+          matches3 ((logviewer/find-n-matches files 20 1 0 "needle") "matches")]
+
+      (is (= 2 (count matches1)))
+      (is (= 4 (count ((first matches1) "matches"))))
+      (is (= 4 (count ((second matches1) "matches"))))
+      (is (= ((first matches1) "fileName") "src/dev/logviewer-search-context-tests.log"))
+      (is (= ((second matches1) "fileName") "src/dev/logviewer-search-context-tests.log.gz"))
+
+      (is (= 2 (count ((first matches2) "matches"))))
+      (is (= 4 (count ((second matches2) "matches"))))
+
+      (is (= 1 (count matches3)))
+      (is (= 4 (count ((first matches3) "matches")))))))
+
+(deftest test-deep-search-logs-for-topology
+  (let [files [(clojure.java.io/file "src" "dev" "logviewer-search-context-tests.log")
+               (clojure.java.io/file "src" "dev" "logviewer-search-context-tests.log.gz")]
+        attrs (make-array FileAttribute 0)
+        topo-path (.getCanonicalPath (.toFile (Files/createTempDirectory "topoA" attrs)))
+        _ (.createNewFile (clojure.java.io/file topo-path "6400"))
+        _ (.createNewFile (clojure.java.io/file topo-path "6500"))
+        _ (.createNewFile (clojure.java.io/file topo-path "6600"))
+        _ (.createNewFile (clojure.java.io/file topo-path "6700"))]
+    (stubbing
+      [logviewer/logs-for-port files
+       logviewer/find-n-matches nil]
+      (testing "deep-search-logs-for-topology all-ports search-archived = true"
+        (instrumenting
+          [logviewer/find-n-matches
+           logviewer/logs-for-port]
+          (logviewer/deep-search-logs-for-topology "" nil topo-path "search" "20" "*" "20" "199" true nil nil)
+          (verify-call-times-for logviewer/find-n-matches 4)
+          (verify-call-times-for logviewer/logs-for-port 4)
+          ; File offset and byte offset should always be zero when searching multiple workers (multiple ports).
+          (verify-nth-call-args-for 1 logviewer/find-n-matches files 20 0 0 "search")
+          (verify-nth-call-args-for 2 logviewer/find-n-matches files 20 0 0 "search")
+          (verify-nth-call-args-for 3 logviewer/find-n-matches files 20 0 0 "search")
+          (verify-nth-call-args-for 4 logviewer/find-n-matches files 20 0 0 "search")))
+      (testing "deep-search-logs-for-topology all-ports search-archived = false"
+        (instrumenting
+          [logviewer/find-n-matches
+           logviewer/logs-for-port]
+          (logviewer/deep-search-logs-for-topology "" nil topo-path "search" "20" nil "20" "199" nil nil nil)
+          (verify-call-times-for logviewer/find-n-matches 4)
+          (verify-call-times-for logviewer/logs-for-port 4)
+          ; File offset and byte offset should always be zero when searching multiple workers (multiple ports).
+          (verify-nth-call-args-for 1 logviewer/find-n-matches [(first files)] 20 0 0 "search")
+          (verify-nth-call-args-for 2 logviewer/find-n-matches [(first files)] 20 0 0 "search")
+          (verify-nth-call-args-for 3 logviewer/find-n-matches [(first files)] 20 0 0 "search")
+          (verify-nth-call-args-for 4 logviewer/find-n-matches [(first files)] 20 0 0 "search")))
+      (testing "deep-search-logs-for-topology one-port search-archived = true, no file-offset"
+        (instrumenting
+          [logviewer/find-n-matches
+           logviewer/logs-for-port]
+          (logviewer/deep-search-logs-for-topology "" nil topo-path "search" "20" "6700" "0" "0" true nil nil)
+          (verify-call-times-for logviewer/find-n-matches 1)
+          (verify-call-times-for logviewer/logs-for-port 2)
+          (verify-nth-call-args-for 1 logviewer/find-n-matches files 20 0 0 "search")))
+      (testing "deep-search-logs-for-topology one-port search-archived = true, file-offset = 1"
+        (instrumenting
+          [logviewer/find-n-matches
+           logviewer/logs-for-port]
+          (logviewer/deep-search-logs-for-topology "" nil topo-path "search" "20" "6700" "1" "0" true nil nil)
+          (verify-call-times-for logviewer/find-n-matches 1)
+          (verify-call-times-for logviewer/logs-for-port 2)
+          (verify-nth-call-args-for 1 logviewer/find-n-matches files 20 1 0 "search")))
+      (testing "deep-search-logs-for-topology one-port search-archived = false, file-offset = 1"
+        (instrumenting
+          [logviewer/find-n-matches
+           logviewer/logs-for-port]
+          (logviewer/deep-search-logs-for-topology "" nil topo-path "search" "20" "6700" "1" "0" nil nil nil)
+          (verify-call-times-for logviewer/find-n-matches 1)
+          (verify-call-times-for logviewer/logs-for-port 2)
+          ; File offset should be zero, since search-archived is false.
+          (verify-nth-call-args-for 1 logviewer/find-n-matches [(first files)] 20 0 0 "search")))
+      (testing "deep-search-logs-for-topology one-port search-archived = true, file-offset = 1, byte-offset = 100"
+        (instrumenting
+          [logviewer/find-n-matches
+           logviewer/logs-for-port]
+          (logviewer/deep-search-logs-for-topology "" nil topo-path "search" "20" "6700" "1" "100" true nil nil)
+          (verify-call-times-for logviewer/find-n-matches 1)
+          (verify-call-times-for logviewer/logs-for-port 2)
+          ; File offset should be zero, since search-archived is false.
+          (verify-nth-call-args-for 1 logviewer/find-n-matches files 20 1 100 "search")))
+      (testing "deep-search-logs-for-topology bad-port search-archived = false, file-offset = 1"
+        (instrumenting
+          [logviewer/find-n-matches
+           logviewer/logs-for-port]
+          (logviewer/deep-search-logs-for-topology "" nil topo-path "search" "20" "2700" "1" "0" nil nil nil)
+          ; Called with a bad port (not in the config) No searching should be done.
+          (verify-call-times-for logviewer/find-n-matches 0)
+          (verify-call-times-for logviewer/logs-for-port 0)))
+      (rmr topo-path))))
+
diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj
index a918a4b..8081ca5 100644
--- a/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj
+++ b/storm-core/test/clj/backtype/storm/messaging/netty_integration_test.clj
@@ -54,5 +54,4 @@
                                                          ["a"] ["b"]
                                                          ]}
                                      )]
-      (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
-               (read-tuples results "2"))))))
+        (is (= (* 6 4) (.size (read-tuples results "2")))))))
diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj
index f7d3802..1d6f104 100644
--- a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj
+++ b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj
@@ -15,7 +15,8 @@
 ;; limitations under the License.
 (ns backtype.storm.messaging.netty-unit-test
   (:use [clojure test])
-  (:import [backtype.storm.messaging TransportFactory])
+  (:import [backtype.storm.messaging TransportFactory IConnection TaskMessage IConnectionCallback])
+  (:import [backtype.storm.utils Utils])
   (:use [backtype.storm testing util config log])
   (:use [backtype.storm.daemon.worker :only [is-connection-ready]])
   (:import [java.util ArrayList]))
@@ -33,20 +34,54 @@
       (do (log-message "Waiting until all Netty connections are ready...")
           (wait-until-ready connections 0)))
   ([connections waited-ms]
-    (let [interval-ms 10
-          max-wait-ms 5000]
-      (if-not (every? is-connection-ready connections)
-        (if (<= waited-ms max-wait-ms)
-          (do
-            (Thread/sleep interval-ms)
-            (wait-until-ready connections (+ waited-ms interval-ms)))
-          (throw (RuntimeException. (str "Netty connections were not ready within " max-wait-ms " ms"))))
-        (log-message "All Netty connections are ready")))))
+   (loop [connections connections waited-ms waited-ms]
+     (let [interval-ms 10
+           max-wait-ms 5000]
+       (if-not (every? is-connection-ready connections)
+         (if (<= waited-ms max-wait-ms)
+           (do
+             (Thread/sleep interval-ms)
+             (recur connections (+ waited-ms interval-ms)))
+           (throw (RuntimeException. (str "Netty connections were not ready within " max-wait-ms " ms"))))
+         (log-message "All Netty connections are ready"))))))
+
+(defn mk-connection-callback
+  "make an IConnectionCallback"
+  [my-fn]
+  (reify IConnectionCallback
+    (recv [this batch]
+      (doseq [msg batch]
+        (my-fn msg)))))
+
+(defn register-callback
+  "register the local-transfer-fn with the server"
+  [my-fn ^IConnection socket]
+  (.registerRecv socket (mk-connection-callback my-fn)))
+
+(defn- wait-for-not-nil
+  [atm]
+  (while-timeout TEST-TIMEOUT-MS (nil? @atm) (Thread/sleep 10)))
+
+(defn- test-basic-fn [storm-conf]
+  (log-message "1. Should send and receive a basic message")
+  (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz")
+        context (TransportFactory/makeContext storm-conf)
+        port (available-port 6700)
+        resp (atom nil)
+        server (.bind context nil port)
+        _ (register-callback (fn [message] (reset! resp message)) server)
+        client (.connect context nil "localhost" port)
+        _ (wait-until-ready [server client])
+        _ (.send client task (.getBytes req_msg))]
+    (wait-for-not-nil resp)
+    (is (= task (.task @resp)))
+    (is (= req_msg (String. (.message @resp))))
+    (.close client)
+    (.close server)
+    (.term context)))
 
 (deftest test-basic
-  (log-message "Should send and receive a basic message")
-  (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz")
-        storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
+ (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
                     STORM-MESSAGING-NETTY-AUTHENTICATION false
                     STORM-MESSAGING-NETTY-BUFFER-SIZE 1024
                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
@@ -54,24 +89,80 @@
                     STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
                     STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
                     STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
-                    }
+                    TOPOLOGY-KRYO-FACTORY "backtype.storm.serialization.DefaultKryoFactory"
+                    TOPOLOGY-TUPLE-SERIALIZER "backtype.storm.serialization.types.ListDelegateSerializer"
+                    TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false
+                    TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS false}
+        storm-conf-sasl (assoc storm-conf
+                                    STORM-MESSAGING-NETTY-AUTHENTICATION true
+                                    TOPOLOGY-NAME "topo1-netty-sasl"
+                                    STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD (str (Utils/secureRandomLong) ":" (Utils/secureRandomLong)))]
+   (test-basic-fn storm-conf)          ;; test with sasl authentication disabled
+   (test-basic-fn storm-conf-sasl)))   ;; test with sasl authentication enabled
+
+(defn- test-load-fn [storm-conf]
+  (log-message "2 test load")
+  (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz")
         context (TransportFactory/makeContext storm-conf)
+        port (available-port 6700)
+        resp (atom nil)
         server (.bind context nil port)
+        _ (register-callback (fn [message] (reset! resp message)) server)
         client (.connect context nil "localhost" port)
         _ (wait-until-ready [server client])
         _ (.send client task (.getBytes req_msg))
-        iter (.recv server 0 0)
-        resp (.next iter)]
-    (is (= task (.task resp)))
-    (is (= req_msg (String. (.message resp))))
+        _ (.sendLoadMetrics server {(int 1) 0.0 (int 2) 1.0})
+        _ (while-timeout 5000 (empty? (.getLoad client [(int 1) (int 2)])) (Thread/sleep 10))
+        load (.getLoad client [(int 1) (int 2)])]
+    (is (= 0.0 (.getBoltLoad (.get load (int 1)))))
+    (is (= 1.0 (.getBoltLoad (.get load (int 2)))))
+    (wait-for-not-nil resp)
+    (is (= task (.task @resp)))
+    (is (= req_msg (String. (.message @resp))))
+    (.close client)
+    (.close server)
+    (.term context)))
+
+(deftest test-load
+ (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
+                    STORM-MESSAGING-NETTY-AUTHENTICATION false
+                    STORM-MESSAGING-NETTY-BUFFER-SIZE 1024
+                    STORM-MESSAGING-NETTY-MAX-RETRIES 10
+                    STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000
+                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
+                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                    TOPOLOGY-KRYO-FACTORY "backtype.storm.serialization.DefaultKryoFactory"
+                    TOPOLOGY-TUPLE-SERIALIZER "backtype.storm.serialization.types.ListDelegateSerializer"
+                    TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false
+                    TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS false}
+        storm-conf-sasl (assoc storm-conf
+                                    STORM-MESSAGING-NETTY-AUTHENTICATION true
+                                    TOPOLOGY-NAME "topo1-netty-sasl"
+                                    STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD (str (Utils/secureRandomLong) ":" (Utils/secureRandomLong)))]
+   (test-load-fn storm-conf)          ;; test with sasl authentication disabled
+   (test-load-fn storm-conf-sasl)))   ;; test with sasl authentication enabled
+
+(defn test-large-msg-fn [storm-conf]
+  (log-message "3 Should send and receive a large message")
+  (let [req_msg (apply str (repeat 2048000 'c'))
+        context (TransportFactory/makeContext storm-conf)
+        port (available-port 6700)
+        resp (atom nil)
+        server (.bind context nil port)
+        _ (register-callback (fn [message] (reset! resp message)) server)
+        client (.connect context nil "localhost" port)
+        _ (wait-until-ready [server client])
+        _ (.send client task (.getBytes req_msg))]
+    (wait-for-not-nil resp)
+    (is (= task (.task @resp)))
+    (is (= req_msg (String. (.message @resp))))
     (.close client)
     (.close server)
     (.term context)))
 
 (deftest test-large-msg
-  (log-message "Should send and receive a large message")
-  (let [req_msg (apply str (repeat 2048000 'c'))
-        storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
+ (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
                     STORM-MESSAGING-NETTY-AUTHENTICATION false
                     STORM-MESSAGING-NETTY-BUFFER-SIZE 102400
                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
@@ -79,24 +170,92 @@
                     STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
                     STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
                     STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
-                    }
+                    TOPOLOGY-KRYO-FACTORY "backtype.storm.serialization.DefaultKryoFactory"
+                    TOPOLOGY-TUPLE-SERIALIZER "backtype.storm.serialization.types.ListDelegateSerializer"
+                    TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false
+                    TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS false}
+        storm-conf-sasl (assoc storm-conf
+                                    STORM-MESSAGING-NETTY-AUTHENTICATION true
+                                    TOPOLOGY-NAME "topo1-netty-sasl"
+                                    STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD (str (Utils/secureRandomLong) ":" (Utils/secureRandomLong)))]
+   (test-large-msg-fn storm-conf)          ;; test with sasl authentication disabled
+   (test-large-msg-fn storm-conf-sasl)))   ;; test with sasl authentication enabled
+
+(defn- test-server-delayed-fn [storm-conf]
+  (log-message "4. test server delayed")
+  (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz")
         context (TransportFactory/makeContext storm-conf)
-        server (.bind context nil port)
+        resp (atom nil)
+        port (available-port 6700)
         client (.connect context nil "localhost" port)
-        _ (wait-until-ready [server client])
-        _ (.send client task (.getBytes req_msg))
-        iter (.recv server 0 0)
-        resp (.next iter)]
-    (is (= task (.task resp)))
-    (is (= req_msg (String. (.message resp))))
+
+        server (Thread.
+                 (fn []
+                   (Thread/sleep 100)
+                   (let [server (.bind context nil port)]
+                     (register-callback (fn [message] (reset! resp message)) server))))]
+    (.start server)
+    (wait-until-ready [server client])
+    (.send client task (.getBytes req_msg))
+
+    (wait-for-not-nil resp)
+    (is (= task (.task @resp)))
+    (is (= req_msg (String. (.message @resp))))
+
+    (.join server)
     (.close client)
-    (.close server)
     (.term context)))
 
+(deftest test-server-delayed
+ (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
+                    STORM-MESSAGING-NETTY-AUTHENTICATION false
+                    STORM-MESSAGING-NETTY-BUFFER-SIZE 1024
+                    STORM-MESSAGING-NETTY-MAX-RETRIES 10
+                    STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000
+                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
+                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                    TOPOLOGY-KRYO-FACTORY "backtype.storm.serialization.DefaultKryoFactory"
+                    TOPOLOGY-TUPLE-SERIALIZER "backtype.storm.serialization.types.ListDelegateSerializer"
+                    TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false
+                    TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS false}
+        storm-conf-sasl (assoc storm-conf
+                                    STORM-MESSAGING-NETTY-AUTHENTICATION true
+                                    TOPOLOGY-NAME "topo1-netty-sasl"
+                                    STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD (str (Utils/secureRandomLong) ":" (Utils/secureRandomLong)))]
+   (test-server-delayed-fn storm-conf)          ;; test with sasl authentication disabled
+   (test-server-delayed-fn storm-conf-sasl)))   ;; test with sasl authentication enabled
+
+
+(defn- test-batch-fn [storm-conf]
+  (log-message "5. test batch")
+  (let [num-messages 100000
+        _ (log-message "Should send and receive many messages (testing with " num-messages " messages)")
+        resp (ArrayList.)
+        received (atom 0)
+        context (TransportFactory/makeContext storm-conf)
+        port (available-port 6700)
+        server (.bind context nil port)
+        _ (register-callback (fn [message] (.add resp message) (swap! received inc)) server)
+        client (.connect context nil "localhost" port)
+        _ (wait-until-ready [server client])]
+    (doseq [num (range 1 num-messages)]
+      (let [req_msg (str num)]
+        (.send client task (.getBytes req_msg))))
+
+    (while-timeout TEST-TIMEOUT-MS (< (.size resp) (- num-messages 1)) (log-message (.size resp) " " num-messages) (Thread/sleep 10))
+
+    (doseq [num  (range 1 num-messages)]
+      (let [req_msg (str num)
+            resp_msg (String. (.message (.get resp (- num 1))))]
+        (is (= req_msg resp_msg))))
+
+    (.close client)
+    (.close server)
+    (.term context))
 
 (deftest test-batch
-  (let [num-messages 100000
-        storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
+ (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
                     STORM-MESSAGING-NETTY-AUTHENTICATION false
                     STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000
                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
@@ -104,30 +263,65 @@
                     STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
                     STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
                     STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
-                    }
-        _ (log-message "Should send and receive many messages (testing with " num-messages " messages)")
-        context (TransportFactory/makeContext storm-conf)
-        server (.bind context nil port)
-        client (.connect context nil "localhost" port)
-        _ (wait-until-ready [server client])]
-    (doseq [num  (range 1 num-messages)]
-      (let [req_msg (str num)]
-        (.send client task (.getBytes req_msg))))
+                    TOPOLOGY-KRYO-FACTORY "backtype.storm.serialization.DefaultKryoFactory"
+                    TOPOLOGY-TUPLE-SERIALIZER "backtype.storm.serialization.types.ListDelegateSerializer"
+                    TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false
+                    TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS false}
+        storm-conf-sasl (assoc storm-conf
+                                    STORM-MESSAGING-NETTY-AUTHENTICATION true
+                                    TOPOLOGY-NAME "topo1-netty-sasl"
+                                    STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD (str (Utils/secureRandomLong) ":" (Utils/secureRandomLong)))]
+   (test-batch-fn storm-conf)          ;; test with sasl authentication disabled
+   (test-batch-fn storm-conf-sasl)))   ;; test with sasl authentication enabled
+)
 
-    (let [resp (ArrayList.)
-          received (atom 0)]
-      (while (< @received (- num-messages 1))
-        (let [iter (.recv server 0 0)]
-          (while (.hasNext iter)
-            (let [msg (.next iter)]
-              (.add resp msg)
-              (swap! received inc)
-              ))))
-      (doseq [num  (range 1 num-messages)]
-      (let [req_msg (str num)
-            resp_msg (String. (.message (.get resp (- num 1))))]
-        (is (= req_msg resp_msg)))))
+(defn- test-server-always-reconnects-fn [storm-conf]
+  (log-message "6. test server always reconnects")
+    (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz")
+          storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
+                      STORM-MESSAGING-NETTY-AUTHENTICATION false
+                      STORM-MESSAGING-NETTY-BUFFER-SIZE 1024
+                      STORM-MESSAGING-NETTY-MAX-RETRIES 2
+                      STORM-MESSAGING-NETTY-MIN-SLEEP-MS 10
+                      STORM-MESSAGING-NETTY-MAX-SLEEP-MS 50
+                      STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                      STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                      TOPOLOGY-KRYO-FACTORY "backtype.storm.serialization.DefaultKryoFactory"
+                      TOPOLOGY-TUPLE-SERIALIZER "backtype.storm.serialization.types.ListDelegateSerializer"
+                      TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false
+                      TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS false}
+          resp (atom nil)
+          context (TransportFactory/makeContext storm-conf)
+          port (available-port 6700)
+          client (.connect context nil "localhost" port)
+          _ (.send client task (.getBytes req_msg))
+          server (.bind context nil port)
+          _ (register-callback (fn [message] (reset! resp message)) server)
+          _ (wait-until-ready [server client])
+          _ (.send client task (.getBytes req_msg))]
+      (wait-for-not-nil resp)
+      (is (= task (.task @resp)))
+      (is (= req_msg (String. (.message @resp))))
+      (.close client)
+      (.close server)
+      (.term context)))
 
-    (.close client)
-    (.close server)
-    (.term context)))
+(deftest test-server-always-reconnects
+ (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context"
+                    STORM-MESSAGING-NETTY-AUTHENTICATION false
+                    STORM-MESSAGING-NETTY-BUFFER-SIZE 1024
+                    STORM-MESSAGING-NETTY-MAX-RETRIES 2
+                    STORM-MESSAGING-NETTY-MIN-SLEEP-MS 10
+                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 50
+                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                    TOPOLOGY-KRYO-FACTORY "backtype.storm.serialization.DefaultKryoFactory"
+                    TOPOLOGY-TUPLE-SERIALIZER "backtype.storm.serialization.types.ListDelegateSerializer"
+                    TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false
+                    TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS false}
+        storm-conf-sasl (assoc storm-conf
+                                    STORM-MESSAGING-NETTY-AUTHENTICATION true
+                                    TOPOLOGY-NAME "topo1-netty-sasl"
+                                    STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD (str (Utils/secureRandomLong) ":" (Utils/secureRandomLong)))]
+   (test-server-always-reconnects-fn storm-conf)          ;; test with sasl authentication disabled
+   (test-server-always-reconnects-fn storm-conf-sasl)))   ;; test with sasl authentication enabled
diff --git a/storm-core/test/clj/backtype/storm/messaging_test.clj b/storm-core/test/clj/backtype/storm/messaging_test.clj
index 53cd555..86162dd 100644
--- a/storm-core/test/clj/backtype/storm/messaging_test.clj
+++ b/storm-core/test/clj/backtype/storm/messaging_test.clj
@@ -51,8 +51,7 @@
                                                            ["a"] ["b"]
                                                            ]}
                                        )]
-        (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
-                 (read-tuples results "2")))))))
+        (is (= (* 6 4) (.size (read-tuples results "2"))))))))
 
 (extend-type TestEventLogSpout
   CompletableSpout
@@ -62,28 +61,3 @@
     (.cleanup this))
   (startup [this]
     ))
-
-;; Test Adding more receiver threads won't violate the message delivery order gurantee
-(deftest test-receiver-message-order 
-  (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 2
-                                        :daemon-conf {TOPOLOGY-WORKERS 2
-                                                      ;; Configure multiple receiver threads per worker 
-                                                      WORKER-RECEIVER-THREAD-COUNT 2
-                                                      STORM-LOCAL-MODE-ZMQ  true 
-                                                      STORM-MESSAGING-TRANSPORT 
-                                                      "backtype.storm.messaging.netty.Context"}]
-      (let [topology (thrift/mk-topology
-                       
-                       ;; TestEventLogSpout output(sourceId, eventId), eventId is Monotonically increasing
-                       {"1" (thrift/mk-spout-spec (TestEventLogSpout. 4000) :parallelism-hint 8)}
-                       
-                       ;; field grouping, message from same "source" task will be delivered to same bolt task
-                       ;; When received message order is not kept, Emit an error Tuple 
-                       {"2" (thrift/mk-bolt-spec {"1" ["source"]} (TestEventOrderCheckBolt.)
-                                                 :parallelism-hint 4)
-                        })
-            results (complete-topology cluster
-                                       topology)]
-        
-        ;; No error Tuple from Bolt TestEventOrderCheckBolt
-        (is (empty? (read-tuples results "2"))))))
diff --git a/storm-core/test/clj/backtype/storm/metrics_test.clj b/storm-core/test/clj/backtype/storm/metrics_test.clj
index 90a3e0c..ab3a8ea 100644
--- a/storm-core/test/clj/backtype/storm/metrics_test.clj
+++ b/storm-core/test/clj/backtype/storm/metrics_test.clj
@@ -47,7 +47,7 @@
             ))))))
 
 (defn assert-loop [afn ids]
-  (while (not (every? afn ids))
+  (while-timeout TEST-TIMEOUT-MS (not (every? afn ids))
     (Thread/sleep 1)))
 
 (defn assert-acked [tracker & ids]
diff --git a/storm-core/test/clj/backtype/storm/multilang_test.clj b/storm-core/test/clj/backtype/storm/multilang_test.clj
index a624403..ff8f2f1 100644
--- a/storm-core/test/clj/backtype/storm/multilang_test.clj
+++ b/storm-core/test/clj/backtype/storm/multilang_test.clj
@@ -47,9 +47,9 @@
                "test"
                {TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true}
                topology)
-       (Thread/sleep 10000)
+       (Thread/sleep 11000)
        (.killTopology nimbus "test")
-       (Thread/sleep 10000)
+       (Thread/sleep 11000)
        )))
 
 
diff --git a/storm-core/test/clj/backtype/storm/nimbus_test.clj b/storm-core/test/clj/backtype/storm/nimbus_test.clj
index 0645f39..53d4bb8 100644
--- a/storm-core/test/clj/backtype/storm/nimbus_test.clj
+++ b/storm-core/test/clj/backtype/storm/nimbus_test.clj
@@ -19,12 +19,14 @@
   (:require [backtype.storm.daemon [nimbus :as nimbus]])
   (:require [backtype.storm [converter :as converter]])
   (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount
-            TestAggregatesCounter TestPlannerSpout TestPlannerBolt])
+            TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
+           [backtype.storm.nimbus InMemoryTopologyActionNotifier])
   (:import [backtype.storm.scheduler INimbus])
   (:import [backtype.storm.nimbus ILeaderElector NimbusInfo])
   (:import [backtype.storm.generated Credentials NotAliveException SubmitOptions
             TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
-            InvalidTopologyException AuthorizationException])
+            InvalidTopologyException AuthorizationException
+            LogConfig LogLevel LogLevelAction])
   (:import [java.util HashMap])
   (:import [java.io File])
   (:import [backtype.storm.utils Time])
@@ -190,7 +192,7 @@
       )))
 
 (deftest test-assignment
-  (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3
+  (with-simulated-time-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3
                        :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
           nimbus (:nimbus cluster)
@@ -205,6 +207,7 @@
                       "4" (thrift/mk-bolt-spec {"1" :global "2" :none} (TestPlannerBolt.) :parallelism-hint 4)}
                      )
           _ (submit-local-topology nimbus "mystorm" {TOPOLOGY-WORKERS 4} topology)
+          _ (advance-cluster-time cluster 11)
           task-info (storm-component->task-info cluster "mystorm")]
       (check-consistency cluster "mystorm")
       ;; 3 should be assigned once (if it were optimized, we'd have
@@ -215,6 +218,7 @@
       (is (= 1 (count (task-info "3"))))
       (is (= 4 (storm-num-workers state "mystorm")))
       (submit-local-topology nimbus "storm2" {TOPOLOGY-WORKERS 20} topology2)
+      (advance-cluster-time cluster 11)
       (check-consistency cluster "storm2")
       (is (= 2 (count (.assignments state nil))))
       (let [task-info (storm-component->task-info cluster "storm2")]
@@ -293,28 +297,28 @@
                       {"1" (thrift/mk-spout-spec (TestPlannerSpout. false) :parallelism-hint 3)}
                       {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 5)
                        "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.))}))
-          
+
       (submit-local-topology nimbus "noniso" {TOPOLOGY-WORKERS 4} topology)
-      (advance-cluster-time cluster 1)
+      (advance-cluster-time cluster 11)
       (is (= 4 (topology-num-nodes state "noniso")))
       (is (= 4 (storm-num-workers state "noniso")))
 
       (submit-local-topology nimbus "tester1" {TOPOLOGY-WORKERS 6} topology)
       (submit-local-topology nimbus "tester2" {TOPOLOGY-WORKERS 6} topology)
-      (advance-cluster-time cluster 1)
-    
+      (advance-cluster-time cluster 11)
+
       (bind task-info-tester1 (storm-component->task-info cluster "tester1"))
       (bind task-info-tester2 (storm-component->task-info cluster "tester2"))
-          
+
 
       (is (= 1 (topology-num-nodes state "noniso")))
       (is (= 3 (storm-num-workers state "noniso")))
 
       (is (= {2 3} (topology-node-distribution state "tester1")))
       (is (= {3 2} (topology-node-distribution state "tester2")))
-      
+
       (is (apply disjoint? (map (partial topology-nodes state) ["noniso" "tester1" "tester2"])))
-      
+
       (check-consistency cluster "tester1")
       (check-consistency cluster "tester2")
       (check-consistency cluster "noniso")
@@ -322,16 +326,16 @@
       ;;check that nothing gets reassigned
       (bind tester1-slots (topology-slots state "tester1"))
       (bind tester2-slots (topology-slots state "tester2"))
-      (bind noniso-slots (topology-slots state "noniso"))    
+      (bind noniso-slots (topology-slots state "noniso"))
       (advance-cluster-time cluster 20)
       (is (= tester1-slots (topology-slots state "tester1")))
       (is (= tester2-slots (topology-slots state "tester2")))
       (is (= noniso-slots (topology-slots state "noniso")))
-      
+
       )))
 
 (deftest test-zero-executor-or-tasks
-  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
+  (with-simulated-time-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
           nimbus (:nimbus cluster)
           topology (thrift/mk-topology
@@ -339,6 +343,7 @@
                     {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 1 :conf {TOPOLOGY-TASKS 2})
                      "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.) :conf {TOPOLOGY-TASKS 5})})
           _ (submit-local-topology nimbus "mystorm" {TOPOLOGY-WORKERS 4} topology)
+          _ (advance-cluster-time cluster 11)
           task-info (storm-component->task-info cluster "mystorm")]
       (check-consistency cluster "mystorm")
       (is (= 0 (count (task-info "1"))))
@@ -348,20 +353,21 @@
       )))
 
 (deftest test-executor-assignments
-  (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
+  (with-simulated-time-local-cluster[cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [nimbus (:nimbus cluster)
           topology (thrift/mk-topology
                     {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3 :conf {TOPOLOGY-TASKS 5})}
                     {"2" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 8 :conf {TOPOLOGY-TASKS 2})
                      "3" (thrift/mk-bolt-spec {"2" :none} (TestPlannerBolt.) :parallelism-hint 3)})
           _ (submit-local-topology nimbus "mystorm" {TOPOLOGY-WORKERS 4} topology)
+          _ (advance-cluster-time cluster 11)
           task-info (storm-component->task-info cluster "mystorm")
           executor-info (->> (storm-component->executor-info cluster "mystorm")
                              (map-val #(map executor-id->tasks %)))]
       (check-consistency cluster "mystorm")
       (is (= 5 (count (task-info "1"))))
       (check-distribution (executor-info "1") [2 2 1])
-      
+
       (is (= 2 (count (task-info "2"))))
       (check-distribution (executor-info "2") [1 1])
 
@@ -370,7 +376,7 @@
       )))
 
 (deftest test-over-parallelism-assignment
-  (with-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5
+  (with-simulated-time-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5
                        :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
     (let [state (:storm-cluster-state cluster)
           nimbus (:nimbus cluster)
@@ -381,6 +387,7 @@
                       "4" (thrift/mk-bolt-spec {"1" :none} (TestPlannerBolt.) :parallelism-hint 10)}
                      )
           _ (submit-local-topology nimbus "test" {TOPOLOGY-WORKERS 7} topology)
+          _ (advance-cluster-time cluster 11)
           task-info (storm-component->task-info cluster "test")]
       (check-consistency cluster "test")
       (is (= 21 (count (task-info "1"))))
@@ -408,7 +415,7 @@
       (bind state (:storm-cluster-state cluster))
       (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20} topology)
       (bind storm-id (get-storm-id state "test"))
-      (advance-cluster-time cluster 5)
+      (advance-cluster-time cluster 15)
       (is (not-nil? (.storm-base state storm-id nil)))
       (is (not-nil? (.assignment-info state storm-id nil)))
       (.killTopology (:nimbus cluster) "test")
@@ -429,23 +436,25 @@
 
       (is (thrown? NotAliveException (.killTopology (:nimbus cluster) "lalala")))
       (submit-local-topology (:nimbus cluster) "2test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10} topology)
+      (advance-cluster-time cluster 11)
       (is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology)))
+      (advance-cluster-time cluster 11)
       (bind storm-id (get-storm-id state "2test"))
       (is (not-nil? (.storm-base state storm-id nil)))
       (.killTopology (:nimbus cluster) "2test")
       (is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology)))
-      (advance-cluster-time cluster 5)
+      (advance-cluster-time cluster 11)
       (is (= 1 (count (.heartbeat-storms state))))
-      
+
       (advance-cluster-time cluster 6)
       (is (nil? (.storm-base state storm-id nil)))
       (is (nil? (.assignment-info state storm-id nil)))
       (advance-cluster-time cluster 11)
       (is (= 0 (count (.heartbeat-storms state))))
-      
+
       (submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology)
       (bind storm-id3 (get-storm-id state "test3"))
-      (advance-cluster-time cluster 1)
+      (advance-cluster-time cluster 11)
       (.remove-storm! state storm-id3)
       (is (nil? (.storm-base state storm-id3 nil)))
       (is (nil? (.assignment-info state storm-id3 nil)))
@@ -456,12 +465,13 @@
       ;; this guarantees that monitor thread won't trigger for 10 more seconds
       (advance-time-secs! 11)
       (wait-until-cluster-waiting cluster)
-      
+
       (submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology)
       (bind storm-id3 (get-storm-id state "test3"))
 
+      (advance-cluster-time cluster 11)
       (bind executor-id (first (topology-executors cluster storm-id3)))
-      
+
       (do-executor-heartbeat cluster storm-id3 executor-id)
 
       (.killTopology (:nimbus cluster) "test3")
@@ -472,12 +482,13 @@
 
       ;; test kill with opts
       (submit-local-topology (:nimbus cluster) "test4" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 100} topology)
+      (advance-cluster-time cluster 11)
       (.killTopologyWithOpts (:nimbus cluster) "test4" (doto (KillOptions.) (.set_wait_secs 10)))
       (bind storm-id4 (get-storm-id state "test4"))
       (advance-cluster-time cluster 9)
       (is (not-nil? (.assignment-info state storm-id4 nil)))
       (advance-cluster-time cluster 2)
-      (is (nil? (.assignment-info state storm-id4 nil)))      
+      (is (nil? (.assignment-info state storm-id4 nil)))
       )))
 
 (deftest test-reassignment
@@ -497,13 +508,14 @@
                        ))
       (bind state (:storm-cluster-state cluster))
       (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 2} topology)
+      (advance-cluster-time cluster 11)
       (check-consistency cluster "test")
       (bind storm-id (get-storm-id state "test"))
       (bind [executor-id1 executor-id2]  (topology-executors cluster storm-id))
       (bind ass1 (executor-assignment cluster storm-id executor-id1))
       (bind ass2 (executor-assignment cluster storm-id executor-id2))
 
-      (advance-cluster-time cluster 59)
+      (advance-cluster-time cluster 30)
       (do-executor-heartbeat cluster storm-id executor-id1)
       (do-executor-heartbeat cluster storm-id executor-id2)
 
@@ -523,7 +535,7 @@
       (do-executor-heartbeat cluster storm-id executor-id1)
       (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
       (check-consistency cluster "test")
-      
+
       (advance-cluster-time cluster 11)
       (is (= ass1 (executor-assignment cluster storm-id executor-id1)))
       (is (not= ass2 (executor-assignment cluster storm-id executor-id2)))
@@ -595,13 +607,14 @@
                        ))
       (bind state (:storm-cluster-state cluster))
       (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 2} topology)
+      (advance-cluster-time cluster 11)
       (check-consistency cluster "test")
       (bind storm-id (get-storm-id state "test"))
       (bind [executor-id1 executor-id2]  (topology-executors cluster storm-id))
       (bind ass1 (executor-assignment cluster storm-id executor-id1))
       (bind ass2 (executor-assignment cluster storm-id executor-id2))
-      
-      (advance-cluster-time cluster 59)
+
+      (advance-cluster-time cluster 30)
       (do-executor-heartbeat cluster storm-id executor-id1)
       (do-executor-heartbeat cluster storm-id executor-id2)
 
@@ -649,6 +662,7 @@
                         {}))
       (bind state (:storm-cluster-state cluster))
       (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 4} topology)  ; distribution should be 2, 2, 2, 3 ideally
+      (advance-cluster-time cluster 11)
       (bind storm-id (get-storm-id state "test"))
       (bind slot-executors (slot-assignments cluster storm-id))
       (check-executor-distribution slot-executors [9])
@@ -673,7 +687,7 @@
       (bind common (first (find-first (fn [[k v]] (= 3 (count v))) slot-executors2)))
       (is (not-nil? common))
       (is (= (slot-executors2 common) (slot-executors common)))
-      
+
       ;; check that start times are changed for everything but the common one
       (bind same-executors (slot-executors2 common))
       (bind changed-executors (apply concat (vals (dissoc slot-executors2 common))))
@@ -699,11 +713,12 @@
                              "test"
                              {TOPOLOGY-WORKERS 3
                               TOPOLOGY-MESSAGE-TIMEOUT-SECS 60} topology)
+      (advance-cluster-time cluster 11)
       (bind storm-id (get-storm-id state "test"))
       (add-supervisor cluster :ports 3)
       (add-supervisor cluster :ports 3)
 
-      (advance-cluster-time cluster 91)
+      (advance-cluster-time cluster 11)
 
       (bind slot-executors (slot-assignments cluster storm-id))
       ;; check that all workers are on one machine
@@ -711,7 +726,7 @@
       (check-num-nodes slot-executors 1)
       (.rebalance (:nimbus cluster) "test" (RebalanceOptions.))
 
-      (advance-cluster-time cluster 31)
+      (advance-cluster-time cluster 30)
       (check-executor-distribution slot-executors [1 1 1])
       (check-num-nodes slot-executors 1)
 
@@ -745,6 +760,7 @@
                              "test"
                              {TOPOLOGY-WORKERS 3
                               TOPOLOGY-MESSAGE-TIMEOUT-SECS 30} topology)
+      (advance-cluster-time cluster 11)
       (bind storm-id (get-storm-id state "test"))
       (bind checker (fn [distribution]
                       (check-executor-distribution
@@ -835,7 +851,7 @@
                              {TOPOLOGY-WORKERS 3
                               TOPOLOGY-MESSAGE-TIMEOUT-SECS 90} topology3)
 
-      (advance-cluster-time cluster 31)
+      (advance-cluster-time cluster 11)
 
       (check-for-collisions state)
       (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.)
@@ -876,7 +892,7 @@
       (is (thrown? InvalidTopologyException
                    (submit-local-topology (:nimbus cluster)
                                           "test"
-                                          {TOPOLOGY-WORKERS 3} 
+                                          {TOPOLOGY-WORKERS 3}
                                           topology)))
       (bind topology (thrift/mk-topology
                       {"1" (thrift/mk-spout-spec (TestPlannerSpout. true)
@@ -1023,6 +1039,9 @@
 
               ;first we verify that the master nimbus can perform all actions, even with another nimbus present.
               (submit-local-topology nimbus "t1" {} topology)
+              ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+              (.rebalance nimbus "t1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+              (Thread/sleep 1000)
               (.deactivate nimbus "t1")
               (.activate nimbus "t1")
               (.rebalance nimbus "t1" (RebalanceOptions.))
@@ -1054,15 +1073,15 @@
           (.disconnect cluster-state))))))
 
 (deftest test-nimbus-iface-submitTopologyWithOpts-checks-authorization
-  (with-local-cluster [cluster 
-                       :daemon-conf {NIMBUS-AUTHORIZER 
+  (with-local-cluster [cluster
+                       :daemon-conf {NIMBUS-AUTHORIZER
                           "backtype.storm.security.auth.authorizer.DenyAuthorizer"}]
     (let [
           nimbus (:nimbus cluster)
           topology (thrift/mk-topology {} {})
          ]
       (is (thrown? AuthorizationException
-          (submit-local-topology-with-opts nimbus "mystorm" {} topology 
+          (submit-local-topology-with-opts nimbus "mystorm" {} topology
             (SubmitOptions. TopologyInitialStatus/INACTIVE))
         ))
     )
@@ -1070,8 +1089,8 @@
 )
 
 (deftest test-nimbus-iface-methods-check-authorization
-  (with-local-cluster [cluster 
-                       :daemon-conf {NIMBUS-AUTHORIZER 
+  (with-local-cluster [cluster
+                       :daemon-conf {NIMBUS-AUTHORIZER
                           "backtype.storm.security.auth.authorizer.DenyAuthorizer"}]
     (let [
           nimbus (:nimbus cluster)
@@ -1079,7 +1098,7 @@
          ]
       ; Fake good authorization as part of setup.
       (mocking [nimbus/check-authorization!]
-          (submit-local-topology-with-opts nimbus "test" {} topology 
+          (submit-local-topology-with-opts nimbus "test" {} topology
               (SubmitOptions. TopologyInitialStatus/INACTIVE))
       )
       (stubbing [nimbus/storm-active? true]
@@ -1214,16 +1233,16 @@
       (stubbing [topology-bases bogus-bases]
         (let [topos (.get_topologies (.getClusterInfo nimbus))]
           ; The number of topologies in the summary is correct.
-          (is (= (count 
+          (is (= (count
             (filter (fn [b] (second b)) bogus-bases)) (count topos)))
           ; Each topology present has a valid name.
           (is (empty?
             (filter (fn [t] (or (nil? t) (nil? (.get_name t)))) topos)))
           ; The topologies are those with valid bases.
           (is (empty?
-            (filter (fn [t] 
-              (or 
-                (nil? t) 
+            (filter (fn [t]
+              (or
+                (nil? t)
                 (not (number? (read-string (.get_id t))))
                 (odd? (read-string (.get_id t)))
               )) topos)))
@@ -1249,7 +1268,8 @@
                      NIMBUS-THRIFT-PORT 6666}
           expected-acls nimbus/NIMBUS-ZK-ACLS
           fake-inimbus (reify INimbus (getForcedScheduler [this] nil))]
-      (stubbing [mk-authorization-handler nil
+      (stubbing [nimbus-topo-history-state nil
+                 mk-authorization-handler nil
                  cluster/mk-storm-cluster-state nil
                  nimbus/file-cache-map nil
                  uptime-computer nil
@@ -1312,6 +1332,44 @@
         (.disconnect cluster-state)
         ))))
 
+(deftest test-topology-action-notifier
+  (with-inprocess-zookeeper zk-port
+    (with-local-tmp [nimbus-dir]
+      (stubbing [zk-leader-elector (mock-leader-elector)]
+        (letlocals
+          (bind conf (merge (read-storm-config)
+                       {STORM-ZOOKEEPER-SERVERS ["localhost"]
+                        STORM-CLUSTER-MODE "local"
+                        STORM-ZOOKEEPER-PORT zk-port
+                        STORM-LOCAL-DIR nimbus-dir
+                        NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
+          (bind cluster-state (cluster/mk-storm-cluster-state conf))
+          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
+          (bind notifier (InMemoryTopologyActionNotifier.))
+          (sleep-secs 1)
+          (bind topology (thrift/mk-topology
+                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                           {}))
+          (submit-local-topology nimbus "test-notification" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 30} topology)
+
+          (.deactivate nimbus "test-notification")
+
+          (.activate nimbus "test-notification")
+
+          (.rebalance nimbus "test-notification" (doto (RebalanceOptions.)
+                                                   (.set_wait_secs 0)))
+
+          (.killTopologyWithOpts nimbus "test-notification" (doto (KillOptions.)
+                                                      (.set_wait_secs 0)))
+
+          (.shutdown nimbus)
+
+          ; ensure notifier was invoked for each action,and in the correct order.
+          (is (= ["submitTopology", "activate", "deactivate", "activate", "rebalance", "killTopology"]
+                (.getTopologyActions notifier "test-notification")))
+          (.disconnect cluster-state)
+          )))))
+
 (deftest test-debug-on-component
   (with-local-cluster [cluster]
     (let [nimbus (:nimbus cluster)
@@ -1329,3 +1387,56 @@
                      {})]
       (submit-local-topology nimbus "t1" {TOPOLOGY-WORKERS 1} topology)
       (.debug nimbus "t1" "" true 100))))
+
+;; if the user sends an empty log config, nimbus will say that all 
+;; log configs it contains are LogLevelAction/UNCHANGED
+(deftest empty-save-config-results-in-all-unchanged-actions
+  (with-local-cluster [cluster]
+    (let [nimbus (:nimbus cluster)
+          previous-config (LogConfig.)
+          level (LogLevel.)
+          mock-config (LogConfig.)]
+      ;; send something with content to nimbus beforehand
+      (.set_target_log_level level "ERROR")
+      (.set_action level LogLevelAction/UPDATE)
+      (.put_to_named_logger_level previous-config "test" level)
+      (stubbing [nimbus/check-storm-active! nil
+                 nimbus/try-read-storm-conf {}]
+        (.setLogConfig nimbus "foo" previous-config)
+        (.setLogConfig nimbus "foo" mock-config)
+        (let [saved-config (.getLogConfig nimbus "foo")
+              levels (.get_named_logger_level saved-config)]
+           (is (= (.get_action (.get levels "test")) LogLevelAction/UNCHANGED)))))))
+
+(deftest log-level-update-merges-and-flags-existent-log-level
+  (with-local-cluster [cluster]
+    (stubbing [nimbus/check-storm-active! nil
+               nimbus/try-read-storm-conf {}]
+      (let [nimbus (:nimbus cluster)
+            previous-config (LogConfig.)
+            level (LogLevel.)
+            other-level (LogLevel.)
+            mock-config (LogConfig.)]
+        ;; send something with content to nimbus beforehand
+        (.set_target_log_level level "ERROR")
+        (.set_action level LogLevelAction/UPDATE)
+        (.put_to_named_logger_level previous-config "test" level)
+
+        (.set_target_log_level other-level "DEBUG")
+        (.set_action other-level LogLevelAction/UPDATE)
+        (.put_to_named_logger_level previous-config "other-test" other-level)
+        (.setLogConfig nimbus "foo" previous-config)
+
+        ;; only change "test"
+        (.set_target_log_level level "INFO")
+        (.set_action level LogLevelAction/UPDATE)
+        (.put_to_named_logger_level mock-config "test" level)
+        (.setLogConfig nimbus "foo" mock-config)
+
+        (let [saved-config (.getLogConfig nimbus "foo")
+              levels (.get_named_logger_level saved-config)]
+           (is (= (.get_action (.get levels "test")) LogLevelAction/UPDATE))
+           (is (= (.get_target_log_level (.get levels "test")) "INFO"))
+
+           (is (= (.get_action (.get levels "other-test")) LogLevelAction/UNCHANGED))
+           (is (= (.get_target_log_level (.get levels "other-test")) "DEBUG")))))))
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.)]
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..aec297d
--- /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 (= "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 (= "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 (= "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 (= "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 (= "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 (= "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 (= "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 (= "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 (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "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 (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")))
+        (is (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")))
+        (is (= "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 (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology1")) 1 0)
+                scheduled-topos (+ scheduled-topos (if (= "Fully Scheduled" (.get (.getStatusMap cluster) "topology2")) 1 0))
+                scheduled-topos (+ scheduled-topos (if (= "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 (= "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") "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")  "Unsuccessful in scheduling")))
+    
+    (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)))
+       
+  )))
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
diff --git a/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj
index e7b44cf..00c1d76 100644
--- a/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj
+++ b/storm-core/test/clj/backtype/storm/security/auth/DefaultHttpCredentialsPlugin_test.clj
@@ -48,22 +48,28 @@
         (is (.equals exp-name (.getUserName handler req)))))
 
     (testing "returns doAsUser from requests principal when Header has doAsUser param set"
-      (let [exp-name "Alice"
-            do-as-user-name "Bob"
-            princ (SingleUserPrincipal. exp-name)
-            req (Mockito/mock HttpServletRequest)
-            _ (. (Mockito/when (. req getUserPrincipal))
-              thenReturn princ)
-            _ (. (Mockito/when (. req getHeader "doAsUser"))
-              thenReturn do-as-user-name)
-            context (.populateContext handler (ReqContext/context) req)]
-        (is (= true (.isImpersonating context)))
-        (is (.equals exp-name (.getName (.realPrincipal context))))
-        (is (.equals do-as-user-name (.getName (.principal context))))))))
+      (try
+        (let [exp-name "Alice"
+              do-as-user-name "Bob"
+              princ (SingleUserPrincipal. exp-name)
+              req (Mockito/mock HttpServletRequest)
+              _ (. (Mockito/when (. req getUserPrincipal))
+                thenReturn princ)
+              _ (. (Mockito/when (. req getHeader "doAsUser"))
+                thenReturn do-as-user-name)
+              context (.populateContext handler (ReqContext/context) req)]
+          (is (= true (.isImpersonating context)))
+          (is (.equals exp-name (.getName (.realPrincipal context))))
+          (is (.equals do-as-user-name (.getName (.principal context)))))
+        (finally
+          (ReqContext/reset))))))
 
 (deftest test-populate-req-context-on-null-user
-  (let [req (Mockito/mock HttpServletRequest)
-        handler (doto (DefaultHttpCredentialsPlugin.) (.prepare {}))
-        subj (Subject. false (set [(SingleUserPrincipal. "test")]) (set []) (set []))
-        context (ReqContext. subj)]
-    (is (= 0 (-> handler (.populateContext context req) (.subject) (.getPrincipals) (.size))))))
+  (try
+    (let [req (Mockito/mock HttpServletRequest)
+          handler (doto (DefaultHttpCredentialsPlugin.) (.prepare {}))
+          subj (Subject. false (set [(SingleUserPrincipal. "test")]) (set []) (set []))
+          context (ReqContext. subj)]
+      (is (= 0 (-> handler (.populateContext context req) (.subject) (.getPrincipals) (.size)))))
+    (finally
+      (ReqContext/reset))))
diff --git a/storm-core/test/clj/backtype/storm/serialization_test.clj b/storm-core/test/clj/backtype/storm/serialization_test.clj
index 7f1c0a9..68a710c 100644
--- a/storm-core/test/clj/backtype/storm/serialization_test.clj
+++ b/storm-core/test/clj/backtype/storm/serialization_test.clj
@@ -18,10 +18,8 @@
   (:import [backtype.storm.serialization KryoTupleSerializer KryoTupleDeserializer
             KryoValuesSerializer KryoValuesDeserializer])
   (:import [backtype.storm.testing TestSerObject TestKryoDecorator])
-  (:import [backtype.storm ConfigValidation])
-  (:use [backtype.storm util config])
-  )
-
+  (:import [backtype.storm.validation ConfigValidation$KryoRegValidator])
+  (:use [backtype.storm util config]))
 
 (defn mk-conf [extra]
   (merge (read-default-config) extra))
@@ -42,19 +40,19 @@
     (deserialize (serialize vals conf) conf)))
 
 (deftest validate-kryo-conf-basic
-  (.validateField ConfigValidation/KryoRegValidator "test" ["a" "b" "c" {"d" "e"} {"f" "g"}]))
+  (.validateField (ConfigValidation$KryoRegValidator. ) "test" ["a" "b" "c" {"d" "e"} {"f" "g"}]))
 
 (deftest validate-kryo-conf-fail
   (try
-    (.validateField ConfigValidation/KryoRegValidator "test" {"f" "g"})
+    (.validateField (ConfigValidation$KryoRegValidator. ) "test" {"f" "g"})
     (assert false)
     (catch IllegalArgumentException e))
   (try
-    (.validateField ConfigValidation/KryoRegValidator "test" [1])
+    (.validateField (ConfigValidation$KryoRegValidator. ) "test" [1])
     (assert false)
     (catch IllegalArgumentException e))
   (try
-    (.validateField ConfigValidation/KryoRegValidator "test" [{"a" 1}])
+    (.validateField (ConfigValidation$KryoRegValidator. ) "test" [{"a" 1}])
     (assert false)
     (catch IllegalArgumentException e))
 )
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index b17ea5e..04c8600 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -23,6 +23,9 @@
   (:import [backtype.storm.scheduler ISupervisor])
   (:import [backtype.storm.generated RebalanceOptions])
   (:import [java.util UUID])
+  (:import [java.io File])
+  (:import [java.nio.file Files])
+  (:import [java.nio.file.attribute FileAttribute])
   (:use [backtype.storm config testing util timer])
   (:use [backtype.storm.daemon common])
   (:require [backtype.storm.daemon [worker :as worker] [supervisor :as supervisor]]
@@ -75,7 +78,7 @@
 
 (deftest launches-assignment
   (with-simulated-time-local-cluster [cluster :supervisors 0
-    :daemon-conf {NIMBUS-REASSIGN false
+    :daemon-conf {NIMBUS-DO-NOT-REASSIGN true
                   SUPERVISOR-WORKER-START-TIMEOUT-SECS 5
                   SUPERVISOR-WORKER-TIMEOUT-SECS 15
                   SUPERVISOR-MONITOR-FREQUENCY-SECS 3}]
@@ -85,23 +88,29 @@
                        {}))
       (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
       (bind changed (capture-changed-workers
-                        (submit-mocked-assignment
-                          (:nimbus cluster)
-                          "test"
-                          {TOPOLOGY-WORKERS 3}
-                          topology
-                          {1 "1"
-                           2 "1"
-                           3 "1"
-                           4 "1"}
-                          {[1 1] ["sup1" 1]
-                           [2 2] ["sup1" 2]
-                           [3 3] ["sup1" 3]
-                           [4 4] ["sup1" 3]
-                           })
-                        (advance-cluster-time cluster 2)
-                        (heartbeat-workers cluster "sup1" [1 2 3])
-                        (advance-cluster-time cluster 10)))
+                      (submit-mocked-assignment
+                        (:nimbus cluster)
+                        (:storm-cluster-state cluster)
+                        "test"
+                        {TOPOLOGY-WORKERS 3}
+                        topology
+                        {1 "1"
+                         2 "1"
+                         3 "1"
+                         4 "1"}
+                        {[1 1] ["sup1" 1]
+                         [2 2] ["sup1" 2]
+                         [3 3] ["sup1" 3]
+                         [4 4] ["sup1" 3]}
+                        {["sup1" 1] [0.0 0.0 0.0]
+                         ["sup1" 2] [0.0 0.0 0.0]
+                         ["sup1" 3] [0.0 0.0 0.0]
+                         })
+                      ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+                      (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+                      (advance-cluster-time cluster 2)
+                      (heartbeat-workers cluster "sup1" [1 2 3])
+                      (advance-cluster-time cluster 10)))
       (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test"))
       (is (empty? (:shutdown changed)))
       (validate-launched-once (:launched changed) {"sup1" [1 2 3]} storm-id)
@@ -122,7 +131,7 @@
 
 (deftest test-multiple-active-storms-multiple-supervisors
   (with-simulated-time-local-cluster [cluster :supervisors 0
-    :daemon-conf {NIMBUS-REASSIGN false
+    :daemon-conf {NIMBUS-DO-NOT-REASSIGN true
                   SUPERVISOR-WORKER-START-TIMEOUT-SECS 5
                   SUPERVISOR-WORKER-TIMEOUT-SECS 15
                   SUPERVISOR-MONITOR-FREQUENCY-SECS 3}]
@@ -136,44 +145,55 @@
       (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
       (bind sup2 (add-supervisor cluster :id "sup2" :ports [1 2]))
       (bind changed (capture-changed-workers
-                        (submit-mocked-assignment
-                          (:nimbus cluster)
-                          "test"
-                          {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 40}
-                          topology
-                          {1 "1"
-                           2 "1"
-                           3 "1"
-                           4 "1"}
-                          {[1 1] ["sup1" 1]
-                           [2 2] ["sup1" 2]
-                           [3 3] ["sup2" 1]
-                           [4 4] ["sup2" 1]
-                           })
-                        (advance-cluster-time cluster 2)
-                        (heartbeat-workers cluster "sup1" [1 2])
-                        (heartbeat-workers cluster "sup2" [1])
-                        ))
+                      (submit-mocked-assignment
+                        (:nimbus cluster)
+                        (:storm-cluster-state cluster)
+                        "test"
+                        {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 40}
+                        topology
+                        {1 "1"
+                         2 "1"
+                         3 "1"
+                         4 "1"}
+                        {[1 1] ["sup1" 1]
+                         [2 2] ["sup1" 2]
+                         [3 3] ["sup2" 1]
+                         [4 4] ["sup2" 1]}
+                        {["sup1" 1] [0.0 0.0 0.0]
+                         ["sup1" 2] [0.0 0.0 0.0]
+                         ["sup2" 1] [0.0 0.0 0.0]
+                         })
+                      ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+                      (.rebalance (:nimbus cluster) "test" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+                      (advance-cluster-time cluster 2)
+                      (heartbeat-workers cluster "sup1" [1 2])
+                      (heartbeat-workers cluster "sup2" [1])
+                      ))
       (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test"))
       (is (empty? (:shutdown changed)))
       (validate-launched-once (:launched changed) {"sup1" [1 2] "sup2" [1]} storm-id)
       (bind changed (capture-changed-workers
-                        (submit-mocked-assignment
-                          (:nimbus cluster)
-                          "test2"
-                          {TOPOLOGY-WORKERS 2}
-                          topology2
-                          {1 "1"
-                           2 "1"
-                           3 "1"}
-                          {[1 1] ["sup1" 3]
-                           [2 2] ["sup1" 3]
-                           [3 3] ["sup2" 2]
-                           })
-                        (advance-cluster-time cluster 2)
-                        (heartbeat-workers cluster "sup1" [3])
-                        (heartbeat-workers cluster "sup2" [2])
-                        ))
+                      (submit-mocked-assignment
+                        (:nimbus cluster)
+                        (:storm-cluster-state cluster)
+                        "test2"
+                        {TOPOLOGY-WORKERS 2}
+                        topology2
+                        {1 "1"
+                         2 "1"
+                         3 "1"}
+                        {[1 1] ["sup1" 3]
+                         [2 2] ["sup1" 3]
+                         [3 3] ["sup2" 2]}
+                        {["sup1" 3] [0.0 0.0 0.0]
+                         ["sup2" 2] [0.0 0.0 0.0]
+                         })
+                      ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+                      (.rebalance (:nimbus cluster) "test2" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+                      (advance-cluster-time cluster 2)
+                      (heartbeat-workers cluster "sup1" [3])
+                      (heartbeat-workers cluster "sup2" [2])
+                      ))
       (bind storm-id2 (get-storm-id (:storm-cluster-state cluster) "test2"))
       (is (empty? (:shutdown changed)))
       (validate-launched-once (:launched changed) {"sup1" [3] "sup2" [2]} storm-id2)
@@ -251,30 +271,35 @@
     (let [mock-port "42"
           mock-storm-id "fake-storm-id"
           mock-worker-id "fake-worker-id"
+          mock-mem-onheap 512
           mock-cp (str file-path-separator "base" class-path-separator file-path-separator "stormjar.jar")
           mock-sensitivity "S3"
           mock-cp "/base:/stormjar.jar"
           exp-args-fn (fn [opts topo-opts classpath]
-                       (concat [(supervisor/java-cmd) "-cp" classpath 
-                               (str "-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log")
+                       (concat [(supervisor/java-cmd) "-cp" classpath
+                               (str "-Dlogfile.name=" "worker.log")
                                "-Dstorm.home="
-                                (str "-Dstorm.id=" mock-storm-id)
-                                (str "-Dworker.id=" mock-worker-id)
-                                (str "-Dworker.port=" mock-port)
+                               (str "-Dworkers.artifacts=" "/tmp/workers-artifacts")
+                               (str "-Dstorm.id=" mock-storm-id)
+                               (str "-Dworker.id=" mock-worker-id)
+                               (str "-Dworker.port=" mock-port)
                                "-Dstorm.log.dir=/logs"
                                "-Dlog4j.configurationFile=/log4j2/worker.xml"
+                               "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector"
                                "backtype.storm.LogWriter"]
                                [(supervisor/java-cmd) "-server"]
                                opts
                                topo-opts
                                ["-Djava.library.path="
-                                (str "-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log")
+                                (str "-Dlogfile.name=" "worker.log")
                                 "-Dstorm.home="
+                                "-Dworkers.artifacts=/tmp/workers-artifacts"
                                 "-Dstorm.conf.file="
                                 "-Dstorm.options="
                                 (str "-Dstorm.log.dir=" file-path-separator "logs")
                                 (str "-Dlogging.sensitivity=" mock-sensitivity)
                                 (str "-Dlog4j.configurationFile=" file-path-separator "log4j2" file-path-separator "worker.xml")
+                                "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector"
                                 (str "-Dstorm.id=" mock-storm-id)
                                 (str "-Dworker.id=" mock-worker-id)
                                 (str "-Dworker.port=" mock-port)
@@ -298,11 +323,13 @@
                      launch-process nil
                      set-worker-user! nil
                      supervisor/jlp nil
+                     worker-artifacts-root "/tmp/workers-artifacts"
                      supervisor/write-log-metadata! nil]
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
-                                      mock-worker-id)
+                                      mock-worker-id
+                                      mock-mem-onheap)
             (verify-first-call-args-for-indices launch-process
                                                 [0]
                                                 exp-args))))
@@ -319,11 +346,13 @@
                      launch-process nil
                      set-worker-user! nil
                      supervisor/jlp nil
+                     worker-artifacts-root "/tmp/workers-artifacts"
                      supervisor/write-log-metadata! nil]
             (supervisor/launch-worker mock-supervisor
                                       mock-storm-id
                                       mock-port
-                                      mock-worker-id)
+                                      mock-worker-id
+                                      mock-mem-onheap)
             (verify-first-call-args-for-indices launch-process
                                                 [0]
                                                 exp-args))))
@@ -334,6 +363,7 @@
           (stubbing [read-supervisor-storm-conf {TOPOLOGY-CLASSPATH topo-cp}
                      supervisor-stormdist-root nil
                      supervisor/jlp nil
+                     worker-artifacts-root "/tmp/workers-artifacts"
                      set-worker-user! nil
                      supervisor/write-log-metadata! nil
                      launch-process nil
@@ -341,7 +371,8 @@
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
-                                              mock-worker-id)
+                                              mock-worker-id
+                                              mock-mem-onheap)
                     (verify-first-call-args-for-indices launch-process
                                                         [0]
                                                         exp-args))))
@@ -353,6 +384,7 @@
           (stubbing [read-supervisor-storm-conf {TOPOLOGY-ENVIRONMENT topo-env}
                      supervisor-stormdist-root nil
                      supervisor/jlp nil
+                     worker-artifacts-root "/tmp/workers-artifacts"
                      launch-process nil
                      set-worker-user! nil
                      supervisor/write-log-metadata! nil
@@ -360,25 +392,22 @@
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
-                                              mock-worker-id)
+                                              mock-worker-id
+                                              mock-mem-onheap)
                     (verify-first-call-args-for-indices launch-process
                                                         [2]
                                                         full-env)))))))
 
-(defn rm-r [f]
-  (if (.isDirectory f)
-    (for [sub (.listFiles f)] (rm-r sub))
-    (.delete f)
-  ))
-
 (deftest test-worker-launch-command-run-as-user
   (testing "*.worker.childopts configuration"
     (let [mock-port "42"
           mock-storm-id "fake-storm-id"
           mock-worker-id "fake-worker-id"
+          mock-mem-onheap 512
           mock-sensitivity "S3"
           mock-cp "mock-classpath'quote-on-purpose"
-          storm-local (str "/tmp/" (UUID/randomUUID))
+          attrs (make-array FileAttribute 0)
+          storm-local (.getCanonicalPath (.toFile (Files/createTempDirectory "storm-local" attrs)))
           worker-script (str storm-local "/workers/" mock-worker-id "/storm-worker-script.sh")
           exp-launch ["/bin/worker-launcher"
                       "me"
@@ -386,92 +415,100 @@
                       (str storm-local "/workers/" mock-worker-id)
                       worker-script]
           exp-script-fn (fn [opts topo-opts]
-                       (str "#!/bin/bash\n'export' 'LD_LIBRARY_PATH=';\n\nexec 'java'"
-                                " '-cp' 'mock-classpath'\"'\"'quote-on-purpose'"
-                                " '-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log'"
-                                " '-Dstorm.home='"
-                                " '-Dstorm.id=" mock-storm-id "'"
-                                " '-Dworker.id=" mock-worker-id "'"
-                                " '-Dworker.port=" mock-port "'"
-                                " '-Dstorm.log.dir=/logs'"
-                                " '-Dlog4j.configurationFile=/log4j2/worker.xml'"
-                                " 'backtype.storm.LogWriter'"
-                                " 'java' '-server'"
-                                " " (shell-cmd opts)
-                                " " (shell-cmd topo-opts)
-                                " '-Djava.library.path='"
-                                " '-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log'"
-                                " '-Dstorm.home='"
-                                " '-Dstorm.conf.file='"
-                                " '-Dstorm.options='"
-                                " '-Dstorm.log.dir=/logs'"
-                                " '-Dlogging.sensitivity=" mock-sensitivity "'"
-                                " '-Dlog4j.configurationFile=/log4j2/worker.xml'"
-                                " '-Dstorm.id=" mock-storm-id "'"
-                                " '-Dworker.id=" mock-worker-id "'"
-                                " '-Dworker.port=" mock-port "'"
-                                " '-cp' 'mock-classpath'\"'\"'quote-on-purpose'"
-                                " 'backtype.storm.daemon.worker'"
-                                " '" mock-storm-id "'"
-                                " '" mock-port "'"
-                                " '" mock-worker-id "';"))]
+                          (str "#!/bin/bash\n'export' 'LD_LIBRARY_PATH=';\n\nexec 'java'"
+                               " '-cp' 'mock-classpath'\"'\"'quote-on-purpose'"
+                               " '-Dlogfile.name=" "worker.log'"
+                               " '-Dstorm.home='"
+                               " '-Dworkers.artifacts=" (str storm-local "/workers-artifacts'")
+                               " '-Dstorm.id=" mock-storm-id "'"
+                               " '-Dworker.id=" mock-worker-id "'"
+                               " '-Dworker.port=" mock-port "'"
+                               " '-Dstorm.log.dir=/logs'"
+                               " '-Dlog4j.configurationFile=/log4j2/worker.xml'"
+                               " '-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector'"
+                               " 'backtype.storm.LogWriter'"
+                               " 'java' '-server'"
+                               " " (shell-cmd opts)
+                               " " (shell-cmd topo-opts)
+                               " '-Djava.library.path='"
+                               " '-Dlogfile.name=" "worker.log'"
+                               " '-Dstorm.home='"
+                               " '-Dworkers.artifacts=" (str storm-local "/workers-artifacts'")
+                               " '-Dstorm.conf.file='"
+                               " '-Dstorm.options='"
+                               " '-Dstorm.log.dir=/logs'"
+                               " '-Dlogging.sensitivity=" mock-sensitivity "'"
+                               " '-Dlog4j.configurationFile=/log4j2/worker.xml'"
+                               " '-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector'"
+                               " '-Dstorm.id=" mock-storm-id "'"
+                               " '-Dworker.id=" mock-worker-id "'"
+                               " '-Dworker.port=" mock-port "'"
+                               " '-cp' 'mock-classpath'\"'\"'quote-on-purpose'"
+                               " 'backtype.storm.daemon.worker'"
+                               " '" mock-storm-id "'"
+                               " '" mock-port "'"
+                               " '" mock-worker-id "';"))]
+      (try
+        (testing "testing *.worker.childopts as strings with extra spaces"
+          (let [string-opts "-Dfoo=bar  -Xmx1024m"
+                topo-string-opts "-Dkau=aux   -Xmx2048m"
+                exp-script (exp-script-fn ["-Dfoo=bar" "-Xmx1024m"]
+                                          ["-Dkau=aux" "-Xmx2048m"])
+                _ (.mkdirs (io/file storm-local "workers" mock-worker-id))
+                mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed
+                                        STORM-LOCAL-DIR storm-local
+                                        SUPERVISOR-RUN-WORKER-AS-USER true
+                                        WORKER-CHILDOPTS string-opts}}]
+            (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS
+                                                   topo-string-opts
+                                                   TOPOLOGY-SUBMITTER-USER "me"}
+                       add-to-classpath mock-cp
+                       supervisor-stormdist-root nil
+                       launch-process nil
+                       set-worker-user! nil
+                       supervisor/java-cmd "java"
+                       supervisor/jlp nil
+                       supervisor/write-log-metadata! nil]
+                      (supervisor/launch-worker mock-supervisor
+                                                mock-storm-id
+                                                mock-port
+                                                mock-worker-id
+                                                mock-mem-onheap)
+                      (verify-first-call-args-for-indices launch-process
+                                                          [0]
+                                                          exp-launch))
+            (is (= (slurp worker-script) exp-script))))
+        (finally (rmr storm-local)))
       (.mkdirs (io/file storm-local "workers" mock-worker-id))
       (try
-      (testing "testing *.worker.childopts as strings with extra spaces"
-        (let [string-opts "-Dfoo=bar  -Xmx1024m"
-              topo-string-opts "-Dkau=aux   -Xmx2048m"
-              exp-script (exp-script-fn ["-Dfoo=bar" "-Xmx1024m"]
-                                    ["-Dkau=aux" "-Xmx2048m"])
-              mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed
-                                      STORM-LOCAL-DIR storm-local
-                                      SUPERVISOR-RUN-WORKER-AS-USER true
-                                      WORKER-CHILDOPTS string-opts}}]
-          (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS
-                                                 topo-string-opts
-                                                 TOPOLOGY-SUBMITTER-USER "me"}
-                     add-to-classpath mock-cp
-                     supervisor-stormdist-root nil
-                     launch-process nil
-                     set-worker-user! nil
-                     supervisor/java-cmd "java"
-                     supervisor/jlp nil
-                     supervisor/write-log-metadata! nil]
-            (supervisor/launch-worker mock-supervisor
-                                      mock-storm-id
-                                      mock-port
-                                      mock-worker-id)
-            (verify-first-call-args-for-indices launch-process
-                                                [0]
-                                                exp-launch))
-          (is (= (slurp worker-script) exp-script))))
-      (testing "testing *.worker.childopts as list of strings, with spaces in values"
-        (let [list-opts '("-Dopt1='this has a space in it'" "-Xmx1024m")
-              topo-list-opts '("-Dopt2='val with spaces'" "-Xmx2048m")
-              exp-script (exp-script-fn list-opts topo-list-opts)
-              mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed
-                                      STORM-LOCAL-DIR storm-local
-                                      SUPERVISOR-RUN-WORKER-AS-USER true
-                                      WORKER-CHILDOPTS list-opts}}]
-          (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS
-                                                 topo-list-opts
-                                                 TOPOLOGY-SUBMITTER-USER "me"}
-                     add-to-classpath mock-cp
-                     supervisor-stormdist-root nil
-                     launch-process nil
-                     set-worker-user! nil
-                     supervisor/java-cmd "java"
-                     supervisor/jlp nil
-                     supervisor/write-log-metadata! nil]
-            (supervisor/launch-worker mock-supervisor
-                                      mock-storm-id
-                                      mock-port
-                                      mock-worker-id)
-            (verify-first-call-args-for-indices launch-process
-                                                [0]
-                                                exp-launch))
-          (is (= (slurp worker-script) exp-script))))
-(finally (rm-r (io/file storm-local)))
-))))
+        (testing "testing *.worker.childopts as list of strings, with spaces in values"
+          (let [list-opts '("-Dopt1='this has a space in it'" "-Xmx1024m")
+                topo-list-opts '("-Dopt2='val with spaces'" "-Xmx2048m")
+                exp-script (exp-script-fn list-opts topo-list-opts)
+                mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed
+                                        STORM-LOCAL-DIR storm-local
+                                        SUPERVISOR-RUN-WORKER-AS-USER true
+                                        WORKER-CHILDOPTS list-opts}}]
+            (stubbing [read-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS
+                                                   topo-list-opts
+                                                   TOPOLOGY-SUBMITTER-USER "me"}
+                       add-to-classpath mock-cp
+                       supervisor-stormdist-root nil
+                       launch-process nil
+                       set-worker-user! nil
+                       supervisor/java-cmd "java"
+                       supervisor/jlp nil
+                       supervisor/write-log-metadata! nil]
+                      (supervisor/launch-worker mock-supervisor
+                                                mock-storm-id
+                                                mock-port
+                                                mock-worker-id
+                                                mock-mem-onheap)
+                      (verify-first-call-args-for-indices launch-process
+                                                          [0]
+                                                          exp-launch))
+            (is (= (slurp worker-script) exp-script))))
+        (finally (rmr storm-local))))))
 
 (deftest test-workers-go-bananas
   ;; test that multiple workers are started for a port, and test that
@@ -554,9 +591,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
-          childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m"
-          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          mem-onheap 512
+          childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
+          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-happy-path-list
@@ -564,9 +602,21 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
-          childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m")
-          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          mem-onheap 512
+          childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
+          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+      (is (= expected-childopts childopts-with-ids)))))
+
+(deftest test-substitute-childopts-happy-path-list-arraylist
+  (testing "worker-launcher replaces ids in childopts"
+    (let [worker-id "w-01"
+          topology-id "s-01"
+          port 9999
+          mem-onheap 512
+          childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
+          expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-topology-id-alone
@@ -574,9 +624,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-no-keys
@@ -584,9 +635,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-nil-childopts
@@ -594,9 +646,10 @@
     (let [worker-id "w-01"
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts nil
           expected-childopts nil
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-substitute-childopts-nil-ids
@@ -604,16 +657,17 @@
     (let [worker-id nil
           topology-id "s-01"
           port 9999
+          mem-onheap 512
           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
-          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port)]
+          childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
       (is (= expected-childopts childopts-with-ids)))))
 
 (deftest test-retry-read-assignments
   (with-simulated-time-local-cluster [cluster
                                       :supervisors 0
                                       :ports-per-supervisor 2
-                                      :daemon-conf {NIMBUS-REASSIGN false
+                                      :daemon-conf {NIMBUS-DO-NOT-REASSIGN true
                                                     NIMBUS-MONITOR-FREQ-SECS 10
                                                     TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
                                                     TOPOLOGY-ACKER-EXECUTORS 0}]
@@ -629,25 +683,32 @@
      (bind changed (capture-changed-workers
                     (submit-mocked-assignment
                      (:nimbus cluster)
+                     (:storm-cluster-state cluster)
                      "topology1"
                      {TOPOLOGY-WORKERS 2}
                      topology1
                      {1 "1"
                       2 "1"}
                      {[1 1] ["sup1" 1]
-                      [2 2] ["sup1" 2]
+                      [2 2] ["sup1" 2]}
+                     {["sup1" 1] [0.0 0.0 0.0]
+                      ["sup1" 2] [0.0 0.0 0.0]
                       })
                     (submit-mocked-assignment
                      (:nimbus cluster)
+                     (:storm-cluster-state cluster)
                      "topology2"
                      {TOPOLOGY-WORKERS 2}
                      topology2
                      {1 "1"
                       2 "1"}
                      {[1 1] ["sup1" 1]
-                      [2 2] ["sup1" 2]
+                      [2 2] ["sup1" 2]}
+                     {["sup1" 1] [0.0 0.0 0.0]
+                      ["sup1" 2] [0.0 0.0 0.0]
                       })
-                    (advance-cluster-time cluster 10)
+                    ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+                    (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
                     ))
      (is (empty? (:launched changed)))
      (bind options (RebalanceOptions.))
diff --git a/storm-core/test/clj/backtype/storm/testing4j_test.clj b/storm-core/test/clj/backtype/storm/testing4j_test.clj
index 36f3df6..b504f28 100644
--- a/storm-core/test/clj/backtype/storm/testing4j_test.clj
+++ b/storm-core/test/clj/backtype/storm/testing4j_test.clj
@@ -118,6 +118,7 @@
                           "test-acking2"
                           (Config.)
                           (.getTopology tracked))
+         (advance-cluster-time (.getState cluster) 11)
          (.feed feeder [1])
          (Testing/trackedWait tracked (int 1))
          (checker 0)
diff --git a/storm-core/test/clj/backtype/storm/transactional_test.clj b/storm-core/test/clj/backtype/storm/transactional_test.clj
index fc9137d..143df0c 100644
--- a/storm-core/test/clj/backtype/storm/transactional_test.clj
+++ b/storm-core/test/clj/backtype/storm/transactional_test.clj
@@ -20,6 +20,7 @@
   (:import [backtype.storm.transactional TransactionalSpoutCoordinator ITransactionalSpout ITransactionalSpout$Coordinator TransactionAttempt
             TransactionalTopologyBuilder])
   (:import [backtype.storm.transactional.state TransactionalState TestTransactionalState RotatingTransactionalState RotatingTransactionalState$StateInitializer])
+  (:import [backtype.storm.generated RebalanceOptions])
   (:import [backtype.storm.spout SpoutOutputCollector ISpoutOutputCollector])
   (:import [backtype.storm.task OutputCollector IOutputCollector])
   (:import [backtype.storm.coordination BatchBoltExecutor])
@@ -411,7 +412,7 @@
                               "transactional-test"
                               {TOPOLOGY-MAX-SPOUT-PENDING 2}
                               (:topology topo-info))
-
+       (advance-cluster-time cluster 11)
        (bind ack-tx! (fn [txid]
                        (let [[to-ack not-to-ack] (separate
                                                   #(-> %
@@ -666,7 +667,7 @@
                               {TOPOLOGY-MAX-SPOUT-PENDING 2
                                }
                               (:topology topo-info))
-
+       (advance-cluster-time cluster 11)
        (bind ack-tx! (fn [txid]
                        (let [[to-ack not-to-ack] (separate
                                                   #(-> %
diff --git a/storm-core/test/clj/backtype/storm/worker_test.clj b/storm-core/test/clj/backtype/storm/worker_test.clj
index 2e0533d..b8e5f0f 100644
--- a/storm-core/test/clj/backtype/storm/worker_test.clj
+++ b/storm-core/test/clj/backtype/storm/worker_test.clj
@@ -15,15 +15,184 @@
 ;; limitations under the License.
 (ns backtype.storm.worker-test
   (:use [clojure test])
+  (:require [backtype.storm.daemon [worker :as worker]])
+  (:require [backtype.storm [util :as util]])
+  (:require [conjure.core])
+  (:require [clj-time.core :as time])
+  (:require [clj-time.coerce :as coerce])
+  (:import [backtype.storm.generated LogConfig LogLevel LogLevelAction])
+  (:import [org.apache.logging.log4j Level LogManager])
+  (:import [org.slf4j Logger])
+  (:use [conjure core])
+  (:use [backtype.storm testing log])
+  (:use [backtype.storm.daemon common])
+  (:use [clojure.string :only [join]])
   (:import [backtype.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
   (:import [org.mockito Mockito])
-  (:use [backtype.storm testing])
-  (:use [backtype.storm.daemon common])
-
-  (:require [backtype.storm.daemon [worker :as worker]])
   )
 
 
+(deftest test-log-reset-should-not-trigger-for-future-time
+  (with-local-cluster [cluster]
+    (let [worker (:worker cluster)
+          present (time/now)
+          the-future (coerce/to-long (time/plus present (time/secs 1)))
+          mock-config {"foo" {:timeout the-future}}
+          mock-config-atom (atom mock-config)]
+      (stubbing [time/now present]
+        (worker/reset-log-levels mock-config-atom)
+        ;; if the worker doesn't reset log levels, the atom should not be nil
+        (is (not(= @mock-config-atom nil)))))))
+
+(deftest test-log-reset-triggers-for-past-time
+  (with-local-cluster [cluster]
+    (let [worker (:worker cluster)
+          present (time/now)
+          past (time/plus present (time/secs -1))
+          mock-config {"foo" { :timeout (coerce/to-long past)
+                               :target-log-level Level/INFO
+                               :reset-log-level Level/WARN}}
+          mock-config-atom (atom mock-config)]
+      (stubbing [time/now present]
+        (worker/reset-log-levels mock-config-atom)
+        ;; the logger config is removed from atom
+        (is (= @mock-config-atom {}))))))
+
+(deftest test-log-reset-resets-does-nothing-for-empty-log-config
+  (with-local-cluster [cluster]
+    (let [worker (:worker cluster)
+          present (time/now)
+          past (coerce/to-long (time/plus present (time/secs -1)))
+          mock-config {}
+          mock-config-atom (atom mock-config)]
+      (stubbing [worker/set-logger-level nil
+                 time/now present]
+        (worker/reset-log-levels mock-config-atom)
+        ;; if the worker resets log level, the atom is nil'ed out
+        (is (= @mock-config-atom {}))
+        ;; test that the set-logger-level function was not called
+        (verify-call-times-for worker/set-logger-level 0)))))
+
+(deftest test-log-reset-resets-root-logger-if-set
+  (with-local-cluster [cluster]
+    (let [worker (:worker cluster)
+          present (time/now)
+          past (coerce/to-long (time/plus present (time/secs -1)))
+          mock-config {LogManager/ROOT_LOGGER_NAME  {:timeout past
+                                                     :target-log-level Level/DEBUG
+                                                     :reset-log-level Level/WARN}}
+          mock-config-atom (atom mock-config)]
+      (stubbing [worker/set-logger-level nil
+                 time/now present]
+        (worker/reset-log-levels mock-config-atom)
+        ;; if the worker resets log level, the atom is reset to {}
+        (is (= @mock-config-atom {}))
+        ;; ensure we reset back to WARN level
+        (verify-call-times-for worker/set-logger-level 1)
+        (verify-first-call-args-for-indices worker/set-logger-level [1 2] LogManager/ROOT_LOGGER_NAME Level/WARN)))))
+
+;;This should be removed when it goes into conjure
+(defmacro verify-nth-call-args-for-indices
+  "Asserts that the function was called at least once, and the nth call was
+   passed the args specified, into the indices of the arglist specified. In
+   other words, it checks only the particular args you care about."
+  [n fn-name indices & args]
+  `(do
+     (assert-in-fake-context "verify-first-call-args-for-indices")
+     (assert-conjurified-fn "verify-first-call-args-for-indices" ~fn-name)
+     (is (< ~n (count (get @call-times ~fn-name)))
+         (str "(verify-nth-call-args-for-indices " ~n " " ~fn-name " " ~indices " " ~(join " " args) ")"))
+     (let [nth-call-args# (nth (get @call-times ~fn-name) ~n)
+           indices-in-range?# (< (apply max ~indices) (count nth-call-args#))]
+       (if indices-in-range?#
+         (is (= ~(vec args) (map #(nth nth-call-args# %) ~indices))
+             (str "(verify-first-call-args-for-indices " ~n " " ~fn-name " " ~indices " " ~(join " " args) ")"))
+         (is (= :fail (format "indices %s are out of range for the args, %s" ~indices ~(vec args)))
+             (str "(verify-first-call-args-for-indices " ~n " " ~fn-name " " ~indices " " ~(join " " args) ")"))))))
+
+(deftest test-log-resets-named-loggers-with-past-timeout
+  (with-local-cluster [cluster]
+    (let [worker (:worker cluster)
+          present (time/now)
+          past (coerce/to-long (time/plus present (time/secs -1)))
+          mock-config {"my_debug_logger" {:timeout past
+                                          :target-log-level Level/DEBUG
+                                          :reset-log-level Level/INFO} 
+                       "my_info_logger" {:timeout past
+                                         :target-log-level Level/INFO
+                                         :reset-log-level Level/WARN}
+                       "my_error_logger" {:timeout past
+                                          :target-log-level Level/ERROR
+                                          :reset-log-level Level/INFO}}
+          result (atom {})
+          mock-config-atom (atom mock-config)]
+      (stubbing [worker/set-logger-level nil
+                 time/now present]
+          (worker/reset-log-levels mock-config-atom)
+          ;; if the worker resets log level, the atom is reset to {}
+          (is (= @mock-config-atom {}))
+          (verify-call-times-for worker/set-logger-level 3)
+          (verify-nth-call-args-for-indices 0 worker/set-logger-level [1 2] "my_debug_logger" Level/INFO)
+          (verify-nth-call-args-for-indices 1 worker/set-logger-level [1 2] "my_error_logger" Level/INFO)
+          (verify-nth-call-args-for-indices 2 worker/set-logger-level [1 2] "my_info_logger" Level/WARN)))))
+
+(deftest test-process-root-log-level-to-debug-sets-logger-and-timeout-2
+  (with-local-cluster [cluster]
+    (let [worker (:worker cluster)
+          mock-config (LogConfig.)
+          root-level (LogLevel.)
+          mock-config-atom (atom nil)
+          orig-levels (atom {})
+          present (time/now)
+          in-thirty-seconds (coerce/to-long (time/plus present (time/secs 30)))]
+      ;; configure the root logger to be debug
+      (.set_reset_log_level_timeout_epoch root-level in-thirty-seconds)
+      (.set_target_log_level root-level "DEBUG")
+      (.set_action root-level LogLevelAction/UPDATE)
+      (.put_to_named_logger_level mock-config "ROOT" root-level)
+      (stubbing [worker/set-logger-level nil
+                 time/now present]
+          (worker/process-log-config-change mock-config-atom orig-levels mock-config)
+          ;; test that the set-logger-level function was not called
+          (log-message "Tests " @mock-config-atom)
+          (verify-call-times-for worker/set-logger-level 1)
+          (verify-nth-call-args-for-indices 0 worker/set-logger-level [1 2] "" Level/DEBUG)
+          (let [root-result (get @mock-config-atom LogManager/ROOT_LOGGER_NAME)]
+            (is (= (:action root-result) LogLevelAction/UPDATE))
+            (is (= (:target-log-level root-result) Level/DEBUG))
+            ;; defaults to INFO level when the logger isn't found previously
+            (is (= (:reset-log-level root-result) Level/INFO))
+            (is (= (:timeout root-result) in-thirty-seconds)))))))
+
+(deftest test-process-root-log-level-to-debug-sets-logger-and-timeout
+  (with-local-cluster [cluster]
+    (let [worker (:worker cluster)
+          mock-config (LogConfig.)
+          root-level (LogLevel.)
+          orig-levels (atom {})
+          present (time/now)
+          in-thirty-seconds (coerce/to-long (time/plus present (time/secs 30)))
+          mock-config-atom (atom {})]
+      ;; configure the root logger to be debug
+      (doseq [named {"ROOT" "DEBUG"
+                     "my_debug_logger" "DEBUG"
+                     "my_info_logger" "INFO"
+                     "my_error_logger" "ERROR"}]
+        (let [level (LogLevel.)]
+          (.set_action level LogLevelAction/UPDATE)
+          (.set_reset_log_level_timeout_epoch level in-thirty-seconds)
+          (.set_target_log_level level (val named))
+          (.put_to_named_logger_level mock-config (key named) level)))
+      (log-message "Tests " mock-config)
+      (stubbing [worker/set-logger-level nil
+                 time/now present]
+          (worker/process-log-config-change mock-config-atom orig-levels mock-config)
+          (verify-call-times-for worker/set-logger-level 4)
+          (verify-nth-call-args-for-indices 0 worker/set-logger-level [1 2] "" Level/DEBUG)
+          (verify-nth-call-args-for-indices 1 worker/set-logger-level [1 2] "my_debug_logger" Level/DEBUG)
+          (verify-nth-call-args-for-indices 2 worker/set-logger-level [1 2] "my_error_logger" Level/ERROR)
+          (verify-nth-call-args-for-indices 3 worker/set-logger-level [1 2] "my_info_logger" Level/INFO)))))
+
 (deftest test-worker-is-connection-ready
   (let [connection (Mockito/mock ConnectionWithStatus)]
     (. (Mockito/when (.status connection)) thenReturn ConnectionWithStatus$Status/Ready)
@@ -34,4 +203,4 @@
 
     (. (Mockito/when (.status connection)) thenReturn ConnectionWithStatus$Status/Closed)
     (is (= false (worker/is-connection-ready connection)))
-  ))
\ No newline at end of file
+  ))
diff --git a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
new file mode 100644
index 0000000..813ae84
--- /dev/null
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
@@ -0,0 +1,150 @@
+;; 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 org.apache.storm.pacemaker-state-factory-test
+  (:require [clojure.test :refer :all]
+            [conjure.core :refer :all]
+            [org.apache.storm.pacemaker [pacemaker-state-factory :as psf]])
+  (:import [backtype.storm.generated
+            HBExecutionException HBNodes HBRecords
+            HBServerMessageType HBMessage HBMessageData HBPulse]
+           [backtype.storm.cluster ClusterStateContext]
+           [org.mockito Mockito Matchers]))
+
+(defn- string-to-bytes [string]
+  (byte-array (map int string)))
+
+(defn- bytes-to-string [bytez]
+  (apply str (map char bytez)))
+
+(defprotocol send-capture
+  (send [this something])
+  (check-captured [this]))
+
+(defn- make-send-capture [response]
+  (let [captured (atom nil)]
+    (reify send-capture
+      (send [this something] (reset! captured something) response)
+      (check-captured [this] @captured))))
+
+(defmacro with-mock-pacemaker-client-and-state [client state response & body]
+  `(let [~client (make-send-capture ~response)]
+     (stubbing [psf/makeZKState nil
+                psf/makeClient ~client]
+               (let [~state (psf/-mkState nil nil nil nil (ClusterStateContext.))]
+                 ~@body))))
+
+
+(deftest pacemaker_state_set_worker_hb
+  (testing "set_worker_hb"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)
+
+      (.set_worker_hb state "/foo" (string-to-bytes "data") nil)
+      (let [sent (.check-captured client)
+            pulse (.get_pulse (.get_data sent))]
+        (is (= (.get_type sent) HBServerMessageType/SEND_PULSE))
+        (is (= (.get_id pulse) "/foo"))
+        (is (= (bytes-to-string (.get_details pulse)) "data")))))
+
+  (testing "set_worker_hb"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/SEND_PULSE nil)
+
+      (is (thrown? HBExecutionException      
+                   (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
+
+      
+
+(deftest pacemaker_state_delete_worker_hb
+  (testing "delete_worker_hb"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil)
+
+      (.delete_worker_hb state "/foo/bar")
+      (let [sent (.check-captured client)]
+        (is (= (.get_type sent) HBServerMessageType/DELETE_PATH))
+        (is (= (.get_path (.get_data sent)) "/foo/bar")))))
+
+    (testing "delete_worker_hb"
+      (with-mock-pacemaker-client-and-state
+        client state
+        (HBMessage. HBServerMessageType/DELETE_PATH nil)
+        
+        (is (thrown? HBExecutionException
+                     (.delete_worker_hb state "/foo/bar"))))))
+
+(deftest pacemaker_state_get_worker_hb
+  (testing "get_worker_hb"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
+                (HBMessageData/pulse
+                 (doto (HBPulse.)
+                   (.set_id "/foo")
+                   (.set_details (string-to-bytes "some data")))))
+
+      (.get_worker_hb state "/foo" false)
+      (let [sent (.check-captured client)]
+        (is (= (.get_type sent) HBServerMessageType/GET_PULSE))
+        (is (= (.get_path (.get_data sent)) "/foo")))))
+
+  (testing "get_worker_hb - fail (bad response)"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/GET_PULSE nil)
+      
+      (is (thrown? HBExecutionException
+                   (.get_worker_hb state "/foo" false)))))
+  
+  (testing "get_worker_hb - fail (bad data)"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE nil)
+      
+      (is (thrown? HBExecutionException
+                   (.get_worker_hb state "/foo" false))))))
+
+(deftest pacemaker_state_get_worker_hb_children
+  (testing "get_worker_hb_children"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
+                (HBMessageData/nodes
+                 (HBNodes. [])))
+
+      (.get_worker_hb_children state "/foo" false)
+      (let [sent (.check-captured client)]
+        (is (= (.get_type sent) HBServerMessageType/GET_ALL_NODES_FOR_PATH))
+        (is (= (.get_path (.get_data sent)) "/foo")))))
+
+  (testing "get_worker_hb_children - fail (bad response)"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/DELETE_PATH nil)
+
+      (is (thrown? HBExecutionException
+                   (.get_worker_hb_children state "/foo" false)))))
+
+    (testing "get_worker_hb_children - fail (bad data)"
+    (with-mock-pacemaker-client-and-state
+      client state
+      (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE nil)
+      
+      (is (thrown? HBExecutionException
+                   (.get_worker_hb_children state "/foo" false))))))
diff --git a/storm-core/test/clj/org/apache/storm/pacemaker_test.clj b/storm-core/test/clj/org/apache/storm/pacemaker_test.clj
new file mode 100644
index 0000000..0987923
--- /dev/null
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_test.clj
@@ -0,0 +1,242 @@
+;; 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 org.apache.storm.pacemaker-test
+  (:require [clojure.test :refer :all]
+            [org.apache.storm.pacemaker [pacemaker :as pacemaker]]
+            [conjure.core :as conjure])
+  (:import [backtype.storm.generated
+            HBExecutionException HBServerMessageType
+            HBMessage HBMessageData HBPulse]))
+
+(defn- message-with-rand-id [type data]
+  (let [mid (rand-int 1000)
+        message (HBMessage. type data)]
+    (.set_message_id message mid)
+    [message mid]))
+
+(defn- string-to-bytes [string]
+  (byte-array (map int string)))
+
+(defn- bytes-to-string [bytez]
+  (apply str (map char bytez)))
+
+(defn- makenode [handler path]
+  (.handleMessage handler
+                  (HBMessage.
+                   HBServerMessageType/SEND_PULSE
+                   (HBMessageData/pulse
+                    (doto (HBPulse.)
+                      (.set_id path)
+                      (.set_details (string-to-bytes "nothing")))))
+                  true))
+
+(deftest pacemaker-server-create-path
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "CREATE_PATH"
+       (let [[message mid] (message-with-rand-id
+                            HBServerMessageType/CREATE_PATH
+                            (HBMessageData/path "/testpath"))
+             response (.handleMessage handler message true)]
+         (is (= (.get_message_id response) mid))
+         (is (= (.get_type response) HBServerMessageType/CREATE_PATH_RESPONSE))
+         (is (= (.get_data response) nil)))))))
+
+(deftest pacemaker-server-exists
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "EXISTS - false"
+       (let [[message mid] (message-with-rand-id HBServerMessageType/EXISTS
+                                                 (HBMessageData/path "/testpath"))
+             bad-response (.handleMessage handler message false)
+             good-response (.handleMessage handler message true)]
+         (is (= (.get_message_id bad-response) mid))
+         (is (= (.get_type bad-response) HBServerMessageType/NOT_AUTHORIZED))
+
+         (is (= (.get_message_id good-response) mid))
+         (is (= (.get_type good-response) HBServerMessageType/EXISTS_RESPONSE))
+         (is (= (.get_boolval (.get_data good-response)) false))))
+
+     (testing "EXISTS - true"
+       (let [path "/exists_path"
+             data-string "pulse data"]
+         (let [[send _] (message-with-rand-id
+                         HBServerMessageType/SEND_PULSE
+                         (HBMessageData/pulse
+                          (doto (HBPulse.)
+                            (.set_id path)
+                            (.set_details (string-to-bytes data-string)))))
+               _ (.handleMessage handler send true)
+               [message mid] (message-with-rand-id HBServerMessageType/EXISTS
+                                                   (HBMessageData/path path))
+               bad-response (.handleMessage handler message false)
+               good-response (.handleMessage handler message true)]
+           (is (= (.get_message_id bad-response) mid))
+           (is (= (.get_type bad-response) HBServerMessageType/NOT_AUTHORIZED))
+
+           (is (= (.get_message_id good-response) mid))
+           (is (= (.get_type good-response) HBServerMessageType/EXISTS_RESPONSE))
+           (is (= (.get_boolval (.get_data good-response)) true))))))))
+
+(deftest pacemaker-server-send-pulse-get-pulse
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "SEND_PULSE - GET_PULSE"
+       (let [path "/pulsepath"
+             data-string "pulse data"]
+         (let [[message mid] (message-with-rand-id
+                              HBServerMessageType/SEND_PULSE
+                              (HBMessageData/pulse
+                               (doto (HBPulse.)
+                                 (.set_id path)
+                                 (.set_details (string-to-bytes data-string)))))
+               response (.handleMessage handler message true)]
+           (is (= (.get_message_id response) mid))
+           (is (= (.get_type response) HBServerMessageType/SEND_PULSE_RESPONSE))
+           (is (= (.get_data response) nil)))
+         (let [[message mid] (message-with-rand-id
+                              HBServerMessageType/GET_PULSE
+                              (HBMessageData/path path))
+               response (.handleMessage handler message true)]
+           (is (= (.get_message_id response) mid))
+           (is (= (.get_type response) HBServerMessageType/GET_PULSE_RESPONSE))
+           (is (= (bytes-to-string (.get_details (.get_pulse (.get_data response)))) data-string))))))))
+
+(deftest pacemaker-server-get-all-pulse-for-path
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "GET_ALL_PULSE_FOR_PATH"
+       (let [[message mid] (message-with-rand-id HBServerMessageType/GET_ALL_PULSE_FOR_PATH
+                                                 (HBMessageData/path "/testpath"))
+             bad-response (.handleMessage handler message false)
+             good-response (.handleMessage handler message true)]
+         (is (= (.get_message_id bad-response) mid))
+         (is (= (.get_type bad-response) HBServerMessageType/NOT_AUTHORIZED))
+
+         (is (= (.get_message_id good-response) mid))
+         (is (= (.get_type good-response) HBServerMessageType/GET_ALL_PULSE_FOR_PATH_RESPONSE))
+         (is (= (.get_data good-response) nil)))))))
+
+(deftest pacemaker-server-get-all-nodes-for-path
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "GET_ALL_NODES_FOR_PATH"
+       (makenode handler "/some-root-path/foo")
+       (makenode handler "/some-root-path/bar")
+       (makenode handler "/some-root-path/baz")
+       (makenode handler "/some-root-path/boo")
+       (let [[message mid] (message-with-rand-id HBServerMessageType/GET_ALL_NODES_FOR_PATH
+                                                 (HBMessageData/path "/some-root-path"))
+             bad-response (.handleMessage handler message false)
+             good-response (.handleMessage handler message true)
+             ids (into #{} (.get_pulseIds (.get_nodes (.get_data good-response))))]
+         (is (= (.get_message_id bad-response) mid))
+         (is (= (.get_type bad-response) HBServerMessageType/NOT_AUTHORIZED))
+
+         (is (= (.get_message_id good-response) mid))
+         (is (= (.get_type good-response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE))
+         (is (contains? ids "foo"))
+         (is (contains? ids "bar"))
+         (is (contains? ids "baz"))
+         (is (contains? ids "boo")))
+
+       (makenode handler "/some/deeper/path/foo")
+       (makenode handler "/some/deeper/path/bar")
+       (makenode handler "/some/deeper/path/baz")
+       (let [[message mid] (message-with-rand-id HBServerMessageType/GET_ALL_NODES_FOR_PATH
+                                                 (HBMessageData/path "/some/deeper/path"))
+             bad-response (.handleMessage handler message false)
+             good-response (.handleMessage handler message true)
+             ids (into #{} (.get_pulseIds (.get_nodes (.get_data good-response))))]
+         (is (= (.get_message_id bad-response) mid))
+         (is (= (.get_type bad-response) HBServerMessageType/NOT_AUTHORIZED))
+
+         (is (= (.get_message_id good-response) mid))
+         (is (= (.get_type good-response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE))
+         (is (contains? ids "foo"))
+         (is (contains? ids "bar"))
+         (is (contains? ids "baz")))))))
+
+(deftest pacemaker-server-get-pulse
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "GET_PULSE"
+       (makenode handler "/some-root/GET_PULSE")
+       (let [[message mid] (message-with-rand-id HBServerMessageType/GET_PULSE
+                                                 (HBMessageData/path "/some-root/GET_PULSE"))
+             bad-response (.handleMessage handler message false)
+             good-response (.handleMessage handler message true)
+             good-pulse (.get_pulse (.get_data good-response))]
+         (is (= (.get_message_id bad-response) mid))
+         (is (= (.get_type bad-response) HBServerMessageType/NOT_AUTHORIZED))
+         (is (= (.get_data bad-response) nil))
+
+         (is (= (.get_message_id good-response) mid))
+         (is (= (.get_type good-response) HBServerMessageType/GET_PULSE_RESPONSE))
+         (is (= (.get_id good-pulse) "/some-root/GET_PULSE"))
+         (is (= (bytes-to-string (.get_details good-pulse)) "nothing")))))))
+
+(deftest pacemaker-server-delete-path
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "DELETE_PATH"
+       (makenode handler "/some-root/DELETE_PATH/foo")
+       (makenode handler "/some-root/DELETE_PATH/bar")
+       (makenode handler "/some-root/DELETE_PATH/baz")
+       (makenode handler "/some-root/DELETE_PATH/boo")
+       (let [[message mid] (message-with-rand-id HBServerMessageType/DELETE_PATH
+                                                 (HBMessageData/path "/some-root/DELETE_PATH"))
+             response (.handleMessage handler message true)]
+         (is (= (.get_message_id response) mid))
+         (is (= (.get_type response) HBServerMessageType/DELETE_PATH_RESPONSE))
+         (is (= (.get_data response) nil)))
+       (let [[message mid] (message-with-rand-id HBServerMessageType/GET_ALL_NODES_FOR_PATH
+                                                 (HBMessageData/path "/some-root/DELETE_PATH"))
+             response (.handleMessage handler message true)
+             ids (into #{} (.get_pulseIds (.get_nodes (.get_data response))))]
+         (is (= (.get_message_id response) mid))
+         (is (= (.get_type response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE))
+         (is (empty? ids)))))))
+
+(deftest pacemaker-server-delete-pulse-id
+  (conjure/stubbing
+   [pacemaker/register nil]
+   (let [handler (pacemaker/mk-handler {})]
+     (testing "DELETE_PULSE_ID"
+       (makenode handler "/some-root/DELETE_PULSE_ID/foo")
+       (makenode handler "/some-root/DELETE_PULSE_ID/bar")
+       (makenode handler "/some-root/DELETE_PULSE_ID/baz")
+       (makenode handler "/some-root/DELETE_PULSE_ID/boo")
+       (let [[message mid] (message-with-rand-id HBServerMessageType/DELETE_PULSE_ID
+                                                 (HBMessageData/path "/some-root/DELETE_PULSE_ID/foo"))
+             response (.handleMessage handler message true)]
+         (is (= (.get_message_id response) mid))
+         (is (= (.get_type response) HBServerMessageType/DELETE_PULSE_ID_RESPONSE))
+         (is (= (.get_data response) nil)))
+       (let [[message mid] (message-with-rand-id HBServerMessageType/GET_ALL_NODES_FOR_PATH
+                                                 (HBMessageData/path "/some-root/DELETE_PULSE_ID"))
+             response (.handleMessage handler message true)
+             ids (into #{} (.get_pulseIds (.get_nodes (.get_data response))))]
+         (is (= (.get_message_id response) mid))
+         (is (= (.get_type response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE))
+         (is (not (contains? ids "foo"))))))))
diff --git a/storm-core/test/jvm/backtype/storm/TestConfigValidate.java b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
new file mode 100644
index 0000000..7f193cc
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/TestConfigValidate.java
@@ -0,0 +1,660 @@
+/**
+ * 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;
+
+import backtype.storm.utils.Utils;
+import backtype.storm.validation.ConfigValidation;
+import backtype.storm.validation.ConfigValidation.*;
+import backtype.storm.validation.ConfigValidationAnnotations.*;
+import org.junit.Test;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class TestConfigValidate {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestConfigValidate.class);
+
+    @Test
+    public void validPacemakerAuthTest() throws InstantiationException, IllegalAccessException, NoSuchFieldException, NoSuchMethodException, InvocationTargetException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        conf.put(Config.PACEMAKER_AUTH_METHOD, "NONE");
+        ConfigValidation.validateFields(conf);
+        conf.put(Config.PACEMAKER_AUTH_METHOD, "DIGEST");
+        ConfigValidation.validateFields(conf);
+        conf.put(Config.PACEMAKER_AUTH_METHOD, "KERBEROS");
+        ConfigValidation.validateFields(conf);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void invalidPacemakerAuthTest() throws InstantiationException, IllegalAccessException, NoSuchFieldException, NoSuchMethodException, InvocationTargetException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        conf.put(Config.PACEMAKER_AUTH_METHOD, "invalid");
+        ConfigValidation.validateFields(conf);
+    }
+    
+    @Test
+    public void validConfigTest() throws InstantiationException, IllegalAccessException, NoSuchFieldException, NoSuchMethodException, InvocationTargetException {
+
+
+        Map<String, Object> conf = new HashMap<String, Object>();
+        conf.put(Config.STORM_MESSAGING_NETTY_SOCKET_BACKLOG, 5);
+        conf.put(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS, 500);
+        conf.put(Config.STORM_MESSAGING_NETTY_AUTHENTICATION, true);
+
+        ConfigValidation.validateFields(conf);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void invalidConfigTest() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        conf.put(Config.STORM_MESSAGING_NETTY_SOCKET_BACKLOG, 5);
+        conf.put(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS, 500);
+        conf.put(Config.STORM_MESSAGING_NETTY_AUTHENTICATION, "invalid");
+
+        ConfigValidation.validateFields(conf);
+    }
+
+    @Test
+    public void defaultYamlTest() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        Map conf = Utils.readStormConfig();
+        ConfigValidation.validateFields(conf);
+    }
+
+    @Test
+    public void testTopologyWorkersIsInteger() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        conf.put(Config.TOPOLOGY_WORKERS, 42);
+        ConfigValidation.validateFields(conf);
+
+        conf.put(Config.TOPOLOGY_WORKERS, 3.14159);
+        try {
+            ConfigValidation.validateFields(conf);
+            Assert.fail("Expected Exception not Thrown");
+        } catch (IllegalArgumentException ex) {
+        }
+    }
+
+    @Test
+    public void testTopologyStatsSampleRateIsFloat() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, 0.5);
+        ConfigValidation.validateFields(conf);
+        conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, 10);
+        ConfigValidation.validateFields(conf);
+        conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, Double.MAX_VALUE);
+        ConfigValidation.validateFields(conf);
+    }
+
+    @Test
+    public void testIsolationSchedulerMachinesIsMap() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        Map<String, Integer> isolationMap = new HashMap<String, Integer>();
+        conf.put(Config.ISOLATION_SCHEDULER_MACHINES, isolationMap);
+        ConfigValidation.validateFields(conf);
+
+        isolationMap.put("host0", 1);
+        isolationMap.put("host1", 2);
+
+        conf.put(Config.ISOLATION_SCHEDULER_MACHINES, isolationMap);
+        ConfigValidation.validateFields(conf);
+
+        conf.put(Config.ISOLATION_SCHEDULER_MACHINES, 42);
+        try {
+            ConfigValidation.validateFields(conf);
+            Assert.fail("Expected Exception not Thrown");
+        } catch (IllegalArgumentException ex) {
+        }
+    }
+
+    @Test
+    public void testWorkerChildoptsIsStringOrStringList() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+
+        passCases.add(null);
+        passCases.add("some string");
+        String[] stuff = {"some", "string", "list"};
+        passCases.add(Arrays.asList(stuff));
+
+        failCases.add(42);
+        Integer[] wrongStuff = {1, 2, 3};
+        failCases.add(Arrays.asList(wrongStuff));
+
+        //worker.childopts validates
+        for (Object value : passCases) {
+            conf.put(Config.WORKER_CHILDOPTS, value);
+            ConfigValidation.validateFields(conf);
+        }
+
+        for (Object value : failCases) {
+            try {
+                conf.put(Config.WORKER_CHILDOPTS, value);
+                ConfigValidation.validateFields(conf);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+
+        //topology.worker.childopts validates
+        conf = new HashMap<String, Object>();
+        for (Object value : passCases) {
+            conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, value);
+            ConfigValidation.validateFields(conf);
+        }
+
+        for (Object value : failCases) {
+            try {
+                conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, value);
+                ConfigValidation.validateFields(conf);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testSupervisorSlotsPorts() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+
+        Integer[] test1 = {1233, 1234, 1235};
+        Integer[] test2 = {1233};
+        passCases.add(Arrays.asList(test1));
+        passCases.add(Arrays.asList(test2));
+
+        String[] test3 = {"1233", "1234", "1235"};
+        //duplicate case
+        Integer[] test4 = {1233, 1233, 1235};
+        failCases.add(test3);
+        failCases.add(test4);
+        failCases.add(null);
+        failCases.add("1234");
+        failCases.add(1234);
+
+        for (Object value : passCases) {
+            conf.put(Config.SUPERVISOR_SLOTS_PORTS, value);
+            ConfigValidation.validateFields(conf);
+        }
+
+        for (Object value : failCases) {
+            try {
+                conf.put(Config.SUPERVISOR_SLOTS_PORTS, value);
+                ConfigValidation.validateFields(conf);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testValidity() {
+        Map<String, Object> conf = new HashMap<String, Object>();
+        conf.put(Config.TOPOLOGY_DEBUG, true);
+        conf.put("q", "asasdasd");
+        conf.put("aaa", new Integer("123"));
+        conf.put("bbb", new Long("456"));
+        List<Object> testList = new ArrayList<Object>();
+        testList.add(1);
+        testList.add(2);
+        testList.add(new Integer("3"));
+        testList.add(new Long("4"));
+        conf.put("eee", testList);
+        Utils.isValidConf(conf);
+    }
+
+    @Test
+    public void testPowerOf2Validator() {
+        PowerOf2Validator validator = new PowerOf2Validator();
+
+        Object[] failCases = {42.42, 42, -33, 23423423423.0, -32, -1, -0.00001, 0, -0, "Forty-two"};
+        for (Object value : failCases) {
+            try {
+                validator.validateField("test", value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+
+        Object[] passCases = {64, 4294967296.0, 1, null};
+        for (Object value : passCases) {
+            validator.validateField("test", value);
+        }
+    }
+
+    @Test
+    public void testPositiveNumberValidator() {
+        PositiveNumberValidator validator = new PositiveNumberValidator();
+
+        Object[] passCases = {null, 1.0, 0.01, 1, 2147483647, 42};
+
+        for (Object value : passCases) {
+            validator.validateField("test", value);
+        }
+
+        Object[] failCases = {-1.0, -1, -0.01, 0.0, 0, "43", "string"};
+
+        for (Object value : failCases) {
+            try {
+                validator.validateField("test", value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+
+        Object[] passCasesIncludeZero = {null, 1.0, 0.01, 0, 2147483647, 0.0};
+
+        for (Object value : passCasesIncludeZero) {
+            validator.validateField("test", true, value);
+        }
+
+        Object[] failCasesIncludeZero = {-1.0, -1, -0.01, "43", "string"};
+
+        for (Object value : failCasesIncludeZero) {
+            try {
+                validator.validateField("test", true, value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testIntegerValidator() {
+        IntegerValidator validator = new IntegerValidator();
+
+        Object[] passCases = {null, 1000, Integer.MAX_VALUE};
+
+        for (Object value : passCases) {
+            validator.validateField("test", value);
+        }
+
+        Object[] failCases = {1.34, new Long(Integer.MAX_VALUE) + 1};
+
+        for (Object value : failCases) {
+            try {
+                validator.validateField("test", value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void NoDuplicateInListValidator() {
+        NoDuplicateInListValidator validator = new NoDuplicateInListValidator();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+
+        Object[] passCase1 = {1000, 0, -1000};
+        Object[] passCase2 = {"one", "two", "three"};
+        Object[] passCase3 = {false, true};
+        Object[] passCase4 = {false, true, 1000, 0, -1000, "one", "two", "three"};
+        Object[] passCase5 = {1000.0, 0.0, -1000.0};
+        passCases.add(Arrays.asList(passCase1));
+        passCases.add(Arrays.asList(passCase2));
+        passCases.add(Arrays.asList(passCase3));
+        passCases.add(Arrays.asList(passCase4));
+        passCases.add(Arrays.asList(passCase5));
+        passCases.add(null);
+
+        for (Object value : passCases) {
+            validator.validateField("test", value);
+        }
+
+        Object[] failCase1 = {1000, 0, 1000};
+        Object[] failCase2 = {"one", "one", "two"};
+        Object[] failCase3 = {5.0, 5.0, 6};
+        failCases.add(Arrays.asList(failCase1));
+        failCases.add(Arrays.asList(failCase2));
+        failCases.add(Arrays.asList(failCase3));
+        for (Object value : failCases) {
+            try {
+                validator.validateField("test", value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testListEntryTypeValidator() {
+        Collection<Object> testCases1 = new LinkedList<Object>();
+        Collection<Object> testCases2 = new LinkedList<Object>();
+        Collection<Object> testCases3 = new LinkedList<Object>();
+
+        Object[] testCase1 = {"one", "two", "three"};
+        ;
+        Object[] testCase2 = {"three"};
+        testCases1.add(Arrays.asList(testCase1));
+        testCases1.add(Arrays.asList(testCase2));
+
+        for (Object value : testCases1) {
+            ListEntryTypeValidator.validateField("test", String.class, value);
+        }
+
+        for (Object value : testCases1) {
+            try {
+                ListEntryTypeValidator.validateField("test", Number.class, value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+
+        Object[] testCase3 = {1000, 0, 1000};
+        Object[] testCase4 = {5};
+        Object[] testCase5 = {5.0, 5.0, 6};
+        testCases2.add(Arrays.asList(testCase3));
+        testCases2.add(Arrays.asList(testCase4));
+        testCases2.add(Arrays.asList(testCase5));
+        for (Object value : testCases2) {
+            try {
+                ListEntryTypeValidator.validateField("test", String.class, value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+        for (Object value : testCases2) {
+            ListEntryTypeValidator.validateField("test", Number.class, value);
+        }
+
+        Object[] testCase6 = {1000, 0, 1000, "5"};
+        Object[] testCase7 = {"4", "5", 5};
+        testCases3.add(Arrays.asList(testCase6));
+        testCases3.add(Arrays.asList(testCase7));
+        for (Object value : testCases3) {
+            try {
+                ListEntryTypeValidator.validateField("test", String.class, value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+        for (Object value : testCases1) {
+            try {
+                ListEntryTypeValidator.validateField("test", Number.class, value);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testMapEntryTypeAnnotation() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        TestConfig config = new TestConfig();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+        Map<Object, Object> passCase1 = new HashMap<Object, Object>();
+        passCase1.put("aaa", 5);
+        passCase1.put("bbb", 6);
+        passCase1.put("ccc", 7);
+        passCases.add(passCase1);
+        passCases.add(null);
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        Map<Object, Object> failCase1 = new HashMap<Object, Object>();
+        failCase1.put("aaa", 5);
+        failCase1.put(5, 6);
+        failCase1.put("ccc", 7);
+        Map<Object, Object> failCase2 = new HashMap<Object, Object>();
+        failCase2.put("aaa", "str");
+        failCase2.put("bbb", 6);
+        failCase2.put("ccc", 7);
+
+        failCases.add(failCase1);
+        failCases.add(failCase2);
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testMapEntryCustomAnnotation() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        TestConfig config = new TestConfig();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+        Map<Object, Object> passCase1 = new HashMap<Object, Object>();
+        passCase1.put("aaa", 5);
+        passCase1.put("bbb", 100);
+        passCase1.put("ccc", Integer.MAX_VALUE);
+        passCases.add(passCase1);
+        passCases.add(null);
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG_2, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        Map<Object, Object> failCase1 = new HashMap<Object, Object>();
+        failCase1.put("aaa", 5);
+        failCase1.put(5, 6);
+        failCase1.put("ccc", 7);
+        Map<Object, Object> failCase2 = new HashMap<Object, Object>();
+        failCase2.put("aaa", "str");
+        failCase2.put("bbb", 6);
+        failCase2.put("ccc", 7);
+        Map<Object, Object> failCase3 = new HashMap<Object, Object>();
+        failCase3.put("aaa", -1);
+        failCase3.put("bbb", 6);
+        failCase3.put("ccc", 7);
+        Map<Object, Object> failCase4 = new HashMap<Object, Object>();
+        failCase4.put("aaa", 1);
+        failCase4.put("bbb", 6);
+        failCase4.put("ccc", 7.4);
+
+        failCases.add(failCase1);
+        failCases.add(failCase2);
+        failCases.add(failCase3);
+        failCases.add(failCase4);
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG_2, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testListEntryTypeAnnotation() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        TestConfig config = new TestConfig();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+        Object[] passCase1 = {1, 5.0, -0.01, 0, Integer.MAX_VALUE, Double.MIN_VALUE};
+        Object[] passCase2 = {1};
+        passCases.add(Arrays.asList(passCase1));
+        passCases.add(Arrays.asList(passCase2));
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG_3, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        Object[] failCase1 = {1, 5.0, -0.01, 0, "aaa"};
+        Object[] failCase2 = {"aaa"};
+        failCases.add(failCase1);
+        failCases.add(failCase2);
+        failCases.add(1);
+        failCases.add("b");
+        failCases.add(null);
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG_3, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void testListEntryCustomAnnotation() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        TestConfig config = new TestConfig();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+        Object[] passCase1 = {1, 5.0, 0.01, Double.MAX_VALUE};
+        Object[] passCase2 = {1};
+        passCases.add(Arrays.asList(passCase1));
+        passCases.add(Arrays.asList(passCase2));
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG_4, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        Object[] failCase1 = {1, 5.0, -0.01, 3.0};
+        Object[] failCase2 = {1, 5.0, -0.01, 1};
+        Object[] failCase3 = {"aaa", "bbb", "aaa"};
+        Object[] failCase4 = {1, 5.0, null, 1};
+        Object[] failCase5 = {1, 5.0, 0, 1};
+
+        failCases.add(Arrays.asList(failCase1));
+        failCases.add(Arrays.asList(failCase2));
+        failCases.add(Arrays.asList(failCase3));
+        failCases.add(Arrays.asList(failCase4));
+        failCases.add(Arrays.asList(failCase5));
+        failCases.add(1);
+        failCases.add("b");
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG_4, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void TestAcceptedStrings() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        TestConfig config = new TestConfig();
+        String[] passCases = {"aaa", "bbb", "ccc"};
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG_5, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        String[] failCases = {"aa", "bb", "cc", "abc", "a", "b", "c", ""};
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG_5, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+    }
+
+    @Test
+    public void TestImpersonationAclUserEntryValidator() throws InvocationTargetException, NoSuchMethodException, NoSuchFieldException, InstantiationException, IllegalAccessException {
+        TestConfig config = new TestConfig();
+        Collection<Object> passCases = new LinkedList<Object>();
+        Collection<Object> failCases = new LinkedList<Object>();
+
+        Map<String, Map<String, List<String>>> passCase1 = new HashMap<String, Map<String, List<String>>>();
+        Map<String, List<String>> passCase1_hostsAndGroups = new HashMap<String, List<String>>();
+        String[] hosts = {"host.1", "host.2", "host.3"};
+        passCase1_hostsAndGroups.put("hosts", Arrays.asList(hosts));
+        String[] groups = {"group.1", "group.2", "group.3"};
+        passCase1_hostsAndGroups.put("groups", Arrays.asList(groups));
+        passCase1.put("jerry", passCase1_hostsAndGroups);
+        passCases.add(passCase1);
+
+        for (Object value : passCases) {
+            config.put(TestConfig.TEST_MAP_CONFIG_6, value);
+            ConfigValidation.validateFields(config, TestConfig.class);
+        }
+
+        Map<String, Map<String, List<String>>> failCase1 = new HashMap<String, Map<String, List<String>>>();
+        Map<String, List<String>> failCase1_hostsAndGroups = new HashMap<String, List<String>>();
+        String[] failhosts = {"host.1", "host.2", "host.3"};
+        failCase1_hostsAndGroups.put("hosts", Arrays.asList(hosts));
+        failCase1.put("jerry", failCase1_hostsAndGroups);
+
+
+        Map<String, Map<String, List<String>>> failCase2 = new HashMap<String, Map<String, List<String>>>();
+        Map<String, List<String>> failCase2_hostsAndGroups = new HashMap<String, List<String>>();
+        String[] failgroups = {"group.1", "group.2", "group.3"};
+        failCase2_hostsAndGroups.put("groups", Arrays.asList(groups));
+        failCase2.put("jerry", failCase2_hostsAndGroups);
+
+        failCases.add(failCase1);
+        failCases.add(failCase2);
+        failCases.add("stuff");
+        failCases.add(5);
+
+        for (Object value : failCases) {
+            try {
+                config.put(TestConfig.TEST_MAP_CONFIG_6, value);
+                ConfigValidation.validateFields(config, TestConfig.class);
+                Assert.fail("Expected Exception not Thrown for value: " + value);
+            } catch (IllegalArgumentException Ex) {
+            }
+        }
+
+
+
+    }
+
+    public class TestConfig extends HashMap<String, Object> {
+        @isMapEntryType(keyType = String.class, valueType = Integer.class)
+        public static final String TEST_MAP_CONFIG = "test.map.config";
+
+        @isMapEntryCustom(
+                keyValidatorClasses = {StringValidator.class},
+                valueValidatorClasses = {PositiveNumberValidator.class, IntegerValidator.class})
+        public static final String TEST_MAP_CONFIG_2 = "test.map.config.2";
+
+        @isListEntryType(type = Number.class)
+        @NotNull
+        public static final String TEST_MAP_CONFIG_3 = "test.map.config.3";
+
+        @isListEntryCustom(
+                entryValidatorClasses = {PositiveNumberValidator.class, NotNullValidator.class})
+        @isNoDuplicateInList
+        public static final String TEST_MAP_CONFIG_4 = "test.map.config.4";
+
+        @isString(acceptedValues = {"aaa", "bbb", "ccc"})
+        public static final String TEST_MAP_CONFIG_5 = "test.map.config.5";
+
+        @isMapEntryCustom(keyValidatorClasses = {StringValidator.class}, valueValidatorClasses = {ImpersonationAclUserEntryValidator.class})
+        public static final String TEST_MAP_CONFIG_6 = "test.map.config.6";
+    }
+}
diff --git a/storm-core/test/jvm/backtype/storm/metric/internal/CountStatAndMetricTest.java b/storm-core/test/jvm/backtype/storm/metric/internal/CountStatAndMetricTest.java
new file mode 100644
index 0000000..7fa8087
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/metric/internal/CountStatAndMetricTest.java
@@ -0,0 +1,86 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Map;
+import java.util.HashMap;
+
+import org.junit.Test;
+import junit.framework.TestCase;
+import static org.junit.Assert.*;
+
+/**
+ * Unit test for CountStatAndMetric
+ */
+public class CountStatAndMetricTest extends TestCase {
+    final long TEN_MIN = 10 * 60 * 1000;
+    final long THIRTY_SEC = 30 * 1000;
+    final long THREE_HOUR = 3 * 60 * 60 * 1000;
+    final long ONE_DAY = 24 * 60 * 60 * 1000;
+
+    @Test
+    public void testBasic() {
+        long time = 0l;
+        CountStatAndMetric count = new CountStatAndMetric(10, time);
+        while (time < TEN_MIN) {
+            //For this part of the test we interleve the differnt rotation types.
+            count.incBy(50);
+            time += THIRTY_SEC/2;
+            count.rotateSched(time);
+            count.incBy(50);
+            time += THIRTY_SEC/2;
+            assertEquals(100l, ((Long)count.getValueAndReset(time)).longValue());
+        }
+
+        long val = 100 * TEN_MIN/THIRTY_SEC;
+        Map<String, Long> expected = new HashMap<String, Long>();
+        expected.put("600", val);
+        expected.put("10800", val);
+        expected.put("86400", val);
+        expected.put(":all-time", val);
+        assertEquals(expected, count.getTimeCounts(time));
+
+        while (time < THREE_HOUR) {
+            count.incBy(100);
+            time += THIRTY_SEC;
+            assertEquals(100l, ((Long)count.getValueAndReset(time)).longValue());
+        }
+
+        val = 100 * THREE_HOUR/THIRTY_SEC;
+        expected = new HashMap<String, Long>();
+        expected.put("600", 100 * TEN_MIN/THIRTY_SEC);
+        expected.put("10800", val);
+        expected.put("86400", val);
+        expected.put(":all-time", val);
+        assertEquals(expected, count.getTimeCounts(time));
+
+        while (time < ONE_DAY) {
+            count.incBy(100);
+            time += THIRTY_SEC;
+            assertEquals(100l, ((Long)count.getValueAndReset(time)).longValue());
+        }
+
+        val = 100 * ONE_DAY/THIRTY_SEC;
+        expected = new HashMap<String, Long>();
+        expected.put("600", 100 * TEN_MIN/THIRTY_SEC);
+        expected.put("10800", 100 * THREE_HOUR/THIRTY_SEC);
+        expected.put("86400", val);
+        expected.put(":all-time", val);
+        assertEquals(expected, count.getTimeCounts(time));
+    }
+}
diff --git a/storm-core/test/jvm/backtype/storm/metric/internal/LatencyStatAndMetricTest.java b/storm-core/test/jvm/backtype/storm/metric/internal/LatencyStatAndMetricTest.java
new file mode 100644
index 0000000..3b83d95
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/metric/internal/LatencyStatAndMetricTest.java
@@ -0,0 +1,83 @@
+/**
+ * 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.metric.internal;
+
+import java.util.Map;
+import java.util.HashMap;
+
+import org.junit.Test;
+import junit.framework.TestCase;
+import static org.junit.Assert.*;
+
+/**
+ * Unit test for LatencyStatAndMetric
+ */
+public class LatencyStatAndMetricTest extends TestCase {
+    final long TEN_MIN = 10 * 60 * 1000;
+    final long THIRTY_SEC = 30 * 1000;
+    final long THREE_HOUR = 3 * 60 * 60 * 1000;
+    final long ONE_DAY = 24 * 60 * 60 * 1000;
+
+    @Test
+    public void testBasic() {
+        long time = 0l;
+        LatencyStatAndMetric lat = new LatencyStatAndMetric(10, time);
+        while (time < TEN_MIN) {
+            lat.record(100);
+            time += THIRTY_SEC;
+            assertEquals(100.0, ((Double)lat.getValueAndReset(time)).doubleValue(), 0.01);
+        }
+
+        Map<String, Double> found = lat.getTimeLatAvg(time);
+        assertEquals(4, found.size());
+        assertEquals(100.0, found.get("600").doubleValue(), 0.01);
+        assertEquals(100.0, found.get("10800").doubleValue(), 0.01);
+        assertEquals(100.0, found.get("86400").doubleValue(), 0.01);
+        assertEquals(100.0, found.get(":all-time").doubleValue(), 0.01);
+
+        while (time < THREE_HOUR) {
+            lat.record(200);
+            time += THIRTY_SEC;
+            assertEquals(200.0, ((Double)lat.getValueAndReset(time)).doubleValue(), 0.01);
+        }
+
+        double expected = ((100.0 * TEN_MIN/THIRTY_SEC) + (200.0 * (THREE_HOUR - TEN_MIN)/THIRTY_SEC)) /
+                          (THREE_HOUR/THIRTY_SEC);
+        found = lat.getTimeLatAvg(time);
+        assertEquals(4, found.size());
+        assertEquals(200.0, found.get("600").doubleValue(), 0.01); //flushed the buffers completely
+        assertEquals(expected, found.get("10800").doubleValue(), 0.01);
+        assertEquals(expected, found.get("86400").doubleValue(), 0.01);
+        assertEquals(expected, found.get(":all-time").doubleValue(), 0.01);
+
+        while (time < ONE_DAY) {
+            lat.record(300);
+            time += THIRTY_SEC;
+            assertEquals(300.0, ((Double)lat.getValueAndReset(time)).doubleValue(), 0.01);
+        }
+
+        expected = ((100.0 * TEN_MIN/THIRTY_SEC) + (200.0 * (THREE_HOUR - TEN_MIN)/THIRTY_SEC) + (300.0 * (ONE_DAY - THREE_HOUR)/THIRTY_SEC)) /
+                          (ONE_DAY/THIRTY_SEC);
+        found = lat.getTimeLatAvg(time);
+        assertEquals(4, found.size());
+        assertEquals(300.0, found.get("600").doubleValue(), 0.01); //flushed the buffers completely
+        assertEquals(300.0, found.get("10800").doubleValue(), 0.01);
+        assertEquals(expected, found.get("86400").doubleValue(), 0.01);
+        assertEquals(expected, found.get(":all-time").doubleValue(), 0.01);
+    }
+}
diff --git a/storm-core/test/jvm/backtype/storm/metric/internal/RateTrackerTest.java b/storm-core/test/jvm/backtype/storm/metric/internal/RateTrackerTest.java
new file mode 100644
index 0000000..debb922
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/metric/internal/RateTrackerTest.java
@@ -0,0 +1,94 @@
+/**
+ * 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.metric.internal;
+
+import org.junit.Test;
+import junit.framework.TestCase;
+import static org.junit.Assert.*;
+
+/**
+ * Unit test for RateTracker
+ */
+public class RateTrackerTest extends TestCase {
+
+    @Test
+    public void testExactRate() {
+        //This test is in two phases.  The first phase fills up the 10 buckets with 10 tuples each
+        // We purposely simulate a 1 second bucket size so the rate will always be 10 per second.
+        final long interval = 1000l;
+        long time = 0l;
+        RateTracker rt = new RateTracker(10000, 10, time);
+        double [] expected = new double[] {10.0, 10.0, 10.0, 10.0, 10.0, 10.0, 10.0, 10.0, 10.0, 10.0};
+        for (int i = 0; i < expected.length; i++) {
+            double exp = expected[i];
+            rt.notify(10);
+            time += interval;
+            double actual = rt.reportRate(time);
+            rt.forceRotate(1, interval);
+            assertEquals("Expected rate on iteration "+i+" is wrong.", exp, actual, 0.00001);
+        }
+        //In the second part of the test the rate doubles to 20 per second but the rate tracker
+        // increases its result slowly as we push the 10 tuples per second buckets out and relpace them
+        // with 20 tuples per second. 
+        expected = new double[] {11.0, 12.0, 13.0, 14.0, 15.0, 16.0, 17.0, 18.0, 19.0, 20.0};
+        for (int i = 0; i < expected.length; i++) {
+            double exp = expected[i];
+            rt.notify(20);
+            time += interval;
+            double actual = rt.reportRate(time);
+            rt.forceRotate(1, interval);
+            assertEquals("Expected rate on iteration "+i+" is wrong.", exp, actual, 0.00001);
+        }
+    }
+
+
+    @Test
+    public void testEclipsedAllWindows() {
+        long time = 0;
+        RateTracker rt = new RateTracker(10000, 10, time);
+        rt.notify(10);
+        rt.forceRotate(10, 1000l);
+        assertEquals(0.0, rt.reportRate(10000l), 0.00001);
+    }
+
+    @Test
+    public void testEclipsedOneWindow() {
+        long time = 0;
+        RateTracker rt = new RateTracker(10000, 10, time);
+        rt.notify(1);
+        double r1 = rt.reportRate(1000l);
+        rt.forceRotate(1, 1000l);
+        rt.notify(1);
+        double r2 = rt.reportRate(2000l);
+
+        assertEquals(r1, r2, 0.00001);
+    }
+
+    @Test
+    public void testEclipsedNineWindows() {
+        long time = 0;
+        RateTracker rt = new RateTracker(10000, 10, time);
+        rt.notify(1);
+        double r1 = rt.reportRate(1000);
+        rt.forceRotate(9, 1000);
+        rt.notify(9);
+        double r2 = rt.reportRate(10000);
+
+        assertEquals(r1, r2, 0.00001);
+    }
+}
diff --git a/storm-core/test/jvm/backtype/storm/nimbus/InMemoryTopologyActionNotifier.java b/storm-core/test/jvm/backtype/storm/nimbus/InMemoryTopologyActionNotifier.java
new file mode 100644
index 0000000..16cac76
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/nimbus/InMemoryTopologyActionNotifier.java
@@ -0,0 +1,53 @@
+/**
+ * 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.nimbus;
+
+import java.util.Map;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.HashMap;
+
+public class InMemoryTopologyActionNotifier implements  ITopologyActionNotifierPlugin {
+
+    //static to ensure eventhough the class is created using reflection we can still get
+    //the topology to actions
+    private static final Map<String, LinkedList<String>> topologyToActions = new HashMap<>();
+
+
+    @Override
+    public void prepare(Map StormConf) {
+        //no-op
+    }
+
+    @Override
+    public synchronized void notify(String topologyName, String action) {
+        if(!topologyToActions.containsKey(topologyName)) {
+            topologyToActions.put(topologyName, new LinkedList<String>());
+        }
+        topologyToActions.get(topologyName).addLast(action);
+    }
+
+    public List<String> getTopologyActions(String topologyName) {
+        return topologyToActions.get(topologyName);
+    }
+
+    @Override
+    public void cleanup() {
+        //no-op
+    }
+}
diff --git a/storm-core/test/jvm/backtype/storm/topology/TopologyBuilderTest.java b/storm-core/test/jvm/backtype/storm/topology/TopologyBuilderTest.java
index 934bd69..c0891a6 100644
--- a/storm-core/test/jvm/backtype/storm/topology/TopologyBuilderTest.java
+++ b/storm-core/test/jvm/backtype/storm/topology/TopologyBuilderTest.java
@@ -39,6 +39,11 @@
         builder.setSpout("spout", mock(IRichSpout.class), 0);
     }
 
+    @Test(expected = IllegalArgumentException.class)
+    public void testAddWorkerHook() {
+        builder.addWorkerHook(null);
+    }
+
     // TODO enable if setStateSpout gets implemented
 //    @Test(expected = IllegalArgumentException.class)
 //    public void testSetStateSpout() {
diff --git a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueBackpressureTest.java b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueBackpressureTest.java
index 197744f..fc10aa1 100644
--- a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueBackpressureTest.java
+++ b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueBackpressureTest.java
@@ -20,9 +20,8 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
-import com.lmax.disruptor.BlockingWaitStrategy;
 import com.lmax.disruptor.EventHandler;
-import com.lmax.disruptor.MultiThreadedClaimStrategy;
+import com.lmax.disruptor.dsl.ProducerType;
 import org.junit.Assert;
 import org.junit.Test;
 import junit.framework.TestCase;
@@ -30,7 +29,6 @@
 import org.slf4j.LoggerFactory;
 
 public class DisruptorQueueBackpressureTest extends TestCase {
-
     private static final Logger LOG = LoggerFactory.getLogger(DisruptorQueueBackpressureTest.class);
 
     private final static int MESSAGES = 100;
@@ -38,7 +36,6 @@
     private final static double HIGH_WATERMARK = 0.6;
     private final static double LOW_WATERMARK = 0.2;
 
-
     @Test
     public void testBackPressureCallback() throws Exception {
 
@@ -53,7 +50,6 @@
 
         DisruptorBackpressureCallbackImpl cb = new DisruptorBackpressureCallbackImpl(queue, throttleOn, consumerCursor);
         queue.registerBackpressureCallback(cb);
-        queue.consumerStarted();
 
         for (int i = 0; i < MESSAGES; i++) {
             queue.publish(String.valueOf(i));
@@ -69,9 +65,9 @@
 
 
         Assert.assertEquals("Check the calling time of throttle on. ",
-                cb.highWaterMarkCalledPopulation, queue.getHighWaterMark());
+                queue.getHighWaterMark(), cb.highWaterMarkCalledPopulation);
         Assert.assertEquals("Checking the calling time of throttle off. ",
-                cb.lowWaterMarkCalledPopulation, queue.getLowWaterMark());
+                queue.getLowWaterMark(), cb.lowWaterMarkCalledPopulation);
     }
 
     class DisruptorBackpressureCallbackImpl implements DisruptorBackpressureCallback {
@@ -94,7 +90,7 @@
         @Override
         public void highWaterMark() throws Exception {
             if (!throttleOn.get()) {
-                highWaterMarkCalledPopulation = queue.getMetrics().population();
+                highWaterMarkCalledPopulation = queue.getMetrics().population() + queue.getMetrics().overflow();
                 throttleOn.set(true);
             }
         }
@@ -102,14 +98,13 @@
         @Override
         public void lowWaterMark() throws Exception {
              if (throttleOn.get()) {
-                 lowWaterMarkCalledPopulation = queue.getMetrics().writePos() - consumerCursor.get();
+                 lowWaterMarkCalledPopulation = queue.getMetrics().writePos() - consumerCursor.get() + queue.getMetrics().overflow();
                  throttleOn.set(false);
              }
         }
     }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, new MultiThreadedClaimStrategy(
-                queueSize), new BlockingWaitStrategy(), 10L);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L);
     }
 }
diff --git a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java
index ddc0982..2ae2d35 100644
--- a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java
+++ b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java
@@ -20,10 +20,8 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.lmax.disruptor.BlockingWaitStrategy;
 import com.lmax.disruptor.EventHandler;
-import com.lmax.disruptor.InsufficientCapacityException;
-import com.lmax.disruptor.MultiThreadedClaimStrategy;
+import com.lmax.disruptor.dsl.ProducerType;
 import org.junit.Assert;
 import org.junit.Test;
 import junit.framework.TestCase;
@@ -40,9 +38,9 @@
 
         queue.publish("FIRST");
 
-        Runnable producer = new Producer(queue, String.valueOf(i));
+        Runnable producer = new IncProducer(queue, i+100);
 
-        final AtomicReference<Object> result = new AtomicReference<Object>();
+        final AtomicReference<Object> result = new AtomicReference<>();
         Runnable consumer = new Consumer(queue, new EventHandler<Object>() {
             private boolean head = true;
 
@@ -56,79 +54,107 @@
             }
         });
 
-        run(producer, consumer);
+        run(producer, consumer, queue);
         Assert.assertEquals("We expect to receive first published message first, but received " + result.get(),
                 "FIRST", result.get());
       }
     }
    
     @Test 
-    public void testConsumerHang() throws InterruptedException {
-        final AtomicBoolean messageConsumed = new AtomicBoolean(false);
+    public void testInOrder() throws InterruptedException {
+        final AtomicBoolean allInOrder = new AtomicBoolean(true);
 
-        // Set queue length to 1, so that the RingBuffer can be easily full
-        // to trigger consumer blocking
-        DisruptorQueue queue = createQueue("consumerHang", 1);
-        Runnable producer = new Producer(queue, "msg");
+        DisruptorQueue queue = createQueue("consumerHang", 1024);
+        Runnable producer = new IncProducer(queue, 1024*1024);
         Runnable consumer = new Consumer(queue, new EventHandler<Object>() {
+            long _expected = 0;
             @Override
             public void onEvent(Object obj, long sequence, boolean endOfBatch)
                     throws Exception {
-                messageConsumed.set(true);
+                if (_expected != ((Number)obj).longValue()) {
+                    allInOrder.set(false);
+                    System.out.println("Expected "+_expected+" but got "+obj);
+                }
+                _expected++;
             }
         });
 
-        run(producer, consumer);
-        Assert.assertTrue("disruptor message is never consumed due to consumer thread hangs",
-                messageConsumed.get());
+        run(producer, consumer, queue, 1000, 1);
+        Assert.assertTrue("Messages delivered out of order",
+                allInOrder.get());
+    }
+
+    @Test 
+    public void testInOrderBatch() throws InterruptedException {
+        final AtomicBoolean allInOrder = new AtomicBoolean(true);
+
+        DisruptorQueue queue = createQueue("consumerHang", 10, 1024);
+        Runnable producer = new IncProducer(queue, 1024*1024);
+        Runnable consumer = new Consumer(queue, new EventHandler<Object>() {
+            long _expected = 0;
+            @Override
+            public void onEvent(Object obj, long sequence, boolean endOfBatch)
+                    throws Exception {
+                if (_expected != ((Number)obj).longValue()) {
+                    allInOrder.set(false);
+                    System.out.println("Expected "+_expected+" but got "+obj);
+                }
+                _expected++;
+            }
+        });
+
+        run(producer, consumer, queue, 1000, 1);
+        Assert.assertTrue("Messages delivered out of order",
+                allInOrder.get());
     }
 
 
-    private void run(Runnable producer, Runnable consumer)
+    private void run(Runnable producer, Runnable consumer, DisruptorQueue queue)
+            throws InterruptedException {
+        run(producer, consumer, queue, 10, PRODUCER_NUM);
+    }
+
+    private void run(Runnable producer, Runnable consumer, DisruptorQueue queue, int sleepMs, int producerNum)
             throws InterruptedException {
 
-        Thread[] producerThreads = new Thread[PRODUCER_NUM];
-        for (int i = 0; i < PRODUCER_NUM; i++) {
+        Thread[] producerThreads = new Thread[producerNum];
+        for (int i = 0; i < producerNum; i++) {
             producerThreads[i] = new Thread(producer);
             producerThreads[i].start();
         }
         
         Thread consumerThread = new Thread(consumer);
         consumerThread.start();
-        Thread.sleep(10);
-        for (int i = 0; i < PRODUCER_NUM; i++) {
+        Thread.sleep(sleepMs);
+        for (int i = 0; i < producerNum; i++) {
             producerThreads[i].interrupt();
         }
-        consumerThread.interrupt();
         
-        for (int i = 0; i < PRODUCER_NUM; i++) {
+        for (int i = 0; i < producerNum; i++) {
             producerThreads[i].join(TIMEOUT);
             assertFalse("producer "+i+" is still alive", producerThreads[i].isAlive());
         }
+        queue.haltWithInterrupt();
         consumerThread.join(TIMEOUT);
         assertFalse("consumer is still alive", consumerThread.isAlive());
     }
 
-    private static class Producer implements Runnable {
-        private String msg;
+    private static class IncProducer implements Runnable {
         private DisruptorQueue queue;
+        private long _max;
 
-        Producer(DisruptorQueue queue, String msg) {
-            this.msg = msg;
+        IncProducer(DisruptorQueue queue, long max) {
             this.queue = queue;
+            this._max = max;
         }
 
         @Override
         public void run() {
-            try {
-                while (true) {
-                    queue.publish(msg, false);
-                }
-            } catch (InsufficientCapacityException e) {
-                return;
+            for (long i = 0; i < _max && !(Thread.currentThread().isInterrupted()); i++) {
+                queue.publish(i);
             }
         }
-    };
+    }
 
     private static class Consumer implements Runnable {
         private EventHandler handler;
@@ -141,19 +167,21 @@
 
         @Override
         public void run() {
-            queue.consumerStarted();
             try {
                 while(true) {
                     queue.consumeBatchWhenAvailable(handler);
                 }
-            }catch(RuntimeException e) {
+            } catch(RuntimeException e) {
                 //break
             }
         }
-    };
+    }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, new MultiThreadedClaimStrategy(
-                queueSize), new BlockingWaitStrategy(), 10L);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L);
+    }
+
+    private static DisruptorQueue createQueue(String name, int batchSize, int queueSize) {
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, batchSize, 1L);
     }
 }
diff --git a/examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java b/storm-core/test/jvm/backtype/storm/utils/MockTupleHelpers.java
similarity index 82%
rename from examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java
rename to storm-core/test/jvm/backtype/storm/utils/MockTupleHelpers.java
index b253350..a78a168 100644
--- a/examples/storm-starter/test/jvm/storm/starter/tools/MockTupleHelpers.java
+++ b/storm-core/test/jvm/backtype/storm/utils/MockTupleHelpers.java
@@ -15,12 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package storm.starter.tools;
+package backtype.storm.utils;
 
 import backtype.storm.Constants;
 import backtype.storm.tuple.Tuple;
 
-import static org.mockito.Mockito.*;
+import org.mockito.Mockito;
 
 public final class MockTupleHelpers {
 
@@ -32,9 +32,9 @@
   }
 
   public static Tuple mockTuple(String componentId, String streamId) {
-    Tuple tuple = mock(Tuple.class);
-    when(tuple.getSourceComponent()).thenReturn(componentId);
-    when(tuple.getSourceStreamId()).thenReturn(streamId);
+    Tuple tuple = Mockito.mock(Tuple.class);
+    Mockito.when(tuple.getSourceComponent()).thenReturn(componentId);
+    Mockito.when(tuple.getSourceStreamId()).thenReturn(streamId);
     return tuple;
   }
 }
diff --git a/storm-core/test/jvm/backtype/storm/utils/RateTrackerTest.java b/storm-core/test/jvm/backtype/storm/utils/RateTrackerTest.java
deleted file mode 100644
index d11bbf6..0000000
--- a/storm-core/test/jvm/backtype/storm/utils/RateTrackerTest.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <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.utils;
-
-import org.junit.Assert;
-import org.junit.Test;
-import junit.framework.TestCase;
-
-/**
- * Unit test for RateTracker
- */
-public class RateTrackerTest extends TestCase {
-
-    @Test
-    public void testEclipsedAllWindows() {
-        RateTracker rt = new RateTracker(10000, 10, true);
-        rt.notify(10);
-        rt.forceUpdateSlides(10);
-        assert (rt.reportRate() == 0);
-    }
-
-    @Test
-    public void testEclipsedOneWindow() {
-        RateTracker rt = new RateTracker(10000, 10, true);
-        rt.notify(1);
-        float r1 = rt.reportRate();
-        rt.forceUpdateSlides(1);
-        rt.notify(1);
-        float r2 = rt.reportRate();
-
-        System.out.format("r1:%f, r2:%f\n", r1, r2);
-
-        assert (r1 == r2);
-    }
-
-    @Test
-    public void testEclipsedNineWindows() {
-        RateTracker rt = new RateTracker(10000, 10, true);
-        rt.notify(1);
-        float r1 = rt.reportRate();
-        rt.forceUpdateSlides(9);
-        rt.notify(9);
-        float r2 = rt.reportRate();
-
-        assert (r1 == r2);
-    }
-}
\ No newline at end of file
diff --git a/storm-core/test/jvm/backtype/storm/utils/ThriftTopologyUtilsTest.java b/storm-core/test/jvm/backtype/storm/utils/ThriftTopologyUtilsTest.java
new file mode 100644
index 0000000..6793665
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/utils/ThriftTopologyUtilsTest.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package backtype.storm.utils;
+
+import backtype.storm.generated.*;
+import backtype.storm.hooks.BaseWorkerHook;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import junit.framework.TestCase;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+public class ThriftTopologyUtilsTest extends TestCase {
+    @Test
+    public void testIsWorkerHook() {
+        Assert.assertEquals(false, ThriftTopologyUtils.isWorkerHook(StormTopology._Fields.BOLTS));
+        Assert.assertEquals(false, ThriftTopologyUtils.isWorkerHook(StormTopology._Fields.SPOUTS));
+        Assert.assertEquals(false, ThriftTopologyUtils.isWorkerHook(StormTopology._Fields.STATE_SPOUTS));
+        Assert.assertEquals(true, ThriftTopologyUtils.isWorkerHook(StormTopology._Fields.WORKER_HOOKS));
+    }
+
+    @Test
+    public void testGetComponentIdsWithWorkerHook() {
+        StormTopology stormTopology = genereateStormTopology(true);
+        Set<String> componentIds = ThriftTopologyUtils.getComponentIds(stormTopology);
+        Assert.assertEquals(
+                "We expect to get the IDs of the components sans the Worker Hook",
+                ImmutableSet.of("bolt-1", "spout-1"),
+                componentIds);
+    }
+
+    @Test
+    public void testGetComponentIdsWithoutWorkerHook() {
+        StormTopology stormTopology = genereateStormTopology(false);
+        Set<String> componentIds = ThriftTopologyUtils.getComponentIds(stormTopology);
+        Assert.assertEquals(
+                "We expect to get the IDs of the components sans the Worker Hook",
+                ImmutableSet.of("bolt-1", "spout-1"),
+                componentIds);
+    }
+
+    @Test
+    public void testGetComponentCommonWithWorkerHook() {
+        StormTopology stormTopology = genereateStormTopology(true);
+        ComponentCommon componentCommon = ThriftTopologyUtils.getComponentCommon(stormTopology, "bolt-1");
+        Assert.assertEquals(
+                "We expect to get bolt-1's common",
+                new Bolt().get_common(),
+                componentCommon);
+    }
+
+    @Test
+    public void testGetComponentCommonWithoutWorkerHook() {
+        StormTopology stormTopology = genereateStormTopology(false);
+        ComponentCommon componentCommon = ThriftTopologyUtils.getComponentCommon(stormTopology, "bolt-1");
+        Assert.assertEquals(
+                "We expect to get bolt-1's common",
+                new Bolt().get_common(),
+                componentCommon);
+    }
+
+    private StormTopology genereateStormTopology(boolean withWorkerHook) {
+        ImmutableMap<String,SpoutSpec> spouts = ImmutableMap.of("spout-1", new SpoutSpec());
+        ImmutableMap<String,Bolt> bolts = ImmutableMap.of("bolt-1", new Bolt());
+        ImmutableMap<String,StateSpoutSpec> state_spouts = ImmutableMap.of();
+
+        StormTopology stormTopology = new StormTopology(spouts, bolts, state_spouts);
+
+        if(withWorkerHook) {
+            BaseWorkerHook workerHook = new BaseWorkerHook();
+            stormTopology.add_to_worker_hooks(ByteBuffer.wrap(Utils.javaSerialize(workerHook)));
+        }
+
+        return stormTopology;
+    }
+}
diff --git a/storm-core/test/jvm/backtype/storm/windowing/WindowManagerTest.java b/storm-core/test/jvm/backtype/storm/windowing/WindowManagerTest.java
new file mode 100644
index 0000000..6531817
--- /dev/null
+++ b/storm-core/test/jvm/backtype/storm/windowing/WindowManagerTest.java
@@ -0,0 +1,250 @@
+/**
+ * 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.windowing;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static backtype.storm.topology.base.BaseWindowedBolt.Count;
+import static backtype.storm.topology.base.BaseWindowedBolt.Duration;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Unit tests for {@link WindowManager}
+ */
+public class WindowManagerTest {
+    private WindowManager<Integer> windowManager;
+    private Listener listener;
+
+    private static class Listener implements WindowLifecycleListener<Integer> {
+        List<Integer> onExpiryEvents = Collections.emptyList();
+        List<Integer> onActivationEvents = Collections.emptyList();
+        List<Integer> onActivationNewEvents = Collections.emptyList();
+        List<Integer> onActivationExpiredEvents = Collections.emptyList();
+
+        @Override
+        public void onExpiry(List<Integer> events) {
+            onExpiryEvents = events;
+        }
+
+        @Override
+        public void onActivation(List<Integer> events, List<Integer> newEvents, List<Integer> expired) {
+            onActivationEvents = events;
+            onActivationNewEvents = newEvents;
+            onActivationExpiredEvents = expired;
+        }
+
+        void clear() {
+            onExpiryEvents = Collections.emptyList();
+            onActivationEvents = Collections.emptyList();
+            onActivationNewEvents = Collections.emptyList();
+            onActivationExpiredEvents = Collections.emptyList();
+        }
+    }
+
+    @Before
+    public void setUp() {
+        listener = new Listener();
+        windowManager = new WindowManager<>(listener);
+    }
+
+    @After
+    public void tearDown() {
+        windowManager.shutdown();
+    }
+
+    @Test
+    public void testCountBasedWindow() throws Exception {
+        windowManager.setWindowLength(new Count(5));
+        windowManager.setSlidingInterval(new Count(2));
+        windowManager.add(1);
+        windowManager.add(2);
+        // nothing expired yet
+        assertTrue(listener.onExpiryEvents.isEmpty());
+        assertEquals(seq(1, 2), listener.onActivationEvents);
+        assertEquals(seq(1, 2), listener.onActivationNewEvents);
+        assertTrue(listener.onActivationExpiredEvents.isEmpty());
+        windowManager.add(3);
+        windowManager.add(4);
+        // nothing expired yet
+        assertTrue(listener.onExpiryEvents.isEmpty());
+        assertEquals(seq(1, 4), listener.onActivationEvents);
+        assertEquals(seq(3, 4), listener.onActivationNewEvents);
+        assertTrue(listener.onActivationExpiredEvents.isEmpty());
+        windowManager.add(5);
+        windowManager.add(6);
+        // 1 expired
+        assertEquals(seq(1), listener.onExpiryEvents);
+        assertEquals(seq(2, 6), listener.onActivationEvents);
+        assertEquals(seq(5, 6), listener.onActivationNewEvents);
+        assertEquals(seq(1), listener.onActivationExpiredEvents);
+        listener.clear();
+        windowManager.add(7);
+        // nothing expires until threshold is hit
+        assertTrue(listener.onExpiryEvents.isEmpty());
+        windowManager.add(8);
+        // 1 expired
+        assertEquals(seq(2, 3), listener.onExpiryEvents);
+        assertEquals(seq(4, 8), listener.onActivationEvents);
+        assertEquals(seq(7, 8), listener.onActivationNewEvents);
+        assertEquals(seq(2, 3), listener.onActivationExpiredEvents);
+    }
+
+    @Test
+    public void testExpireThreshold() throws Exception {
+        int threshold = WindowManager.EXPIRE_EVENTS_THRESHOLD;
+        int windowLength = 5;
+        windowManager.setWindowLength(new Count(5));
+        windowManager.setSlidingInterval(new Duration(1, TimeUnit.HOURS));
+        for (int i : seq(1, 5)) {
+            windowManager.add(i);
+        }
+        // nothing expired yet
+        assertTrue(listener.onExpiryEvents.isEmpty());
+        for (int i : seq(6, 10)) {
+            windowManager.add(i);
+        }
+        for (int i : seq(11, threshold)) {
+            windowManager.add(i);
+        }
+        // window should be compacted and events should be expired.
+        assertEquals(seq(1, threshold - windowLength), listener.onExpiryEvents);
+    }
+
+
+    @Test
+    public void testTimeBasedWindow() throws Exception {
+        windowManager.setWindowLength(new Duration(1, TimeUnit.SECONDS));
+        windowManager.setSlidingInterval(new Duration(100, TimeUnit.MILLISECONDS));
+        long now = System.currentTimeMillis();
+
+        // add with past ts
+        for (int i : seq(1, 50)) {
+            windowManager.add(i, now - 1000);
+        }
+
+        // add with current ts
+        for (int i : seq(51, WindowManager.EXPIRE_EVENTS_THRESHOLD)) {
+            windowManager.add(i, now);
+        }
+        // first 50 should have expired due to expire events threshold
+        assertEquals(50, listener.onExpiryEvents.size());
+
+        // add more events with past ts
+        for (int i : seq(WindowManager.EXPIRE_EVENTS_THRESHOLD + 1, WindowManager.EXPIRE_EVENTS_THRESHOLD + 100)) {
+            windowManager.add(i, now - 1000);
+        }
+        // wait for time trigger
+        Thread.sleep(120);
+
+        // 100 events with past ts should expire
+        assertEquals(100, listener.onExpiryEvents.size());
+        assertEquals(seq(WindowManager.EXPIRE_EVENTS_THRESHOLD + 1, WindowManager.EXPIRE_EVENTS_THRESHOLD + 100),
+                     listener.onExpiryEvents);
+        List<Integer> activationsEvents = seq(51, WindowManager.EXPIRE_EVENTS_THRESHOLD);
+        assertEquals(seq(51, WindowManager.EXPIRE_EVENTS_THRESHOLD), listener.onActivationEvents);
+        assertEquals(seq(51, WindowManager.EXPIRE_EVENTS_THRESHOLD), listener.onActivationNewEvents);
+        // activation expired list should contain even the ones expired due to EXPIRE_EVENTS_THRESHOLD
+        List<Integer> expiredList = seq(1, 50);
+        expiredList.addAll(seq(WindowManager.EXPIRE_EVENTS_THRESHOLD + 1, WindowManager.EXPIRE_EVENTS_THRESHOLD + 100));
+        assertEquals(expiredList, listener.onActivationExpiredEvents);
+
+        listener.clear();
+        // add more events with current ts
+        List<Integer> newEvents = seq(WindowManager.EXPIRE_EVENTS_THRESHOLD + 101, WindowManager.EXPIRE_EVENTS_THRESHOLD + 200);
+        for (int i : newEvents) {
+            windowManager.add(i, now);
+        }
+        activationsEvents.addAll(newEvents);
+        // wait for time trigger
+        Thread.sleep(120);
+        assertTrue(listener.onExpiryEvents.isEmpty());
+        assertEquals(activationsEvents, listener.onActivationEvents);
+        assertEquals(newEvents, listener.onActivationNewEvents);
+
+    }
+
+
+    @Test
+    public void testTimeBasedWindowExpiry() throws Exception {
+        windowManager.setWindowLength(new Duration(100, TimeUnit.MILLISECONDS));
+        windowManager.setSlidingInterval(new Duration(60, TimeUnit.MILLISECONDS));
+        // add 10 events
+        for (int i : seq(1, 10)) {
+            windowManager.add(i);
+        }
+        Thread.sleep(70);
+        assertEquals(seq(1, 10), listener.onActivationEvents);
+        assertTrue(listener.onActivationExpiredEvents.isEmpty());
+        listener.clear();
+        // wait so all events expire
+        Thread.sleep(70);
+        assertEquals(seq(1, 10), listener.onActivationExpiredEvents);
+        assertTrue(listener.onActivationEvents.isEmpty());
+        listener.clear();
+        Thread.sleep(70);
+        assertTrue(listener.onActivationExpiredEvents.isEmpty());
+        assertTrue(listener.onActivationEvents.isEmpty());
+
+    }
+
+    @Test
+    public void testTumblingWindow() throws Exception {
+        windowManager.setWindowLength(new Count(3));
+        windowManager.setSlidingInterval(new Count(3));
+        windowManager.add(1);
+        windowManager.add(2);
+        // nothing expired yet
+        assertTrue(listener.onExpiryEvents.isEmpty());
+        windowManager.add(3);
+        assertTrue(listener.onExpiryEvents.isEmpty());
+        assertEquals(seq(1, 3), listener.onActivationEvents);
+        assertTrue(listener.onActivationExpiredEvents.isEmpty());
+        assertEquals(seq(1, 3), listener.onActivationNewEvents);
+
+        listener.clear();
+        windowManager.add(4);
+        windowManager.add(5);
+        windowManager.add(6);
+
+        assertEquals(seq(1, 3), listener.onExpiryEvents);
+        assertEquals(seq(4, 6), listener.onActivationEvents);
+        assertEquals(seq(1, 3), listener.onActivationExpiredEvents);
+        assertEquals(seq(4, 6), listener.onActivationNewEvents);
+
+    }
+
+    private List<Integer> seq(int start) {
+        return seq(start, start);
+    }
+
+    private List<Integer> seq(int start, int stop) {
+        List<Integer> ints = new ArrayList<>();
+        for (int i = start; i <= stop; i++) {
+            ints.add(i);
+        }
+        return ints;
+    }
+}
\ No newline at end of file
diff --git a/storm-core/test/jvm/storm/trident/TestTridentTopology.java b/storm-core/test/jvm/storm/trident/TestTridentTopology.java
new file mode 100644
index 0000000..0b0218a
--- /dev/null
+++ b/storm-core/test/jvm/storm/trident/TestTridentTopology.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package storm.trident;
+
+import backtype.storm.generated.Bolt;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import org.junit.Assert;
+import org.junit.Test;
+import storm.trident.operation.builtin.Count;
+import storm.trident.operation.builtin.Sum;
+import storm.trident.testing.FixedBatchSpout;
+import storm.trident.testing.Split;
+import storm.trident.testing.StringLength;
+
+import java.util.Map;
+import java.util.Set;
+
+public class TestTridentTopology {
+
+    private StormTopology buildTopology() {
+        FixedBatchSpout spout = new FixedBatchSpout(new Fields("sentence"), 3,
+                new Values("the cow jumped over the moon"),
+                new Values("the man went to the store and bought some candy"),
+                new Values("four score and seven years ago"),
+                new Values("how many apples can you eat"));
+        spout.setCycle(true);
+
+        TridentTopology topology = new TridentTopology();
+        topology.newStream("spout", spout)
+                //no name
+                .each(new Fields("sentence"), new Split(), new Fields("word"))
+                .partitionBy(new Fields("word"))
+                .name("abc")
+                .each(new Fields("word"), new StringLength(), new Fields("length"))
+                .partitionBy(new Fields("length"))
+                .name("def")
+                .aggregate(new Fields("length"), new Count(), new Fields("count"))
+                .partitionBy(new Fields("count"))
+                .name("ghi")
+                .aggregate(new Fields("count"), new Sum(), new Fields("sum"));
+        return topology.build();
+    }
+
+    @Test
+    public void testGenBoltId() {
+        Set<String> pre = null;
+        for (int i = 0; i < 100; i++) {
+            StormTopology topology = buildTopology();
+            Map<String, Bolt> cur = topology.get_bolts();
+            System.out.println(cur.keySet());
+            if (pre != null) {
+                Assert.assertTrue("bold id not consistent with group name", pre.equals(cur.keySet()));
+            }
+            pre = cur.keySet();
+        }
+    }
+
+}
diff --git a/storm-dist/binary/LICENSE b/storm-dist/binary/LICENSE
index b52a6b5..693cfaf 100644
--- a/storm-dist/binary/LICENSE
+++ b/storm-dist/binary/LICENSE
@@ -568,3 +568,32 @@
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
 SOFTWARE.
 
+-----------------------------------------------------------------------
+
+For statistic image
+
+storm-core/src/ui/public/images/statistic.png
+
+Copyright (c) 2015 Github, Inc.
+
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.  IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
diff --git a/storm-dist/binary/src/main/assembly/binary.xml b/storm-dist/binary/src/main/assembly/binary.xml
index 9f6c8f7..cfd4fab 100644
--- a/storm-dist/binary/src/main/assembly/binary.xml
+++ b/storm-dist/binary/src/main/assembly/binary.xml
@@ -30,38 +30,6 @@
             <useProjectArtifact>false</useProjectArtifact>
             <outputDirectory>lib</outputDirectory>
             <unpack>false</unpack>
-            <excludes>
-                <exclude>org.apache.thrift:*</exclude>
-                <exclude>io.netty:netty</exclude>
-                <exclude>com.google.guava:guava</exclude>
-                <exclude>org.apache.httpcomponents:http*</exclude>
-                <exclude>org.apache.zookeeper:zookeeper</exclude>
-                <exclude>org.apache.curator:*</exclude>
-                <exclude>org.apache.httpcomponents:http*</exclude>
-                <exclude>org.apache.zookeeper:zookeeper</exclude>
-                <exclude>org.apache.curator:*</exclude>
-                <exclude>com.twitter:carbonite</exclude>
-                <exclude>com.twitter:chill-java</exclude>
-                <exclude>org.objenesis:objenesis</exclude>
-                <exclude>org.tukaani:xz</exclude>
-                <exclude>org.yaml:snakeyaml</exclude>
-                <exclude>org.jgrapht:jgrapht-core</exclude>
-                <exclude>commons-httpclient:commons-httpclient</exclude>
-                <exclude>org.apache.commons:commons-compress</exclude>
-                <exclude>org.apache.commons:commons-exec</exclude>
-                <exclude>commons-io:commons-io</exclude>
-                <exclude>commons-codec:commons-codec</exclude>
-                <exclude>commons-fileupload:commons-fileupload</exclude>
-                <exclude>commons-lang:commons-lang</exclude>
-                <exclude>com.googlecode.json-simple:json-simple</exclude>
-                <exclude>org.clojure:math.numeric-tower</exclude>
-                <exclude>org.clojure:tools.cli</exclude>
-                <exclude>org.clojure:tools.macro</exclude>
-                <exclude>joda-time:joda-time</exclude>
-                <exclude>org.eclipse.jetty:*</exclude>
-                <exclude>com.fasterxml.jackson.core:*</exclude>
-                <exclude>com.fasterxml.jackson.dataformat:*</exclude>
-            </excludes>
         </dependencySet>
     </dependencySets>
 
@@ -349,11 +317,6 @@
             <outputDirectory>/</outputDirectory>
         </file>
         <file>
-            <source>${project.basedir}/../../DISCLAIMER</source>
-            <outputDirectory>/</outputDirectory>
-        </file>
-
-        <file>
             <source>${project.basedir}/../../CHANGELOG.md</source>
             <outputDirectory>/</outputDirectory>
         </file>