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-